diff --git a/docs/changelog/118122.yaml b/docs/changelog/118122.yaml new file mode 100644 index 0000000000000..ca27cc94a7cb2 --- /dev/null +++ b/docs/changelog/118122.yaml @@ -0,0 +1,5 @@ +pr: 118122 +summary: "ES|QL: Partial result on demand for async queries" +area: ES|QL +type: enhancement +issues: [] diff --git a/docs/reference/esql/esql-across-clusters.asciidoc b/docs/reference/esql/esql-across-clusters.asciidoc index db266fafde9d6..d3ad58d95d972 100644 --- a/docs/reference/esql/esql-across-clusters.asciidoc +++ b/docs/reference/esql/esql-across-clusters.asciidoc @@ -215,6 +215,7 @@ Which returns: { "is_running": false, "took": 42, <1> + "is_partial": false, <7> "columns" : [ { "name" : "COUNT(http.response.status_code)", @@ -280,8 +281,9 @@ Which returns: <2> This section of counters shows all possible cluster search states and how many cluster searches are currently in that state. The clusters can have one of the following statuses: *running*, *successful* (searches on all shards were successful), *skipped* (the search -failed on a cluster marked with `skip_unavailable`=`true`) or *failed* (the search -failed on a cluster marked with `skip_unavailable`=`false`). +failed on a cluster marked with `skip_unavailable`=`true`), *failed* (the search +failed on a cluster marked with `skip_unavailable`=`false`) or **partial** (the search was +<> before finishing). <3> The `_clusters/details` section shows metadata about the search on each cluster. <4> If you included indices from the local cluster you sent the request to in your {ccs}, it is identified as "(local)". @@ -290,6 +292,8 @@ which clusters have slower response times than others. <6> The shard details for the search on that cluster, including a count of shards that were skipped due to the can-match phase results. Shards are skipped when they cannot have any matching data and therefore are not included in the full ES|QL query. +<7> The `is_partial` field is set to `true` if the search has partial results for any reason, +for example if it was interrupted before finishing using the <>. The cross-cluster metadata can be used to determine whether any data came back from a cluster. @@ -319,6 +323,7 @@ Which returns: { "is_running": false, "took": 55, + "is_partial": false, "columns": [ ... // not shown ], diff --git a/docs/reference/esql/esql-apis.asciidoc b/docs/reference/esql/esql-apis.asciidoc index 157f4e4357e78..633a202c9dc3d 100644 --- a/docs/reference/esql/esql-apis.asciidoc +++ b/docs/reference/esql/esql-apis.asciidoc @@ -17,6 +17,7 @@ overview of {esql} and related tutorials, see <>. * <> * <> * <> +* <> include::esql-query-api.asciidoc[] @@ -26,3 +27,5 @@ include::esql-async-query-api.asciidoc[] include::esql-async-query-get-api.asciidoc[] include::esql-async-query-delete-api.asciidoc[] + +include::esql-async-query-stop-api.asciidoc[] diff --git a/docs/reference/esql/esql-async-query-api.asciidoc b/docs/reference/esql/esql-async-query-api.asciidoc index 8cb974cf6773b..c194818eb0cc7 100644 --- a/docs/reference/esql/esql-async-query-api.asciidoc +++ b/docs/reference/esql/esql-async-query-api.asciidoc @@ -170,3 +170,10 @@ API>> to get the current status and available results for the query. (Boolean) If `true`, the query request is still executing. -- + +`is_partial`:: ++ +-- +(Boolean) +If `true`, the query has partial results - for example, as a result of using the <>. +-- diff --git a/docs/reference/esql/esql-async-query-stop-api.asciidoc b/docs/reference/esql/esql-async-query-stop-api.asciidoc new file mode 100644 index 0000000000000..dba5282d224ed --- /dev/null +++ b/docs/reference/esql/esql-async-query-stop-api.asciidoc @@ -0,0 +1,49 @@ +[[esql-async-query-stop-api]] +=== {esql} async query stop API +++++ +{esql} async query stop API +++++ + +.New API reference +[sidebar] +-- +For the most up-to-date API details, refer to {api-es}/group/endpoint-esql[ES|QL APIs]. +-- + +The <> async query stop API is used to manually stop an async query. Once the stop command is issued, +the query stops processing new data and returns the results that have been already processed. Note that due to the pipelined +nature of {esql} queries, the stop operation is not immediate and may take time to return results. + +The results are returned in <> as the +<>. +If the query has been finished by the time the stop command is issued, the results are returned immediately. + +If the query processing has not finished by the time the stop command is issued, the response will have the `is_partial` +field set to `true`. + +[source,console] +---- +POST /query/async/FkpMRkJGS1gzVDRlM3g4ZzMyRGlLbkEaTXlJZHdNT09TU2VTZVBoNDM3cFZMUToxMDM=/stop +---- +// TEST[skip: no access to query ID] + +[[esql-async-query-stop-api-request]] +==== {api-request-title} + +`POST /_query/async//stop` + +[[esql-async-query-stop-api-prereqs]] +==== {api-prereq-title} + +* If the {es} {security-features} are enabled, only the authenticated user that submitted the original query request +can stop the query. + +[[esql-async-query-stop-api-path-params]] +==== {api-path-parms-title} + +``:: +(Required, string) +Identifier for the query to stop. ++ +A query ID is provided in the <>'s +response for a query that does not complete in the awaited time. diff --git a/docs/reference/esql/esql-rest.asciidoc b/docs/reference/esql/esql-rest.asciidoc index 1cd808b2aef44..c4160e1d65954 100644 --- a/docs/reference/esql/esql-rest.asciidoc +++ b/docs/reference/esql/esql-rest.asciidoc @@ -193,6 +193,7 @@ Which returns: ---- { "took": 28, + "is_partial": false, "columns": [ {"name": "author", "type": "text"}, {"name": "name", "type": "text"}, diff --git a/docs/reference/esql/multivalued-fields.asciidoc b/docs/reference/esql/multivalued-fields.asciidoc index 562ea2a2e6b4a..00d9df04a0bc4 100644 --- a/docs/reference/esql/multivalued-fields.asciidoc +++ b/docs/reference/esql/multivalued-fields.asciidoc @@ -27,6 +27,7 @@ Multivalued fields come back as a JSON array: ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "long"} @@ -78,6 +79,7 @@ And {esql} sees that removal: ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "keyword"} @@ -122,6 +124,7 @@ And {esql} also sees that: ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "long"} @@ -165,6 +168,7 @@ POST /_query ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "keyword"} @@ -198,6 +202,7 @@ POST /_query ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, ], @@ -241,6 +246,7 @@ POST /_query ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "long"}, @@ -278,6 +284,7 @@ POST /_query ---- { "took": 28, + "is_partial": false, "columns": [ { "name": "a", "type": "long"}, { "name": "b", "type": "long"}, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/esql.async_query_stop.json b/rest-api-spec/src/main/resources/rest-api-spec/api/esql.async_query_stop.json new file mode 100644 index 0000000000000..6fbdefef8b689 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/esql.async_query_stop.json @@ -0,0 +1,31 @@ +{ + "esql.async_query_stop": { + "documentation": { + "url": "https://www.elastic.co/guide/en/elasticsearch/reference/master/esql-async-query-stop-api.html", + "description": "Stops a previously submitted async query request given its ID and collects the results." + }, + "stability": "stable", + "visibility": "public", + "headers": { + "accept": [ + "application/json" + ] + }, + "url": { + "paths": [ + { + "path": "/_query/async/{id}/stop", + "methods": [ + "POST" + ], + "parts": { + "id": { + "type": "string", + "description": "The async query ID" + } + } + } + ] + } + } +} diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index cd9e711e83773..bb203a6d39935 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -169,6 +169,7 @@ static TransportVersion def(int id) { public static final TransportVersion RESOLVE_CLUSTER_NO_INDEX_EXPRESSION = def(8_829_00_0); public static final TransportVersion ML_ROLLOVER_LEGACY_INDICES = def(8_830_00_0); public static final TransportVersion ADD_INCLUDE_FAILURE_INDICES_OPTION = def(8_831_00_0); + public static final TransportVersion ESQL_RESPONSE_PARTIAL = def(8_832_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java b/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java index 4af8681bb939a..2e68c094492fe 100644 --- a/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java +++ b/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java @@ -86,7 +86,7 @@ public void skipOnAborted() { public void testSortByManyLongsSuccess() throws IOException { initManyLongs(); Response response = sortByManyLongs(500); - Map map = responseAsMap(response); + Map map = responseAsMap(response); ListMatcher columns = matchesList().item(matchesMap().entry("name", "a").entry("type", "long")) .item(matchesMap().entry("name", "b").entry("type", "long")); ListMatcher values = matchesList(); @@ -95,8 +95,7 @@ public void testSortByManyLongsSuccess() throws IOException { values = values.item(List.of(0, b)); } } - MapMatcher mapMatcher = matchesMap(); - assertMap(map, mapMatcher.entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0))); + assertResultMap(map, columns, values); } /** @@ -236,11 +235,10 @@ private StringBuilder makeSortByManyLongs(int count) { public void testGroupOnSomeLongs() throws IOException { initManyLongs(); Response resp = groupOnManyLongs(200); - Map map = responseAsMap(resp); + Map map = responseAsMap(resp); ListMatcher columns = matchesList().item(matchesMap().entry("name", "MAX(a)").entry("type", "long")); ListMatcher values = matchesList().item(List.of(9)); - MapMatcher mapMatcher = matchesMap(); - assertMap(map, mapMatcher.entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0))); + assertResultMap(map, columns, values); } /** @@ -249,11 +247,10 @@ public void testGroupOnSomeLongs() throws IOException { public void testGroupOnManyLongs() throws IOException { initManyLongs(); Response resp = groupOnManyLongs(5000); - Map map = responseAsMap(resp); + Map map = responseAsMap(resp); ListMatcher columns = matchesList().item(matchesMap().entry("name", "MAX(a)").entry("type", "long")); ListMatcher values = matchesList().item(List.of(9)); - MapMatcher mapMatcher = matchesMap(); - assertMap(map, mapMatcher.entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0))); + assertResultMap(map, columns, values); } private Response groupOnManyLongs(int count) throws IOException { @@ -279,12 +276,11 @@ private StringBuilder makeManyLongs(int count) { public void testSmallConcat() throws IOException { initSingleDocIndex(); Response resp = concat(2); - Map map = responseAsMap(resp); + Map map = responseAsMap(resp); ListMatcher columns = matchesList().item(matchesMap().entry("name", "a").entry("type", "long")) .item(matchesMap().entry("name", "str").entry("type", "keyword")); ListMatcher values = matchesList().item(List.of(1, "1".repeat(100))); - MapMatcher mapMatcher = matchesMap(); - assertMap(map, mapMatcher.entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0))); + assertResultMap(map, columns, values); } public void testHugeConcat() throws IOException { @@ -465,7 +461,7 @@ private void assertManyStrings(Response resp, int strings) throws IOException { public void testManyEval() throws IOException { initManyLongs(); Response resp = manyEval(1); - Map map = responseAsMap(resp); + Map map = responseAsMap(resp); ListMatcher columns = matchesList(); columns = columns.item(matchesMap().entry("name", "a").entry("type", "long")); columns = columns.item(matchesMap().entry("name", "b").entry("type", "long")); @@ -475,8 +471,7 @@ public void testManyEval() throws IOException { for (int i = 0; i < 20; i++) { columns = columns.item(matchesMap().entry("name", "i0" + i).entry("type", "long")); } - MapMatcher mapMatcher = matchesMap(); - assertMap(map, mapMatcher.entry("columns", columns).entry("values", hasSize(10_000)).entry("took", greaterThanOrEqualTo(0))); + assertResultMap(map, columns, hasSize(10_000)); } public void testTooManyEval() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 1538ec1b8b959..48860210da06f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -76,6 +76,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.AbstractBroadcastResponseTestCase; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.MapMatcher; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.NamedXContentRegistry; @@ -84,6 +85,7 @@ import org.elasticsearch.xcontent.XContentParserConfiguration; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xcontent.json.JsonXContent; +import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.junit.After; import org.junit.AfterClass; @@ -133,12 +135,15 @@ import static org.elasticsearch.client.RestClient.IGNORE_RESPONSE_CODES_PARAM; import static org.elasticsearch.cluster.ClusterState.VERSION_INTRODUCING_TRANSPORT_VERSIONS; import static org.elasticsearch.core.Strings.format; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.elasticsearch.test.MapMatcher.matchesMap; import static org.elasticsearch.test.rest.TestFeatureService.ALL_FEATURES; import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.in; import static org.hamcrest.Matchers.notNullValue; @@ -2627,4 +2632,46 @@ public static Request newXContentRequest(HttpMethod method, String endpoint, ToX addXContentBody(request, body); return request; } + + protected static MapMatcher getResultMatcher(boolean includeMetadata, boolean includePartial) { + MapMatcher mapMatcher = matchesMap(); + if (includeMetadata) { + mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); + } + // Older version may not have is_partial + if (includePartial) { + mapMatcher = mapMatcher.entry("is_partial", false); + } + return mapMatcher; + } + + /** + * Create empty result matcher from result, taking into account all metadata items. + */ + protected static MapMatcher getResultMatcher(Map result) { + return getResultMatcher(result.containsKey("took"), result.containsKey("is_partial")); + } + + /** + * Match result columns and values, with default matchers for metadata. + */ + protected static void assertResultMap(Map result, Matcher columnMatcher, Matcher valuesMatcher) { + assertMap(result, getResultMatcher(result).entry("columns", columnMatcher).entry("values", valuesMatcher)); + } + + protected static void assertResultMap(Map result, Object columnMatcher, Object valuesMatcher) { + assertMap(result, getResultMatcher(result).entry("columns", columnMatcher).entry("values", valuesMatcher)); + } + + /** + * Match result columns and values, with default matchers for metadata. + */ + protected static void assertResultMap( + Map result, + MapMatcher mapMatcher, + Matcher columnMatcher, + Matcher valuesMatcher + ) { + assertMap(result, mapMatcher.entry("columns", columnMatcher).entry("values", valuesMatcher)); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncStopRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncStopRequest.java new file mode 100644 index 0000000000000..7113cbca279d3 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncStopRequest.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.core.async; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +/** + * Request for TransportEsqlAsyncStopAction action. + */ +public class AsyncStopRequest extends ActionRequest { + private final String id; + + /** + * Creates a new request + * + * @param id The id of the search progress request. + */ + public AsyncStopRequest(String id) { + this.id = id; + } + + public AsyncStopRequest(StreamInput in) throws IOException { + super(in); + this.id = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(id); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + /** + * Returns the id of the async search. + */ + public String getId() { + return id; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AsyncStopRequest request = (AsyncStopRequest) o; + return Objects.equals(id, request.id); + } + + @Override + public int hashCode() { + return Objects.hash(id); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/esql/EsqlAsyncActionNames.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/esql/EsqlAsyncActionNames.java index 81ab54fc2db5f..7555db8fc85e6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/esql/EsqlAsyncActionNames.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/esql/EsqlAsyncActionNames.java @@ -12,4 +12,5 @@ */ public class EsqlAsyncActionNames { public static final String ESQL_ASYNC_GET_RESULT_ACTION_NAME = "indices:data/read/esql/async/get"; + public static final String ESQL_ASYNC_STOP_ACTION_NAME = "indices:data/read/esql/async/stop"; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java index 62cc4daf5fde5..d1a5d1757bc90 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java @@ -76,6 +76,7 @@ public final class ExchangeService extends AbstractLifecycleComponent { private final BlockFactory blockFactory; private final Map sinks = ConcurrentCollections.newConcurrentMap(); + private final Map exchangeSources = ConcurrentCollections.newConcurrentMap(); public ExchangeService(Settings settings, ThreadPool threadPool, String executorName, BlockFactory blockFactory) { this.threadPool = threadPool; @@ -172,6 +173,32 @@ public static void openExchange( ); } + /** + * Remember the exchange source handler for the given session ID. + * This can be used for async/stop requests. + */ + public void addExchangeSourceHandler(String sessionId, ExchangeSourceHandler sourceHandler) { + exchangeSources.put(sessionId, sourceHandler); + } + + public ExchangeSourceHandler removeExchangeSourceHandler(String sessionId) { + return exchangeSources.remove(sessionId); + } + + /** + * Finishes the session early, i.e., before all sources are finished. + * It is called by async/stop API and should be called on the node that coordinates the async request. + * It will close all sources and return the results - unlike cancel, this does not discard the results. + */ + public void finishSessionEarly(String sessionId, ActionListener listener) { + ExchangeSourceHandler exchangeSource = removeExchangeSourceHandler(sessionId); + if (exchangeSource != null) { + exchangeSource.finishEarly(false, listener); + } else { + listener.onResponse(null); + } + } + private static class OpenExchangeRequest extends TransportRequest { private final String sessionId; private final int exchangeBuffer; diff --git a/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlAsyncSecurityIT.java b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlAsyncSecurityIT.java index b45ef45914985..0a6f73ee648df 100644 --- a/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlAsyncSecurityIT.java +++ b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlAsyncSecurityIT.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Locale; +import java.util.Map; import static org.elasticsearch.core.TimeValue.timeValueNanos; import static org.hamcrest.Matchers.allOf; @@ -50,8 +51,9 @@ protected Response runESQLCommand(String user, String command) throws IOExceptio } @Override - protected MapMatcher responseMatcher() { - return super.responseMatcher().entry("is_running", equalTo(false)).entry("id", allOf(notNullValue(), instanceOf(String.class))); + protected MapMatcher responseMatcher(Map result) { + return super.responseMatcher(result).entry("is_running", equalTo(false)) + .entry("id", allOf(notNullValue(), instanceOf(String.class))); } @Override diff --git a/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java index d8e3b0cccf394..7d96c400cb659 100644 --- a/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java +++ b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java @@ -165,8 +165,8 @@ public void indexDocuments() throws IOException { } } - protected MapMatcher responseMatcher() { - return matchesMap(); + protected MapMatcher responseMatcher(Map result) { + return getResultMatcher(result); } public void testAllowedIndices() throws Exception { @@ -182,10 +182,7 @@ public void testAllowedIndices() throws Exception { Response resp = runESQLCommand(user, "from index-user1 | stats sum=sum(value)"); assertOK(resp); Map responseMap = entityAsMap(resp); - MapMatcher mapMatcher = responseMatcher(); - if (responseMap.get("took") != null) { - mapMatcher = mapMatcher.entry("took", ((Integer) responseMap.get("took")).intValue()); - } + MapMatcher mapMatcher = responseMatcher(responseMap); MapMatcher matcher = mapMatcher.entry("columns", List.of(Map.of("name", "sum", "type", "double"))) .entry("values", List.of(List.of(43.0d))); assertMap(responseMap, matcher); @@ -195,10 +192,7 @@ public void testAllowedIndices() throws Exception { Response resp = runESQLCommand(user, "from index-user2 | stats sum=sum(value)"); assertOK(resp); Map responseMap = entityAsMap(resp); - MapMatcher mapMatcher = responseMatcher(); - if (responseMap.get("took") != null) { - mapMatcher = mapMatcher.entry("took", ((Integer) responseMap.get("took")).intValue()); - } + MapMatcher mapMatcher = responseMatcher(responseMap); MapMatcher matcher = mapMatcher.entry("columns", List.of(Map.of("name", "sum", "type", "double"))) .entry("values", List.of(List.of(72.0d))); assertMap(responseMap, matcher); @@ -208,10 +202,7 @@ public void testAllowedIndices() throws Exception { Response resp = runESQLCommand("metadata1_read2", "from " + index + " | stats sum=sum(value)"); assertOK(resp); Map responseMap = entityAsMap(resp); - MapMatcher mapMatcher = responseMatcher(); - if (responseMap.get("took") != null) { - mapMatcher = mapMatcher.entry("took", ((Integer) responseMap.get("took")).intValue()); - } + MapMatcher mapMatcher = responseMatcher(responseMap); MapMatcher matcher = mapMatcher.entry("columns", List.of(Map.of("name", "sum", "type", "double"))) .entry("values", List.of(List.of(72.0d))); assertMap(responseMap, matcher); @@ -226,9 +217,10 @@ public void testAliases() throws Exception { ); assertOK(resp); Map responseMap = entityAsMap(resp); - MapMatcher matcher = responseMatcher().entry("took", ((Integer) responseMap.get("took")).intValue()) - .entry("columns", List.of(Map.of("name", "sum", "type", "double"), Map.of("name", "index", "type", "keyword"))) - .entry("values", List.of(List.of(72.0d, "index-user2"))); + MapMatcher matcher = responseMatcher(responseMap).entry( + "columns", + List.of(Map.of("name", "sum", "type", "double"), Map.of("name", "index", "type", "keyword")) + ).entry("values", List.of(List.of(72.0d, "index-user2"))); assertMap(responseMap, matcher); } } @@ -238,16 +230,14 @@ public void testAliasFilter() throws Exception { Response resp = runESQLCommand("alias_user1", "from " + index + " METADATA _index" + "| KEEP _index, org, value | LIMIT 10"); assertOK(resp); Map responseMap = entityAsMap(resp); - MapMatcher matcher = responseMatcher().entry("took", ((Integer) responseMap.get("took")).intValue()) - .entry( - "columns", - List.of( - Map.of("name", "_index", "type", "keyword"), - Map.of("name", "org", "type", "keyword"), - Map.of("name", "value", "type", "double") - ) + MapMatcher matcher = responseMatcher(responseMap).entry( + "columns", + List.of( + Map.of("name", "_index", "type", "keyword"), + Map.of("name", "org", "type", "keyword"), + Map.of("name", "value", "type", "double") ) - .entry("values", List.of(List.of("index-user1", "sales", 31.0d))); + ).entry("values", List.of(List.of("index-user1", "sales", 31.0d))); assertMap(responseMap, matcher); } } diff --git a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClustersIT.java b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClustersIT.java index 6e43d40a3005a..b838d8ae284a4 100644 --- a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClustersIT.java +++ b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClustersIT.java @@ -37,12 +37,8 @@ import java.util.stream.Stream; import static org.elasticsearch.test.MapMatcher.assertMap; -import static org.elasticsearch.test.MapMatcher.matchesMap; import static org.elasticsearch.xpack.esql.ccq.Clusters.REMOTE_CLUSTER_NAME; -import static org.hamcrest.Matchers.any; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.*; @ThreadLeakFilters(filters = TestClustersThreadFilter.class) public class MultiClustersIT extends ESRestTestCase { @@ -159,6 +155,17 @@ private Map runEsql(RestEsqlTestCase.RequestObjectBuilder reques } } + private void assertResultMap(boolean includeCCSMetadata, Map result, C columns, V values, boolean remoteOnly) { + MapMatcher mapMatcher = getResultMatcher(ccsMetadataAvailable(), result.containsKey("is_partial")); + if (includeCCSMetadata) { + mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); + } + assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); + if (includeCCSMetadata) { + assertClusterDetailsMap(result, remoteOnly); + } + } + public void testCount() throws Exception { { boolean includeCCSMetadata = includeCCSMetadata(); @@ -166,17 +173,7 @@ public void testCount() throws Exception { var columns = List.of(Map.of("name", "c", "type", "long")); var values = List.of(List.of(localDocs.size() + remoteDocs.size())); - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, false); - } + assertResultMap(includeCCSMetadata, result, columns, values, false); } { boolean includeCCSMetadata = includeCCSMetadata(); @@ -184,17 +181,7 @@ public void testCount() throws Exception { var columns = List.of(Map.of("name", "c", "type", "long")); var values = List.of(List.of(remoteDocs.size())); - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, true); - } + assertResultMap(includeCCSMetadata, result, columns, values, true); } } @@ -207,17 +194,7 @@ public void testUngroupedAggs() throws Exception { var values = List.of(List.of(Math.toIntExact(sum))); // check all sections of map except _cluster/details - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, false); - } + assertResultMap(includeCCSMetadata, result, columns, values, false); } { boolean includeCCSMetadata = includeCCSMetadata(); @@ -226,17 +203,7 @@ public void testUngroupedAggs() throws Exception { long sum = remoteDocs.stream().mapToLong(d -> d.data).sum(); var values = List.of(List.of(Math.toIntExact(sum))); - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, true); - } + assertResultMap(includeCCSMetadata, result, columns, values, true); } { assumeTrue("requires ccs metadata", ccsMetadataAvailable()); @@ -245,15 +212,7 @@ public void testUngroupedAggs() throws Exception { long sum = remoteDocs.stream().mapToLong(d -> d.data).sum(); var values = List.of(List.of(Math.toIntExact(sum))); - MapMatcher mapMatcher = matchesMap(); - assertMap( - result, - mapMatcher.entry("columns", columns) - .entry("values", values) - .entry("took", greaterThanOrEqualTo(0)) - .entry("_clusters", any(Map.class)) - ); - assertClusterDetailsMap(result, true); + assertResultMap(true, result, columns, values, true); } } @@ -325,17 +284,7 @@ public void testGroupedAggs() throws Exception { .map(e -> List.of(Math.toIntExact(e.getValue()), e.getKey())) .toList(); - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, false); - } + assertResultMap(includeCCSMetadata, result, columns, values, false); } { boolean includeCCSMetadata = includeCCSMetadata(); @@ -353,17 +302,7 @@ public void testGroupedAggs() throws Exception { .toList(); // check all sections of map except _clusters/details - MapMatcher mapMatcher = matchesMap(); - if (includeCCSMetadata) { - mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); - } - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); - if (includeCCSMetadata) { - assertClusterDetailsMap(result, true); - } + assertResultMap(includeCCSMetadata, result, columns, values, true); } } @@ -378,11 +317,8 @@ public void testIndexPattern() throws Exception { Map result = run("FROM " + indexPattern + " | STATS c = COUNT(*)", false); var columns = List.of(Map.of("name", "c", "type", "long")); var values = List.of(List.of(localDocs.size() + remoteDocs.size())); - MapMatcher mapMatcher = matchesMap(); - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); + + assertResultMap(false, result, columns, values, false); } { String indexPattern = randomFrom("*:test-remote-index", "*:test-remote-*", "*:test-*"); @@ -390,11 +326,7 @@ public void testIndexPattern() throws Exception { var columns = List.of(Map.of("name", "c", "type", "long")); var values = List.of(List.of(remoteDocs.size())); - MapMatcher mapMatcher = matchesMap(); - if (ccsMetadataAvailable()) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - assertMap(result, mapMatcher.entry("columns", columns).entry("values", values)); + assertResultMap(false, result, columns, values, false); } } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java index daa986f8a6014..7a8511c8b0d4a 100644 --- a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java +++ b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java @@ -44,7 +44,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; @@ -80,12 +79,10 @@ public void testBasicEsql() throws IOException { builder.pragmas(Settings.builder().put("data_partitioning", "shard").build()); } Map result = runEsql(builder); - assertEquals(3, result.size()); + Map colA = Map.of("name", "avg(value)", "type", "double"); - assertEquals(List.of(colA), result.get("columns")); - assertEquals(List.of(List.of(499.5d)), result.get("values")); + assertResultMap(result, List.of(colA), List.of(List.of(499.5d))); assertTrue(result.containsKey("took")); - assertThat(((Number) result.get("took")).longValue(), greaterThanOrEqualTo(0L)); } public void testInvalidPragma() throws IOException { @@ -118,11 +115,8 @@ public void testDoNotLogWithInfo() throws IOException { setLoggingLevel("INFO"); RequestObjectBuilder builder = requestObjectBuilder().query("ROW DO_NOT_LOG_ME = 1"); Map result = runEsql(builder); - assertEquals(3, result.size()); - assertThat(((Integer) result.get("took")).intValue(), greaterThanOrEqualTo(0)); Map colA = Map.of("name", "DO_NOT_LOG_ME", "type", "integer"); - assertEquals(List.of(colA), result.get("columns")); - assertEquals(List.of(List.of(1)), result.get("values")); + assertResultMap(result, List.of(colA), List.of(List.of(1))); for (int i = 0; i < cluster.getNumNodes(); i++) { try (InputStream log = cluster.getNodeLog(i, LogType.SERVER)) { Streams.readAllLines(log, line -> assertThat(line, not(containsString("DO_NOT_LOG_ME")))); @@ -138,11 +132,8 @@ public void testDoLogWithDebug() throws IOException { setLoggingLevel("DEBUG"); RequestObjectBuilder builder = requestObjectBuilder().query("ROW DO_LOG_ME = 1"); Map result = runEsql(builder); - assertEquals(3, result.size()); - assertThat(((Integer) result.get("took")).intValue(), greaterThanOrEqualTo(0)); Map colA = Map.of("name", "DO_LOG_ME", "type", "integer"); - assertEquals(List.of(colA), result.get("columns")); - assertEquals(List.of(List.of(1)), result.get("values")); + assertResultMap(result, List.of(colA), List.of(List.of(1))); boolean[] found = new boolean[] { false }; for (int i = 0; i < cluster.getNumNodes(); i++) { try (InputStream log = cluster.getNodeLog(i, LogType.SERVER)) { @@ -289,13 +280,11 @@ public void testProfile() throws IOException { builder.pragmas(Settings.builder().put("data_partitioning", "shard").build()); } Map result = runEsql(builder); - MapMatcher mapMatcher = matchesMap(); - assertMap( + assertResultMap( result, - mapMatcher.entry("columns", matchesList().item(matchesMap().entry("name", "AVG(value)").entry("type", "double"))) - .entry("values", List.of(List.of(499.5d))) - .entry("profile", matchesMap().entry("drivers", instanceOf(List.class))) - .entry("took", greaterThanOrEqualTo(0)) + getResultMatcher(result).entry("profile", matchesMap().entry("drivers", instanceOf(List.class))), + matchesList().item(matchesMap().entry("name", "AVG(value)").entry("type", "double")), + equalTo(List.of(List.of(499.5d))) ); List> signatures = new ArrayList<>(); @@ -372,24 +361,19 @@ public void testInlineStatsProfile() throws IOException { } Map result = runEsql(builder); - MapMatcher mapMatcher = matchesMap(); ListMatcher values = matchesList(); for (int i = 0; i < 1000; i++) { values = values.item(matchesList().item("2020-12-12T00:00:00.000Z").item("value" + i).item("value" + i).item(i).item(499.5)); } - assertMap( + assertResultMap( result, - mapMatcher.entry( - "columns", - matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date")) - .item(matchesMap().entry("name", "test").entry("type", "text")) - .item(matchesMap().entry("name", "test.keyword").entry("type", "keyword")) - .item(matchesMap().entry("name", "value").entry("type", "long")) - .item(matchesMap().entry("name", "AVG(value)").entry("type", "double")) - ) - .entry("values", values) - .entry("profile", matchesMap().entry("drivers", instanceOf(List.class))) - .entry("took", greaterThanOrEqualTo(0)) + getResultMatcher(result).entry("profile", matchesMap().entry("drivers", instanceOf(List.class))), + matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date")) + .item(matchesMap().entry("name", "test").entry("type", "text")) + .item(matchesMap().entry("name", "test.keyword").entry("type", "keyword")) + .item(matchesMap().entry("name", "value").entry("type", "long")) + .item(matchesMap().entry("name", "AVG(value)").entry("type", "double")), + values ); List> signatures = new ArrayList<>(); @@ -483,20 +467,15 @@ public void testForceSleepsProfile() throws IOException { for (int group2 = 0; group2 < 10; group2++) { expectedValues.add(List.of(1.0, 1, 1, 0, group2)); } - MapMatcher mapMatcher = matchesMap(); - assertMap( + assertResultMap( result, - mapMatcher.entry( - "columns", - matchesList().item(matchesMap().entry("name", "AVG(value)").entry("type", "double")) - .item(matchesMap().entry("name", "MAX(value)").entry("type", "long")) - .item(matchesMap().entry("name", "MIN(value)").entry("type", "long")) - .item(matchesMap().entry("name", "group1").entry("type", "long")) - .item(matchesMap().entry("name", "group2").entry("type", "long")) - ) - .entry("values", expectedValues) - .entry("profile", matchesMap().entry("drivers", instanceOf(List.class))) - .entry("took", greaterThanOrEqualTo(0)) + getResultMatcher(result).entry("profile", matchesMap().entry("drivers", instanceOf(List.class))), + matchesList().item(matchesMap().entry("name", "AVG(value)").entry("type", "double")) + .item(matchesMap().entry("name", "MAX(value)").entry("type", "long")) + .item(matchesMap().entry("name", "MIN(value)").entry("type", "long")) + .item(matchesMap().entry("name", "group1").entry("type", "long")) + .item(matchesMap().entry("name", "group2").entry("type", "long")), + equalTo(expectedValues) ); @SuppressWarnings("unchecked") diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java index 813354db697e1..a320cbfa459d1 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java @@ -23,7 +23,6 @@ import org.elasticsearch.logging.Logger; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ListMatcher; -import org.elasticsearch.test.MapMatcher; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentType; @@ -51,7 +50,6 @@ import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.runEsqlSync; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; /** * Creates indices with many different mappings and fetches values from them to make sure @@ -304,11 +302,7 @@ public void testFlattenedUnsupported() throws IOException { {"flattened": {"a": "foo"}}"""); Map result = runEsql("FROM test* | LIMIT 2"); - assertMap( - result, - matchesMapWithOptionalTook(result.get("took")).entry("columns", List.of(columnInfo("flattened", "unsupported"))) - .entry("values", List.of(matchesList().item(null))) - ); + assertResultMap(result, List.of(columnInfo("flattened", "unsupported")), List.of(matchesList().item(null))); } public void testEmptyMapping() throws IOException { @@ -322,7 +316,7 @@ public void testEmptyMapping() throws IOException { // TODO this is broken in main too // Map result = runEsqlSync(new RestEsqlTestCase.RequestObjectBuilder().query("FROM test* | LIMIT 2")); - // assertMap( + // assertResultMap( // result, // matchesMap().entry("columns", List.of(columnInfo("f", "unsupported"), columnInfo("f.raw", "unsupported"))) // .entry("values", List.of(matchesList().item(null).item(null))) @@ -345,13 +339,10 @@ public void testEmptyMapping() throws IOException { public void testTextFieldWithKeywordSubfield() throws IOException { String value = randomAlphaOfLength(20); Map result = new Test("text").storeAndDocValues(randomBoolean(), null).sub("raw", keywordTest()).roundTrip(value); - - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("text_field", "text"), columnInfo("text_field.raw", "keyword")) - ).entry("values", List.of(matchesList().item(value).item(value))) + List.of(columnInfo("text_field", "text"), columnInfo("text_field.raw", "keyword")), + List.of(matchesList().item(value).item(value)) ); } @@ -372,12 +363,10 @@ public void testTextFieldWithIntegerSubfield() throws IOException { int value = randomInt(); Map result = textTest().sub("int", intTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("text_field", "text"), columnInfo("text_field.int", "integer")) - ).entry("values", List.of(matchesList().item(Integer.toString(value)).item(value))) + List.of(columnInfo("text_field", "text"), columnInfo("text_field.int", "integer")), + List.of(matchesList().item(Integer.toString(value)).item(value)) ); } @@ -398,12 +387,10 @@ public void testTextFieldWithIntegerSubfieldMalformed() throws IOException { String value = randomAlphaOfLength(5); Map result = textTest().sourceMode(SourceMode.DEFAULT).sub("int", intTest().ignoreMalformed(true)).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("text_field", "text"), columnInfo("text_field.int", "integer")) - ).entry("values", List.of(matchesList().item(value).item(null))) + List.of(columnInfo("text_field", "text"), columnInfo("text_field.int", "integer")), + List.of(matchesList().item(value).item(null)) ); } @@ -424,12 +411,10 @@ public void testTextFieldWithIpSubfield() throws IOException { String value = NetworkAddress.format(randomIp(randomBoolean())); Map result = textTest().sub("ip", ipTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("text_field", "text"), columnInfo("text_field.ip", "ip")) - ).entry("values", List.of(matchesList().item(value).item(value))) + List.of(columnInfo("text_field", "text"), columnInfo("text_field.ip", "ip")), + List.of(matchesList().item(value).item(value)) ); } @@ -450,12 +435,10 @@ public void testTextFieldWithIpSubfieldMalformed() throws IOException { String value = randomAlphaOfLength(10); Map result = textTest().sourceMode(SourceMode.DEFAULT).sub("ip", ipTest().ignoreMalformed(true)).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("text_field", "text"), columnInfo("text_field.ip", "ip")) - ).entry("values", List.of(matchesList().item(value).item(null))) + List.of(columnInfo("text_field", "text"), columnInfo("text_field.ip", "ip")), + List.of(matchesList().item(value).item(null)) ); } @@ -477,12 +460,10 @@ public void testIntFieldWithTextOrKeywordSubfield() throws IOException { boolean text = randomBoolean(); Map result = intTest().sub("str", text ? textTest() : keywordTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.str", text ? "text" : "keyword")) - ).entry("values", List.of(matchesList().item(value).item(Integer.toString(value)))) + List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.str", text ? "text" : "keyword")), + List.of(matchesList().item(value).item(Integer.toString(value))) ); } @@ -504,12 +485,10 @@ public void testIntFieldWithTextOrKeywordSubfieldMalformed() throws IOException boolean text = randomBoolean(); Map result = intTest().forceIgnoreMalformed().sub("str", text ? textTest() : keywordTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.str", text ? "text" : "keyword")) - ).entry("values", List.of(matchesList().item(null).item(value))) + List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.str", text ? "text" : "keyword")), + List.of(matchesList().item(null).item(value)) ); } @@ -531,12 +510,10 @@ public void testIpFieldWithTextOrKeywordSubfield() throws IOException { boolean text = randomBoolean(); Map result = ipTest().sub("str", text ? textTest() : keywordTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("ip_field", "ip"), columnInfo("ip_field.str", text ? "text" : "keyword")) - ).entry("values", List.of(matchesList().item(value).item(value))) + List.of(columnInfo("ip_field", "ip"), columnInfo("ip_field.str", text ? "text" : "keyword")), + List.of(matchesList().item(value).item(value)) ); } @@ -558,12 +535,10 @@ public void testIpFieldWithTextOrKeywordSubfieldMalformed() throws IOException { boolean text = randomBoolean(); Map result = ipTest().forceIgnoreMalformed().sub("str", text ? textTest() : keywordTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("ip_field", "ip"), columnInfo("ip_field.str", text ? "text" : "keyword")) - ).entry("values", List.of(matchesList().item(null).item(value))) + List.of(columnInfo("ip_field", "ip"), columnInfo("ip_field.str", text ? "text" : "keyword")), + List.of(matchesList().item(null).item(value)) ); } @@ -585,12 +560,10 @@ public void testIntFieldWithByteSubfield() throws IOException { byte value = randomByte(); Map result = intTest().sub("byte", byteTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.byte", "integer")) - ).entry("values", List.of(matchesList().item((int) value).item((int) value))) + List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.byte", "integer")), + List.of(matchesList().item((int) value).item((int) value)) ); } @@ -614,12 +587,10 @@ public void testIntFieldWithByteSubfieldTooBig() throws IOException { .sub("byte", byteTest().ignoreMalformed(true)) .roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.byte", "integer")) - ).entry("values", List.of(matchesList().item(value).item(null))) + List.of(columnInfo("integer_field", "integer"), columnInfo("integer_field.byte", "integer")), + List.of(matchesList().item(value).item(null)) ); } @@ -641,23 +612,13 @@ public void testByteFieldWithIntSubfield() throws IOException { byte value = randomByte(); Map result = byteTest().sub("int", intTest()).roundTrip(value); - assertMap( + assertResultMap( result, - matchesMapWithOptionalTook(result.get("took")).entry( - "columns", - List.of(columnInfo("byte_field", "integer"), columnInfo("byte_field.int", "integer")) - ).entry("values", List.of(matchesList().item((int) value).item((int) value))) + List.of(columnInfo("byte_field", "integer"), columnInfo("byte_field.int", "integer")), + List.of(matchesList().item((int) value).item((int) value)) ); } - static MapMatcher matchesMapWithOptionalTook(Object tookTimeValue) { - MapMatcher mapMatcher = matchesMap(); - if (tookTimeValue instanceof Number) { - mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); - } - return mapMatcher; - } - /** *
      * "byte_field": {
@@ -676,12 +637,10 @@ public void testByteFieldWithIntSubfieldTooBig() throws IOException {
         int value = randomValueOtherThanMany((Integer v) -> (Byte.MIN_VALUE <= v) && (v <= Byte.MAX_VALUE), ESTestCase::randomInt);
         Map result = byteTest().forceIgnoreMalformed().sub("int", intTest()).roundTrip(value);
 
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("byte_field", "integer"), columnInfo("byte_field.int", "integer"))
-            ).entry("values", List.of(matchesList().item(null).item(value)))
+            List.of(columnInfo("byte_field", "integer"), columnInfo("byte_field.int", "integer")),
+            List.of(matchesList().item(null).item(value))
         );
     }
 
@@ -708,11 +667,7 @@ public void testIncompatibleTypes() throws IOException {
             {"f": 1}""");
 
         Map result = runEsql("FROM test*");
-        assertMap(
-            result,
-            matchesMapWithOptionalTook(result.get("took")).entry("columns", List.of(columnInfo("f", "unsupported")))
-                .entry("values", List.of(matchesList().item(null), matchesList().item(null)))
-        );
+        assertResultMap(result, List.of(columnInfo("f", "unsupported")), List.of(matchesList().item(null), matchesList().item(null)));
         ResponseException e = expectThrows(ResponseException.class, () -> runEsql("FROM test* | SORT f | LIMIT 3"));
         String err = EntityUtils.toString(e.getResponse().getEntity());
         assertThat(
@@ -746,12 +701,10 @@ public void testDistinctInEachIndex() throws IOException {
             {"other": "o2"}""");
 
         Map result = runEsql("FROM test* | SORT file, other");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("file", "keyword"), columnInfo("other", "keyword"))
-            ).entry("values", List.of(matchesList().item("f1").item(null), matchesList().item(null).item("o2")))
+            List.of(columnInfo("file", "keyword"), columnInfo("other", "keyword")),
+            List.of(matchesList().item("f1").item(null), matchesList().item(null).item("o2"))
         );
     }
 
@@ -812,12 +765,10 @@ public void testMergeKeywordAndObject() throws IOException {
         );
 
         Map result = runEsql("FROM test* | SORT file.raw | LIMIT 2");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("file", "unsupported"), columnInfo("file.raw", "keyword"))
-            ).entry("values", List.of(matchesList().item(null).item("o2"), matchesList().item(null).item(null)))
+            List.of(columnInfo("file", "unsupported"), columnInfo("file.raw", "keyword")),
+            List.of(matchesList().item(null).item("o2"), matchesList().item(null).item(null))
         );
     }
 
@@ -859,12 +810,10 @@ public void testPropagateUnsupportedToSubFields() throws IOException {
         assertThat(err, containsString("Cannot use field [f.raw] with unsupported type [ip_range]"));
 
         Map result = runEsql("FROM test* | LIMIT 2");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("f", "unsupported"), columnInfo("f.raw", "unsupported"))
-            ).entry("values", List.of(matchesList().item(null).item(null)))
+            List.of(columnInfo("f", "unsupported"), columnInfo("f.raw", "unsupported")),
+            List.of(matchesList().item(null).item(null))
         );
     }
 
@@ -924,12 +873,10 @@ public void testMergeUnsupportedAndObject() throws IOException {
         assertThat(err, containsString("Cannot use field [f.raw] with unsupported type [ip_range]"));
 
         Map result = runEsql("FROM test* | LIMIT 2");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("f", "unsupported"), columnInfo("f.raw", "unsupported"))
-            ).entry("values", List.of(matchesList().item(null).item(null), matchesList().item(null).item(null)))
+            List.of(columnInfo("f", "unsupported"), columnInfo("f.raw", "unsupported")),
+            List.of(matchesList().item(null).item(null), matchesList().item(null).item(null))
         );
     }
 
@@ -961,11 +908,7 @@ public void testIntegerDocValuesConflict() throws IOException {
             {"emp_no": 2}""");
 
         Map result = runEsql("FROM test* | SORT emp_no | LIMIT 2");
-        assertMap(
-            result,
-            matchesMapWithOptionalTook(result.get("took")).entry("columns", List.of(columnInfo("emp_no", "integer")))
-                .entry("values", List.of(matchesList().item(1), matchesList().item(2)))
-        );
+        assertResultMap(result, List.of(columnInfo("emp_no", "integer")), List.of(matchesList().item(1), matchesList().item(2)));
     }
 
     /**
@@ -1007,11 +950,7 @@ public void testLongIntegerConflict() throws IOException {
         );
 
         Map result = runEsql("FROM test* | LIMIT 2");
-        assertMap(
-            result,
-            matchesMapWithOptionalTook(result.get("took")).entry("columns", List.of(columnInfo("emp_no", "unsupported")))
-                .entry("values", List.of(matchesList().item(null), matchesList().item(null)))
-        );
+        assertResultMap(result, List.of(columnInfo("emp_no", "unsupported")), List.of(matchesList().item(null), matchesList().item(null)));
     }
 
     /**
@@ -1053,11 +992,7 @@ public void testIntegerShortConflict() throws IOException {
         );
 
         Map result = runEsql("FROM test* | LIMIT 2");
-        assertMap(
-            result,
-            matchesMapWithOptionalTook(result.get("took")).entry("columns", List.of(columnInfo("emp_no", "unsupported")))
-                .entry("values", List.of(matchesList().item(null), matchesList().item(null)))
-        );
+        assertResultMap(result, List.of(columnInfo("emp_no", "unsupported")), List.of(matchesList().item(null), matchesList().item(null)));
     }
 
     /**
@@ -1095,7 +1030,7 @@ public void testTypeConflictInObject() throws IOException {
             {"foo": {"emp_no": "cat"}}""");
 
         Map result = runEsql("FROM test* | LIMIT 3");
-        assertMap(result, matchesMap().entry("columns", List.of(columnInfo("foo.emp_no", "unsupported"))).extraOk());
+        assertMap(result, getResultMatcher(result).entry("columns", List.of(columnInfo("foo.emp_no", "unsupported"))).extraOk());
 
         ResponseException e = expectThrows(ResponseException.class, () -> runEsql("FROM test* | SORT foo.emp_no | LIMIT 3"));
         String err = EntityUtils.toString(e.getResponse().getEntity());
@@ -1147,35 +1082,29 @@ public void testOneNestedSubField_AndSameNameSupportedField() throws IOException
             """);
 
         Map result = runEsql("FROM test");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text"))
-            ).entry("values", Collections.EMPTY_LIST)
+            List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text")),
+            Collections.EMPTY_LIST
         );
 
         index("test", """
             {"Responses.process.pid": 123,"process.parent.command_line":"run.bat"}""");
 
         result = runEsql("FROM test");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text"))
-            ).entry("values", List.of(matchesList().item("run.bat").item("run.bat")))
+            List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text")),
+            List.of(matchesList().item("run.bat").item("run.bat"))
         );
 
         result = runEsql("""
             FROM test | where process.parent.command_line == "run.bat"
             """);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text"))
-            ).entry("values", List.of(matchesList().item("run.bat").item("run.bat")))
+            List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text")),
+            List.of(matchesList().item("run.bat").item("run.bat"))
         );
 
         ResponseException e = expectThrows(ResponseException.class, () -> runEsql("FROM test | SORT Responses.process.pid"));
@@ -1235,23 +1164,19 @@ public void testOneNestedSubField_AndSameNameSupportedField_TwoIndices() throws
             {"process.parent.command_line":"run.bat"}""");
 
         Map result = runEsql("FROM test* | SORT process.parent.command_line ASC NULLS FIRST");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text"))
-            ).entry("values", List.of(matchesList().item(null).item(null), matchesList().item("run.bat").item("run.bat")))
+            List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text")),
+            List.of(matchesList().item(null).item(null), matchesList().item("run.bat").item("run.bat"))
         );
 
         result = runEsql("""
             FROM test* | where process.parent.command_line == "run.bat"
             """);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text"))
-            ).entry("values", List.of(matchesList().item("run.bat").item("run.bat")))
+            List.of(columnInfo("process.parent.command_line", "keyword"), columnInfo("process.parent.command_line.text", "text")),
+            List.of(matchesList().item("run.bat").item("run.bat"))
         );
 
         ResponseException e = expectThrows(ResponseException.class, () -> runEsql("FROM test* | SORT Responses.process.pid"));
@@ -1339,61 +1264,47 @@ public void testOneNestedField_AndSameNameSupportedField_TwoIndices() throws IOE
             {"Responses.process": 222,"process.parent.command_line":"run2.bat"}""");
 
         Map result = runEsql("FROM test* | SORT process.parent.command_line");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(
-                    columnInfo("Responses.process", "integer"),
-                    columnInfo("Responses.process.pid", "long"),
-                    columnInfo("process.parent.command_line", "keyword"),
-                    columnInfo("process.parent.command_line.text", "text")
-                )
+            List.of(
+                columnInfo("Responses.process", "integer"),
+                columnInfo("Responses.process.pid", "long"),
+                columnInfo("process.parent.command_line", "keyword"),
+                columnInfo("process.parent.command_line.text", "text")
+            ),
+            List.of(
+                matchesList().item(null).item(null).item("run1.bat").item("run1.bat"),
+                matchesList().item(222).item(222).item("run2.bat").item("run2.bat")
             )
-                .entry(
-                    "values",
-                    List.of(
-                        matchesList().item(null).item(null).item("run1.bat").item("run1.bat"),
-                        matchesList().item(222).item(222).item("run2.bat").item("run2.bat")
-                    )
-                )
         );
 
         result = runEsql("""
             FROM test* | where Responses.process.pid == 111
             """);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(
-                    columnInfo("Responses.process", "integer"),
-                    columnInfo("Responses.process.pid", "long"),
-                    columnInfo("process.parent.command_line", "keyword"),
-                    columnInfo("process.parent.command_line.text", "text")
-                )
-            ).entry("values", List.of())
+            List.of(
+                columnInfo("Responses.process", "integer"),
+                columnInfo("Responses.process.pid", "long"),
+                columnInfo("process.parent.command_line", "keyword"),
+                columnInfo("process.parent.command_line.text", "text")
+            ),
+            List.of()
         );
 
         result = runEsql("FROM test* | SORT process.parent.command_line");
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(
-                    columnInfo("Responses.process", "integer"),
-                    columnInfo("Responses.process.pid", "long"),
-                    columnInfo("process.parent.command_line", "keyword"),
-                    columnInfo("process.parent.command_line.text", "text")
-                )
+            List.of(
+                columnInfo("Responses.process", "integer"),
+                columnInfo("Responses.process.pid", "long"),
+                columnInfo("process.parent.command_line", "keyword"),
+                columnInfo("process.parent.command_line.text", "text")
+            ),
+            List.of(
+                matchesList().item(null).item(null).item("run1.bat").item("run1.bat"),
+                matchesList().item(222).item(222).item("run2.bat").item("run2.bat")
             )
-                .entry(
-                    "values",
-                    List.of(
-                        matchesList().item(null).item(null).item("run1.bat").item("run1.bat"),
-                        matchesList().item(222).item(222).item("run2.bat").item("run2.bat")
-                    )
-                )
         );
 
         result = runEsql("""
@@ -1401,17 +1312,15 @@ public void testOneNestedField_AndSameNameSupportedField_TwoIndices() throws IOE
             | SORT process.parent.command_line
             | WHERE Responses.process IS NULL
             """);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMapWithOptionalTook(result.get("took")).entry(
-                "columns",
-                List.of(
-                    columnInfo("Responses.process", "integer"),
-                    columnInfo("Responses.process.pid", "long"),
-                    columnInfo("process.parent.command_line", "keyword"),
-                    columnInfo("process.parent.command_line.text", "text")
-                )
-            ).entry("values", List.of(matchesList().item(null).item(null).item("run1.bat").item("run1.bat")))
+            List.of(
+                columnInfo("Responses.process", "integer"),
+                columnInfo("Responses.process.pid", "long"),
+                columnInfo("process.parent.command_line", "keyword"),
+                columnInfo("process.parent.command_line.text", "text")
+            ),
+            List.of(matchesList().item(null).item(null).item("run1.bat").item("run1.bat"))
         );
     }
 
@@ -1671,7 +1580,7 @@ void test(Object value, Object expectedValue) throws IOException {
                 values = values.item(expectedValue);
             }
 
-            assertMap(result, matchesMapWithOptionalTook(result.get("took")).entry("columns", columns).entry("values", List.of(values)));
+            assertResultMap(result, columns, List.of(values));
         }
 
         void createIndex(String name, String fieldName) throws IOException {
diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java
index 5e0aeb5b3535d..ba057cbe276ba 100644
--- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java
+++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java
@@ -26,14 +26,12 @@
 import java.util.Map;
 
 import static org.elasticsearch.test.ListMatcher.matchesList;
-import static org.elasticsearch.test.MapMatcher.assertMap;
 import static org.elasticsearch.test.MapMatcher.matchesMap;
 import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.entityToMap;
 import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.requestObjectBuilder;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.nullValue;
@@ -63,42 +61,35 @@ public void testTimestampFilterFromQuery() throws IOException {
 
         // filter includes both indices in the result (all columns, all rows)
         RestEsqlTestCase.RequestObjectBuilder builder = timestampFilter("gte", "2023-01-01").query(from("test*"));
-        Map result = runEsql(builder);
-        assertMap(
-            result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "id2").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "value").entry("type", "long"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(docsTest1 + docsTest2))).entry("took", greaterThanOrEqualTo(0))
+        assertResultMap(
+            runEsql(builder),
+            matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer"))
+                .item(matchesMap().entry("name", "id2").entry("type", "integer"))
+                .item(matchesMap().entry("name", "value").entry("type", "long")),
+            allOf(instanceOf(List.class), hasSize(docsTest1 + docsTest2))
         );
 
         // filter includes only test1. Columns from test2 are filtered out, as well (not only rows)!
         builder = timestampFilter("gte", "2024-01-01").query(from("test*"));
-        assertMap(
+        assertResultMap(
             runEsql(builder),
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "value").entry("type", "long"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(docsTest1))).entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer"))
+                .item(matchesMap().entry("name", "value").entry("type", "long")),
+            allOf(instanceOf(List.class), hasSize(docsTest1))
         );
 
         // filter excludes both indices (no rows); the first analysis step fails because there are no columns, a second attempt succeeds
         // after eliminating the index filter. All columns are returned.
         builder = timestampFilter("gte", "2025-01-01").query(from("test*"));
-        assertMap(
+        assertResultMap(
             runEsql(builder),
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "id2").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "value").entry("type", "long"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(0))).entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer"))
+                .item(matchesMap().entry("name", "id2").entry("type", "integer"))
+                .item(matchesMap().entry("name", "value").entry("type", "long")),
+            allOf(instanceOf(List.class), hasSize(0))
         );
     }
 
@@ -110,27 +101,22 @@ public void testFieldExistsFilter_KeepWildcard() throws IOException {
 
         // filter includes only test1. Columns and rows of test2 are filtered out
         RestEsqlTestCase.RequestObjectBuilder builder = existsFilter("id1").query(from("test*"));
-        Map result = runEsql(builder);
-        assertMap(
-            result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "value").entry("type", "long"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(docsTest1))).entry("took", greaterThanOrEqualTo(0))
+        assertResultMap(
+            runEsql(builder),
+            matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer"))
+                .item(matchesMap().entry("name", "value").entry("type", "long")),
+            allOf(instanceOf(List.class), hasSize(docsTest1))
         );
 
         // filter includes only test1. Columns from test2 are filtered out, as well (not only rows)!
         builder = existsFilter("id1").query(from("test*") + " METADATA _index | KEEP _index, id*");
-        result = runEsql(builder);
-        assertMap(
+        Map result = runEsql(builder);
+        assertResultMap(
             result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "_index").entry("type", "keyword"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(docsTest1))).entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "_index").entry("type", "keyword"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer")),
+            allOf(instanceOf(List.class), hasSize(docsTest1))
         );
         @SuppressWarnings("unchecked")
         var values = (List>) result.get("values");
@@ -151,14 +137,12 @@ public void testFieldExistsFilter_With_ExplicitUseOfDiscardedIndexFields() throw
             from("test*") + " METADATA _index | SORT id2 | KEEP _index, id*"
         );
         Map result = runEsql(builder);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "_index").entry("type", "keyword"))
-                    .item(matchesMap().entry("name", "id1").entry("type", "integer"))
-                    .item(matchesMap().entry("name", "id2").entry("type", "integer"))
-            ).entry("values", allOf(instanceOf(List.class), hasSize(docsTest1))).entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "_index").entry("type", "keyword"))
+                .item(matchesMap().entry("name", "id1").entry("type", "integer"))
+                .item(matchesMap().entry("name", "id2").entry("type", "integer")),
+            allOf(instanceOf(List.class), hasSize(docsTest1))
         );
         @SuppressWarnings("unchecked")
         var values = (List>) result.get("values");
diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEnrichTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEnrichTestCase.java
index bf4a4400e13cf..69bbf7420c72e 100644
--- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEnrichTestCase.java
+++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEnrichTestCase.java
@@ -23,10 +23,7 @@
 import java.util.List;
 import java.util.Map;
 
-import static org.elasticsearch.test.MapMatcher.assertMap;
-import static org.elasticsearch.test.MapMatcher.matchesMap;
 import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 
 public abstract class RestEnrichTestCase extends ESRestTestCase {
 
@@ -194,14 +191,14 @@ public void testMatchField_ImplicitFieldsList() throws IOException {
         Map result = runEsql("from test1 | enrich countries | keep number | sort number");
         var columns = List.of(Map.of("name", "number", "type", "long"));
         var values = List.of(List.of(1000), List.of(1000), List.of(5000));
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
     }
 
     public void testMatchField_ImplicitFieldsList_WithStats() throws IOException {
         Map result = runEsql("from test1 | enrich countries | stats s = sum(number) by country_name");
         var columns = List.of(Map.of("name", "s", "type", "long"), Map.of("name", "country_name", "type", "keyword"));
         var values = List.of(List.of(2000, "United States of America"), List.of(5000, "China"));
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
     }
 
     public void testSimpleIndexFilteringWithEnrich() throws IOException {
@@ -226,7 +223,7 @@ public void testSimpleIndexFilteringWithEnrich() throws IOException {
             Arrays.asList(null, 1000, "US", "test1"),
             Arrays.asList(3, null, "US", "test2")
         );
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
 
         // filter something that won't affect the columns
         result = runEsql("""
@@ -235,7 +232,7 @@ public void testSimpleIndexFilteringWithEnrich() throws IOException {
                 | keep *number, geo.dest, _index
                 | sort geo.dest, _index
             """, b -> b.startObject("exists").field("field", "foobar").endObject());
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", List.of()).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, List.of());
     }
 
     public void testIndexFilteringWithEnrich_RemoveOneIndex() throws IOException {
@@ -259,7 +256,7 @@ public void testIndexFilteringWithEnrich_RemoveOneIndex() throws IOException {
             Arrays.asList(null, 1000, "US", "test1")
         );
 
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
 
         // filter out test2 and use a wildcarded field name in the "keep" command
         result = runEsql("""
@@ -275,7 +272,7 @@ public void testIndexFilteringWithEnrich_RemoveOneIndex() throws IOException {
             Map.of("name", "_index", "type", "keyword")
         );
         values = List.of(Arrays.asList(5000, "CN", "test1"), Arrays.asList(1000, "US", "test1"), Arrays.asList(1000, "US", "test1"));
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
     }
 
     public void testIndexFilteringWithEnrich_ExpectException() throws IOException {
@@ -315,7 +312,7 @@ public void testIndexFilteringWithEnrich_FilterUnusedIndexFields() throws IOExce
             Map.of("name", "_index", "type", "keyword")
         );
         var values = List.of(Arrays.asList(2, "IN", "test2"), Arrays.asList(2, "IN", "test2"), Arrays.asList(3, "US", "test2"));
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
     }
 
     private Map runEsql(String query) throws IOException {
diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java
index 77bfd8a98cd18..93a5e005f6a27 100644
--- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java
+++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java
@@ -59,7 +59,6 @@
 import static java.util.Map.entry;
 import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
 import static org.elasticsearch.test.ListMatcher.matchesList;
-import static org.elasticsearch.test.MapMatcher.assertMap;
 import static org.elasticsearch.test.MapMatcher.matchesMap;
 import static org.elasticsearch.xpack.esql.EsqlTestUtils.as;
 import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.Mode.ASYNC;
@@ -257,7 +256,7 @@ public static RequestObjectBuilder jsonBuilder() throws IOException {
 
     public void testGetAnswer() throws IOException {
         Map answer = runEsql(requestObjectBuilder().query("row a = 1, b = 2"));
-        assertEquals(3, answer.size());
+        assertEquals(4, answer.size());
         assertThat(((Integer) answer.get("took")).intValue(), greaterThanOrEqualTo(0));
         Map colA = Map.of("name", "a", "type", "integer");
         Map colB = Map.of("name", "b", "type", "integer");
@@ -296,21 +295,13 @@ public void testNullInAggs() throws IOException {
         assertThat(EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8), equalTo("{\"errors\":false}"));
 
         RequestObjectBuilder builder = requestObjectBuilder().query(fromIndex() + " | stats min(value)");
-        Map result = runEsql(builder);
-        assertMap(
-            result,
-            matchesMap().entry("values", List.of(List.of(1)))
-                .entry("columns", List.of(Map.of("name", "min(value)", "type", "long")))
-                .entry("took", greaterThanOrEqualTo(0))
-        );
+        assertResultMap(runEsql(builder), List.of(Map.of("name", "min(value)", "type", "long")), List.of(List.of(1)));
 
         builder = requestObjectBuilder().query(fromIndex() + " | stats min(value) by group | sort group, `min(value)`");
-        result = runEsql(builder);
-        assertMap(
-            result,
-            matchesMap().entry("values", List.of(List.of(2, 0), List.of(1, 1)))
-                .entry("columns", List.of(Map.of("name", "min(value)", "type", "long"), Map.of("name", "group", "type", "long")))
-                .entry("took", greaterThanOrEqualTo(0))
+        assertResultMap(
+            runEsql(builder),
+            List.of(Map.of("name", "min(value)", "type", "long"), Map.of("name", "group", "type", "long")),
+            List.of(List.of(2, 0), List.of(1, 1))
         );
     }
 
@@ -570,7 +561,7 @@ public void testMetadataFieldsOnMultipleIndices() throws IOException {
         );
         var values = List.of(List.of(3, testIndexName() + "-2", 1, "id-2"), List.of(2, testIndexName() + "-1", 2, "id-1"));
 
-        assertMap(result, matchesMap().entry("columns", columns).entry("values", values).entry("took", greaterThanOrEqualTo(0)));
+        assertResultMap(result, columns, values);
 
         assertThat(deleteIndex(testIndexName() + "-1").isAcknowledged(), is(true)); // clean up
         assertThat(deleteIndex(testIndexName() + "-2").isAcknowledged(), is(true)); // clean up
@@ -868,17 +859,15 @@ public void testInlineStatsNow() throws IOException {
                     .item(499.5)
             );
         }
-        assertMap(
+        assertResultMap(
             result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
-                    .item(matchesMap().entry("name", "test").entry("type", "text"))
-                    .item(matchesMap().entry("name", "test.keyword").entry("type", "keyword"))
-                    .item(matchesMap().entry("name", "value").entry("type", "long"))
-                    .item(matchesMap().entry("name", "now").entry("type", "date"))
-                    .item(matchesMap().entry("name", "AVG(value)").entry("type", "double"))
-            ).entry("values", values).entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "@timestamp").entry("type", "date"))
+                .item(matchesMap().entry("name", "test").entry("type", "text"))
+                .item(matchesMap().entry("name", "test.keyword").entry("type", "keyword"))
+                .item(matchesMap().entry("name", "value").entry("type", "long"))
+                .item(matchesMap().entry("name", "now").entry("type", "date"))
+                .item(matchesMap().entry("name", "AVG(value)").entry("type", "double")),
+            values
         );
     }
 
@@ -894,11 +883,10 @@ public void testTopLevelFilter() throws IOException {
         }).query(fromIndex() + " | STATS SUM(value)");
 
         Map result = runEsql(builder);
-        assertMap(
+        assertResultMap(
             result,
-            matchesMap().entry("columns", matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")))
-                .entry("values", List.of(List.of(499500)))
-                .entry("took", greaterThanOrEqualTo(0))
+            matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")),
+            List.of(List.of(499500))
         );
     }
 
@@ -913,12 +901,7 @@ public void testTopLevelFilterMerged() throws IOException {
             b.endObject();
         }).query(fromIndex() + " | WHERE value == 12 | STATS SUM(value)");
         Map result = runEsql(builder);
-        assertMap(
-            result,
-            matchesMap().entry("columns", matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")))
-                .entry("values", List.of(List.of(12)))
-                .entry("took", greaterThanOrEqualTo(0))
-        );
+        assertResultMap(result, matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")), List.of(List.of(12)));
     }
 
     public void testTopLevelFilterBoolMerged() throws IOException {
@@ -947,11 +930,10 @@ public void testTopLevelFilterBoolMerged() throws IOException {
                 b.endObject();
             }).query(fromIndex() + " | WHERE @timestamp > \"2010-01-01\" | STATS SUM(value)");
             Map result = runEsql(builder);
-            assertMap(
+            assertResultMap(
                 result,
-                matchesMap().entry("columns", matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")))
-                    .entry("values", List.of(List.of(12)))
-                    .entry("took", greaterThanOrEqualTo(0))
+                matchesList().item(matchesMap().entry("name", "SUM(value)").entry("type", "long")),
+                List.of(List.of(12))
             );
         }
     }
@@ -1133,13 +1115,12 @@ public void testAsyncGetWithoutContentType() throws IOException {
         for (int i = 0; i < count; i++) {
             values = values.item(matchesList().item("keyword" + i).item(i));
         }
-        assertMap(
+        assertResultMap(
             result,
-            matchesMap().entry(
-                "columns",
-                matchesList().item(matchesMap().entry("name", "keyword").entry("type", "keyword"))
-                    .item(matchesMap().entry("name", "integer").entry("type", "integer"))
-            ).entry("values", values).entry("took", greaterThanOrEqualTo(0)).entry("id", id).entry("is_running", false)
+            getResultMatcher(result).entry("id", id).entry("is_running", false),
+            matchesList().item(matchesMap().entry("name", "keyword").entry("type", "keyword"))
+                .item(matchesMap().entry("name", "integer").entry("type", "integer")),
+            values
         );
 
     }
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractCrossClustersUsageTelemetryIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractCrossClustersUsageTelemetryIT.java
index ffbddd52b2551..7df40da0344a9 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractCrossClustersUsageTelemetryIT.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractCrossClustersUsageTelemetryIT.java
@@ -127,7 +127,7 @@ protected CCSTelemetrySnapshot getTelemetryFromFailedQuery(String query) throws
         return getTelemetrySnapshot(queryNode);
     }
 
-    private CCSTelemetrySnapshot getTelemetrySnapshot(String nodeName) {
+    protected CCSTelemetrySnapshot getTelemetrySnapshot(String nodeName) {
         var usage = cluster(LOCAL_CLUSTER).getInstance(UsageService.class, nodeName);
         return usage.getEsqlUsageHolder().getCCSTelemetrySnapshot();
     }
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPauseFieldPlugin.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPauseFieldPlugin.java
index 5554f7e571dfb..492947304d898 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPauseFieldPlugin.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPauseFieldPlugin.java
@@ -34,12 +34,16 @@ protected void onStartExecute() {}
     // Called when the engine needs to wait for further execution to be allowed.
     protected abstract boolean onWait() throws InterruptedException;
 
+    protected String scriptTypeName() {
+        return "pause";
+    }
+
     @Override
     public ScriptEngine getScriptEngine(Settings settings, Collection> contexts) {
         return new ScriptEngine() {
             @Override
             public String getType() {
-                return "pause";
+                return scriptTypeName();
             }
 
             @Override
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncEnrichStopIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncEnrichStopIT.java
new file mode 100644
index 0000000000000..99a81c60a9ad2
--- /dev/null
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncEnrichStopIT.java
@@ -0,0 +1,156 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.action;
+
+import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.xcontent.json.JsonXContent;
+import org.elasticsearch.xpack.core.async.AsyncStopRequest;
+import org.elasticsearch.xpack.esql.plan.logical.Enrich;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.xpack.esql.EsqlTestUtils.getValuesList;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.deleteAsyncId;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.startAsyncQuery;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.waitForCluster;
+import static org.hamcrest.Matchers.equalTo;
+
+// This tests if enrich after stop works correctly
+public class CrossClusterAsyncEnrichStopIT extends AbstractEnrichBasedCrossClusterTestCase {
+
+    @Override
+    protected boolean reuseClusters() {
+        return false;
+    }
+
+    @Override
+    protected boolean tolerateErrorsWhenWipingEnrichPolicies() {
+        // attempt to wipe will fail since some clusters are already closed
+        return true;
+    }
+
+    @Override
+    protected Collection> nodePlugins(String clusterAlias) {
+        List> plugins = new ArrayList<>(super.nodePlugins(clusterAlias));
+        plugins.add(EsqlPluginWithEnterpriseOrTrialLicense.class);
+        plugins.add(SimplePauseFieldPlugin.class);
+        return plugins;
+    }
+
+    @Before
+    public void resetPlugin() {
+        SimplePauseFieldPlugin.resetPlugin();
+    }
+
+    /**
+     * This tests that enrich and aggs work after stop. It works like this:
+     * 1. We launch the async request
+     * 2. c2 index has the pause field which will pause the query until we allow it to proceed
+     * 3. We wait until c1 is done and then stop the async request
+     * 4. We allow the query to proceed
+     * 5. The result should contain the data from local and c1 and coordinator-side enrichments should happen
+     */
+    public void testEnrichAfterStop() throws Exception {
+        setupEventsIndexWithPause("c2");
+        String query = String.format(Locale.ROOT, """
+            FROM *:events,events
+            | eval ip= TO_STR(host)
+            | %s
+            | %s
+            | eval const = coalesce(const, 1)
+            | stats c = sum(const) by vendor
+            | sort vendor
+            """, enrichHosts(Enrich.Mode.COORDINATOR), enrichVendors(Enrich.Mode.COORDINATOR));
+
+        // Start the async query
+        final String asyncExecutionId = startAsyncQuery(client(), query, randomBoolean());
+        SimplePauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+
+        // wait until c1 is done
+        waitForCluster(client(), "c1", asyncExecutionId);
+        waitForCluster(client(), LOCAL_CLUSTER, asyncExecutionId);
+
+        // Run the stop request
+        var stopRequest = new AsyncStopRequest(asyncExecutionId);
+        var stopAction = client().execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        // Allow the processing to proceed
+        SimplePauseFieldPlugin.allowEmitting.countDown();
+
+        try (EsqlQueryResponse resp = stopAction.actionGet(30, TimeUnit.SECONDS)) {
+            // Compare this to CrossClustersEnrichIT.testEnrichTwiceThenAggs - the results from c2 will be absent
+            // because we stopped it before processing the data
+            assertThat(
+                getValuesList(resp),
+                equalTo(
+                    List.of(
+                        List.of(5L, "Apple"),
+                        List.of(6L, "Microsoft"),
+                        List.of(3L, "Redhat"),
+                        List.of(2L, "Samsung"),
+                        Arrays.asList(2L, (String) null)
+                    )
+                )
+            );
+            EsqlExecutionInfo executionInfo = resp.getExecutionInfo();
+            assertThat(executionInfo.clusterAliases(), equalTo(Set.of("", "c1", "c2")));
+        } finally {
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
+        }
+    }
+
+    private void setupEventsIndexWithPause(String clusterAlias) throws IOException {
+        record Event(long timestamp, String user, String host) {}
+        List events = List.of(
+            new Event(1, "park", "192.168.1.25"),
+            new Event(2, "akio", "192.168.1.5"),
+            new Event(3, "park", "192.168.1.2"),
+            new Event(4, "kevin", "192.168.1.3")
+        );
+        // Regular mapping
+        var stdMapping = PutMappingRequest.simpleMapping("timestamp", "type=long", "user", "type=keyword", "host", "type=ip");
+        Map mappingMap = XContentHelper.convertToMap(BytesReference.bytes(stdMapping), false, stdMapping.contentType())
+            .v2();
+        // Pause field mapping
+        var mapping = JsonXContent.contentBuilder().startObject();
+        mapping.startObject("runtime");
+        {
+            mapping.startObject("const");
+            {
+                mapping.field("type", "long");
+                mapping.startObject("script").field("source", "").field("lang", "pause").endObject();
+            }
+            mapping.endObject();
+        }
+        mapping.endObject();
+        mapping.endObject();
+        Map mappingMap2 = XContentHelper.convertToMap(BytesReference.bytes(mapping), false, mapping.contentType()).v2();
+        // Merge the two mappings
+        mappingMap.putAll(mappingMap2);
+
+        var client = client(clusterAlias);
+        assertAcked(client.admin().indices().prepareDelete("events"));
+        assertAcked(client.admin().indices().prepareCreate("events").setMapping(mappingMap));
+        for (var e : events) {
+            client.prepareIndex("events").setSource("timestamp", e.timestamp, "user", e.user, "host", e.host, "const", "1").get();
+        }
+        client.admin().indices().prepareRefresh("events").get();
+    }
+}
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncQueryIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncQueryIT.java
index f79ff8ba61cb3..f6282300c4e3f 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncQueryIT.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterAsyncQueryIT.java
@@ -7,43 +7,53 @@
 
 package org.elasticsearch.xpack.esql.action;
 
-import org.elasticsearch.ElasticsearchTimeoutException;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.ResourceNotFoundException;
+import org.elasticsearch.action.ActionFuture;
 import org.elasticsearch.action.bulk.BulkRequestBuilder;
 import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.action.support.WriteRequest;
-import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.compute.operator.DriverTaskRunner;
 import org.elasticsearch.compute.operator.exchange.ExchangeService;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
-import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.tasks.TaskId;
+import org.elasticsearch.tasks.TaskInfo;
 import org.elasticsearch.test.AbstractMultiClustersTestCase;
 import org.elasticsearch.test.XContentTestUtils;
 import org.elasticsearch.transport.RemoteClusterAware;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xcontent.json.JsonXContent;
-import org.elasticsearch.xpack.core.async.DeleteAsyncResultRequest;
-import org.elasticsearch.xpack.core.async.GetAsyncResultRequest;
-import org.elasticsearch.xpack.core.async.TransportDeleteAsyncResultAction;
+import org.elasticsearch.xpack.core.async.AsyncExecutionId;
+import org.elasticsearch.xpack.core.async.AsyncStopRequest;
 import org.junit.Before;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-import static org.elasticsearch.core.TimeValue.timeValueMillis;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase.randomIncludeCCSMetadata;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.deleteAsyncId;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.getAsyncResponse;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.runAsyncQuery;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.startAsyncQuery;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.startAsyncQueryWithPragmas;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.waitForCluster;
+import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.is;
@@ -57,6 +67,7 @@ public class CrossClusterAsyncQueryIT extends AbstractMultiClustersTestCase {
     private static String LOCAL_INDEX = "logs-1";
     private static String REMOTE_INDEX = "logs-2";
     private static final String INDEX_WITH_RUNTIME_MAPPING = "blocking";
+    private static final String INDEX_WITH_FAIL_MAPPING = "failing";
 
     @Override
     protected Collection remoteClusterAlias() {
@@ -65,7 +76,7 @@ protected Collection remoteClusterAlias() {
 
     @Override
     protected Map skipUnavailableForRemoteClusters() {
-        return Map.of(REMOTE_CLUSTER_1, randomBoolean(), REMOTE_CLUSTER_2, randomBoolean());
+        return Map.of(REMOTE_CLUSTER_1, false, REMOTE_CLUSTER_2, randomBoolean());
     }
 
     @Override
@@ -75,6 +86,8 @@ protected Collection> nodePlugins(String clusterAlias) {
         plugins.add(EsqlAsyncActionIT.LocalStateEsqlAsync.class); // allows the async_search DELETE action
         plugins.add(InternalExchangePlugin.class);
         plugins.add(SimplePauseFieldPlugin.class);
+        plugins.add(FailingPauseFieldPlugin.class);
+        plugins.add(CountingPauseFieldPlugin.class);
         return plugins;
     }
 
@@ -94,6 +107,8 @@ public List> getSettings() {
     @Before
     public void resetPlugin() {
         SimplePauseFieldPlugin.resetPlugin();
+        FailingPauseFieldPlugin.resetPlugin();
+        CountingPauseFieldPlugin.resetPlugin();
     }
 
     /**
@@ -103,42 +118,28 @@ public void testSuccessfulPathways() throws Exception {
         Map testClusterInfo = setupClusters(3);
         int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
         int remote1NumShards = (Integer) testClusterInfo.get("remote1.num_shards");
-        int remote2NumShards = (Integer) testClusterInfo.get("remote2.blocking_index.num_shards");
+        populateRuntimeIndex(REMOTE_CLUSTER_2, "pause", INDEX_WITH_RUNTIME_MAPPING);
 
         Tuple includeCCSMetadata = randomIncludeCCSMetadata();
-        Boolean requestIncludeMeta = includeCCSMetadata.v1();
         boolean responseExpectMeta = includeCCSMetadata.v2();
 
-        AtomicReference asyncExecutionId = new AtomicReference<>();
-
-        String q = "FROM logs-*,cluster-a:logs-*,remote-b:blocking | STATS total=sum(const) | LIMIT 10";
-        try (EsqlQueryResponse resp = runAsyncQuery(q, requestIncludeMeta, null, TimeValue.timeValueMillis(100))) {
-            assertTrue(resp.isRunning());
-            assertNotNull("async execution id is null", resp.asyncExecutionId());
-            asyncExecutionId.set(resp.asyncExecutionId().get());
-            // executionInfo may or may not be set on the initial response when there is a relatively low wait_for_completion_timeout
-            // so we do not check for it here
-        }
-
+        final String asyncExecutionId = startAsyncQuery(
+            client(),
+            "FROM logs-*,cluster-a:logs-*,remote-b:blocking | STATS total=sum(const) | LIMIT 10",
+            includeCCSMetadata.v1()
+        );
         // wait until we know that the query against 'remote-b:blocking' has started
         SimplePauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
 
         // wait until the query of 'cluster-a:logs-*' has finished (it is not blocked since we are not searching the 'blocking' index on it)
-        assertBusy(() -> {
-            try (EsqlQueryResponse asyncResponse = getAsyncResponse(asyncExecutionId.get())) {
-                EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
-                assertNotNull(executionInfo);
-                EsqlExecutionInfo.Cluster clusterA = executionInfo.getCluster("cluster-a");
-                assertThat(clusterA.getStatus(), not(equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING)));
-            }
-        });
+        waitForCluster(client(), "cluster-a", asyncExecutionId);
 
         /* at this point:
          *  the query against cluster-a should be finished
          *  the query against remote-b should be running (blocked on the PauseFieldPlugin.allowEmitting CountDown)
          *  the query against the local cluster should be running because it has a STATS clause that needs to wait on remote-b
          */
-        try (EsqlQueryResponse asyncResponse = getAsyncResponse(asyncExecutionId.get())) {
+        try (EsqlQueryResponse asyncResponse = getAsyncResponse(client(), asyncExecutionId)) {
             EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
             assertThat(asyncResponse.isRunning(), is(true));
             assertThat(
@@ -149,13 +150,8 @@ public void testSuccessfulPathways() throws Exception {
             assertThat(executionInfo.getClusterStateCount(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL), equalTo(1));
 
             EsqlExecutionInfo.Cluster clusterA = executionInfo.getCluster(REMOTE_CLUSTER_1);
-            assertThat(clusterA.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-            assertThat(clusterA.getTotalShards(), greaterThanOrEqualTo(1));
-            assertThat(clusterA.getSuccessfulShards(), equalTo(clusterA.getTotalShards()));
-            assertThat(clusterA.getSkippedShards(), equalTo(0));
-            assertThat(clusterA.getFailedShards(), equalTo(0));
-            assertThat(clusterA.getFailures().size(), equalTo(0));
-            assertThat(clusterA.getTook().millis(), greaterThanOrEqualTo(0L));
+            // Should be done and successful
+            assertClusterInfoSuccess(clusterA, clusterA.getTotalShards());
 
             EsqlExecutionInfo.Cluster local = executionInfo.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
             // should still be RUNNING since the local cluster has to do a STATS on the coordinator, waiting on remoteB
@@ -175,7 +171,7 @@ public void testSuccessfulPathways() throws Exception {
 
         // wait until both remoteB and local queries have finished
         assertBusy(() -> {
-            try (EsqlQueryResponse asyncResponse = getAsyncResponse(asyncExecutionId.get())) {
+            try (EsqlQueryResponse asyncResponse = getAsyncResponse(client(), asyncExecutionId)) {
                 EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
                 assertNotNull(executionInfo);
                 EsqlExecutionInfo.Cluster remoteB = executionInfo.getCluster(REMOTE_CLUSTER_2);
@@ -186,40 +182,30 @@ public void testSuccessfulPathways() throws Exception {
             }
         });
 
-        try (EsqlQueryResponse asyncResponse = getAsyncResponse(asyncExecutionId.get())) {
+        try (EsqlQueryResponse asyncResponse = getAsyncResponse(client(), asyncExecutionId)) {
             EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
             assertNotNull(executionInfo);
             assertThat(executionInfo.overallTook().millis(), greaterThanOrEqualTo(1L));
+            assertThat(executionInfo.isPartial(), equalTo(false));
 
             EsqlExecutionInfo.Cluster clusterA = executionInfo.getCluster(REMOTE_CLUSTER_1);
-            assertThat(clusterA.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-            assertThat(clusterA.getTook().millis(), greaterThanOrEqualTo(0L));
-            assertThat(clusterA.getTotalShards(), equalTo(remote1NumShards));
-            assertThat(clusterA.getSuccessfulShards(), equalTo(remote1NumShards));
-            assertThat(clusterA.getSkippedShards(), equalTo(0));
-            assertThat(clusterA.getFailedShards(), equalTo(0));
-            assertThat(clusterA.getFailures().size(), equalTo(0));
+            assertClusterInfoSuccess(clusterA, remote1NumShards);
 
             EsqlExecutionInfo.Cluster remoteB = executionInfo.getCluster(REMOTE_CLUSTER_2);
-            assertThat(remoteB.getTook().millis(), greaterThanOrEqualTo(0L));
-            assertThat(remoteB.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-            assertThat(remoteB.getTotalShards(), equalTo(remote2NumShards));
-            assertThat(remoteB.getSuccessfulShards(), equalTo(remote2NumShards));
-            assertThat(remoteB.getSkippedShards(), equalTo(0));
-            assertThat(remoteB.getFailedShards(), equalTo(0));
-            assertThat(remoteB.getFailures().size(), equalTo(0));
+            assertClusterInfoSuccess(remoteB, 1);
 
             EsqlExecutionInfo.Cluster local = executionInfo.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
-            assertThat(local.getTook().millis(), greaterThanOrEqualTo(0L));
-            assertThat(local.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-            assertThat(local.getTotalShards(), equalTo(localNumShards));
-            assertThat(local.getSuccessfulShards(), equalTo(localNumShards));
-            assertThat(local.getSkippedShards(), equalTo(0));
-            assertThat(local.getFailedShards(), equalTo(0));
-            assertThat(local.getFailures().size(), equalTo(0));
+            assertClusterInfoSuccess(local, localNumShards);
+
+            // Check that stop produces the same result
+            try (
+                EsqlQueryResponse stopResponse = client().execute(EsqlAsyncStopAction.INSTANCE, new AsyncStopRequest(asyncExecutionId))
+                    .get()
+            ) {
+                assertThat(stopResponse, equalTo(asyncResponse));
+            }
         } finally {
-            AcknowledgedResponse acknowledgedResponse = deleteAsyncId(asyncExecutionId.get());
-            assertThat(acknowledgedResponse.isAcknowledged(), is(true));
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
         }
     }
 
@@ -231,7 +217,7 @@ public void testAsyncQueriesWithLimit0() throws IOException {
 
         final TimeValue waitForCompletion = TimeValue.timeValueNanos(randomFrom(1L, Long.MAX_VALUE));
         String asyncExecutionId = null;
-        try (EsqlQueryResponse resp = runAsyncQuery("FROM logs*,*:logs* | LIMIT 0", requestIncludeMeta, null, waitForCompletion)) {
+        try (EsqlQueryResponse resp = runAsyncQuery(client(), "FROM logs*,*:logs* | LIMIT 0", requestIncludeMeta, waitForCompletion)) {
             EsqlExecutionInfo executionInfo = resp.getExecutionInfo();
             if (resp.isRunning()) {
                 asyncExecutionId = resp.asyncExecutionId().get();
@@ -252,88 +238,294 @@ public void testAsyncQueriesWithLimit0() throws IOException {
                 assertThat(overallTookMillis, greaterThanOrEqualTo(0L));
                 assertThat(executionInfo.includeCCSMetadata(), equalTo(responseExpectMeta));
                 assertThat(executionInfo.clusterAliases(), equalTo(Set.of(LOCAL_CLUSTER, REMOTE_CLUSTER_1, REMOTE_CLUSTER_2)));
+                assertThat(executionInfo.isPartial(), equalTo(false));
 
                 EsqlExecutionInfo.Cluster remoteCluster = executionInfo.getCluster(REMOTE_CLUSTER_1);
-                assertThat(remoteCluster.getIndexExpression(), equalTo("logs*"));
-                assertThat(remoteCluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-                assertThat(remoteCluster.getTook().millis(), greaterThanOrEqualTo(0L));
                 assertThat(remoteCluster.getTook().millis(), lessThanOrEqualTo(overallTookMillis));
-                assertThat(remoteCluster.getTotalShards(), equalTo(0));
-                assertThat(remoteCluster.getSuccessfulShards(), equalTo(0));
-                assertThat(remoteCluster.getSkippedShards(), equalTo(0));
-                assertThat(remoteCluster.getFailedShards(), equalTo(0));
+                assertThat(remoteCluster.getIndexExpression(), equalTo("logs*"));
+                assertClusterInfoSuccess(remoteCluster, 0);
 
-                EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE_CLUSTER_1);
+                EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE_CLUSTER_2);
+                assertClusterInfoSuccess(remote2Cluster, 0);
                 assertThat(remote2Cluster.getIndexExpression(), equalTo("logs*"));
-                assertThat(remote2Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-                assertThat(remote2Cluster.getTook().millis(), greaterThanOrEqualTo(0L));
                 assertThat(remote2Cluster.getTook().millis(), lessThanOrEqualTo(overallTookMillis));
-                assertThat(remote2Cluster.getTotalShards(), equalTo(0));
-                assertThat(remote2Cluster.getSuccessfulShards(), equalTo(0));
-                assertThat(remote2Cluster.getSkippedShards(), equalTo(0));
-                assertThat(remote2Cluster.getFailedShards(), equalTo(0));
 
                 EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER);
+                assertClusterInfoSuccess(localCluster, 0);
                 assertThat(localCluster.getIndexExpression(), equalTo("logs*"));
-                assertThat(localCluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
-                assertThat(localCluster.getTook().millis(), greaterThanOrEqualTo(0L));
                 assertThat(localCluster.getTook().millis(), lessThanOrEqualTo(overallTookMillis));
-                assertThat(remote2Cluster.getTotalShards(), equalTo(0));
-                assertThat(remote2Cluster.getSuccessfulShards(), equalTo(0));
-                assertThat(remote2Cluster.getSkippedShards(), equalTo(0));
-                assertThat(remote2Cluster.getFailedShards(), equalTo(0));
 
                 assertClusterMetadataInResponse(resp, responseExpectMeta, 3);
             }
         } finally {
             if (asyncExecutionId != null) {
-                AcknowledgedResponse acknowledgedResponse = deleteAsyncId(asyncExecutionId);
-                assertThat(acknowledgedResponse.isAcknowledged(), is(true));
+                assertAcked(deleteAsyncId(client(), asyncExecutionId));
             }
         }
     }
 
-    protected EsqlQueryResponse runAsyncQuery(String query, Boolean ccsMetadata, QueryBuilder filter, TimeValue waitCompletionTime) {
-        EsqlQueryRequest request = EsqlQueryRequest.asyncEsqlQueryRequest();
-        request.query(query);
-        request.pragmas(AbstractEsqlIntegTestCase.randomPragmas());
-        request.profile(randomInt(5) == 2);
-        request.columnar(randomBoolean());
-        if (ccsMetadata != null) {
-            request.includeCCSMetadata(ccsMetadata);
+    public void testStopQuery() throws Exception {
+        Map testClusterInfo = setupClusters(3);
+        int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
+        int remote1NumShards = (Integer) testClusterInfo.get("remote1.num_shards");
+        // Create large index so we could be sure we're stopping before the end
+        populateRuntimeIndex(REMOTE_CLUSTER_2, "pause_count", INDEX_WITH_RUNTIME_MAPPING);
+
+        Tuple includeCCSMetadata = randomIncludeCCSMetadata();
+        boolean responseExpectMeta = includeCCSMetadata.v2();
+
+        final String asyncExecutionId = startAsyncQueryWithPragmas(
+            client(),
+            "FROM logs-*,cluster-a:logs-*,remote-b:blocking | STATS total=sum(coalesce(const,v)) | LIMIT 1",
+            includeCCSMetadata.v1(),
+            Map.of("page_size", 1, "data_partitioning", "shard", "task_concurrency", 1)
+        );
+
+        // wait until we know that the query against 'remote-b:blocking' has started
+        CountingPauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+
+        // wait until the query of 'cluster-a:logs-*' has finished (it is not blocked since we are not searching the 'blocking' index on it)
+        waitForCluster(client(), REMOTE_CLUSTER_1, asyncExecutionId);
+        waitForCluster(client(), LOCAL_CLUSTER, asyncExecutionId);
+
+        /* at this point:
+         *  the query against cluster-a should be finished
+         *  the query against remote-b should be running (blocked on the PauseFieldPlugin.allowEmitting CountDown)
+         *  the query against the local cluster should be running because it has a STATS clause that needs to wait on remote-b
+         */
+
+        // run the stop query
+        AsyncStopRequest stopRequest = new AsyncStopRequest(asyncExecutionId);
+        ActionFuture stopAction = client().execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        assertBusy(() -> {
+            List tasks = getDriverTasks(client(REMOTE_CLUSTER_2));
+            List reduceTasks = tasks.stream().filter(t -> t.description().contains("_LuceneSourceOperator") == false).toList();
+            assertThat(reduceTasks, empty());
+        });
+        // allow remoteB query to proceed
+        CountingPauseFieldPlugin.allowEmitting.countDown();
+
+        // Since part of the query has not been stopped, we expect some result to emerge here
+        try (EsqlQueryResponse asyncResponse = stopAction.actionGet(30, TimeUnit.SECONDS)) {
+            // Check that we did not process all the fields on remote-b
+            // Should not be getting more than one page here, and we set page size to 1
+            assertThat(CountingPauseFieldPlugin.count.get(), lessThanOrEqualTo(1L));
+            assertThat(asyncResponse.isRunning(), is(false));
+            assertThat(asyncResponse.columns().size(), equalTo(1));
+            assertThat(asyncResponse.values().hasNext(), is(true));
+            Iterator row = asyncResponse.values().next();
+            // sum of 0-9 is 45, and sum of 0-9 squared is 285
+            assertThat(row.next(), equalTo(330L));
+
+            EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
+            assertNotNull(executionInfo);
+            assertThat(executionInfo.isCrossClusterSearch(), is(true));
+            long overallTookMillis = executionInfo.overallTook().millis();
+            assertThat(overallTookMillis, greaterThanOrEqualTo(0L));
+            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(LOCAL_CLUSTER, REMOTE_CLUSTER_1, REMOTE_CLUSTER_2)));
+            assertThat(executionInfo.isPartial(), equalTo(true));
+
+            EsqlExecutionInfo.Cluster remoteCluster = executionInfo.getCluster(REMOTE_CLUSTER_1);
+            assertThat(remoteCluster.getIndexExpression(), equalTo("logs-*"));
+            assertClusterInfoSuccess(remoteCluster, remote1NumShards);
+
+            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE_CLUSTER_2);
+            assertThat(remote2Cluster.getIndexExpression(), equalTo("blocking"));
+            assertThat(remote2Cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.PARTIAL));
+
+            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER);
+            assertThat(localCluster.getIndexExpression(), equalTo("logs-*"));
+            assertClusterInfoSuccess(localCluster, localNumShards);
+
+            assertClusterMetadataInResponse(asyncResponse, responseExpectMeta, 3);
+        } finally {
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
         }
-        request.waitForCompletionTimeout(waitCompletionTime);
-        request.keepOnCompletion(false);
-        if (filter != null) {
-            request.filter(filter);
+    }
+
+    public void testStopQueryLocal() throws Exception {
+        Map testClusterInfo = setupClusters(3);
+        int remote1NumShards = (Integer) testClusterInfo.get("remote1.num_shards");
+        int remote2NumShards = (Integer) testClusterInfo.get("remote2.num_shards");
+        populateRuntimeIndex(LOCAL_CLUSTER, "pause", INDEX_WITH_RUNTIME_MAPPING);
+
+        Tuple includeCCSMetadata = randomIncludeCCSMetadata();
+        boolean responseExpectMeta = includeCCSMetadata.v2();
+
+        final String asyncExecutionId = startAsyncQuery(
+            client(),
+            "FROM blocking,*:logs-* | STATS total=sum(coalesce(const,v)) | LIMIT 1",
+            includeCCSMetadata.v1()
+        );
+
+        // wait until we know that the query against 'remote-b:blocking' has started
+        SimplePauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+
+        // wait until the remotes are done
+        waitForCluster(client(), REMOTE_CLUSTER_1, asyncExecutionId);
+        waitForCluster(client(), REMOTE_CLUSTER_2, asyncExecutionId);
+
+        /* at this point:
+         *  the query against remotes should be finished
+         *  the query against the local cluster should be running because it's blocked
+         */
+
+        // run the stop query
+        AsyncStopRequest stopRequest = new AsyncStopRequest(asyncExecutionId);
+        ActionFuture stopAction = client().execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        // ensure stop operation is running
+        assertBusy(() -> {
+            try (EsqlQueryResponse asyncResponse = getAsyncResponse(client(), asyncExecutionId)) {
+                EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
+                assertNotNull(executionInfo);
+                assertThat(executionInfo.isPartial(), is(true));
+            }
+        });
+        // allow local query to proceed
+        SimplePauseFieldPlugin.allowEmitting.countDown();
+
+        // Since part of the query has not been stopped, we expect some result to emerge here
+        try (EsqlQueryResponse asyncResponse = stopAction.actionGet(30, TimeUnit.SECONDS)) {
+            assertThat(asyncResponse.isRunning(), is(false));
+            assertThat(asyncResponse.columns().size(), equalTo(1));
+            assertThat(asyncResponse.values().hasNext(), is(true));
+            Iterator row = asyncResponse.values().next();
+            // sum of 0-9 squared is 285, from two remotes it's 570
+            assertThat(row.next(), equalTo(570L));
+
+            EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
+            assertNotNull(executionInfo);
+            assertThat(executionInfo.isCrossClusterSearch(), is(true));
+            long overallTookMillis = executionInfo.overallTook().millis();
+            assertThat(overallTookMillis, greaterThanOrEqualTo(0L));
+            assertThat(executionInfo.clusterAliases(), equalTo(Set.of(LOCAL_CLUSTER, REMOTE_CLUSTER_1, REMOTE_CLUSTER_2)));
+            assertThat(executionInfo.isPartial(), equalTo(true));
+
+            EsqlExecutionInfo.Cluster remoteCluster = executionInfo.getCluster(REMOTE_CLUSTER_1);
+            assertThat(remoteCluster.getIndexExpression(), equalTo("logs-*"));
+            assertClusterInfoSuccess(remoteCluster, remote1NumShards);
+
+            EsqlExecutionInfo.Cluster remote2Cluster = executionInfo.getCluster(REMOTE_CLUSTER_2);
+            assertThat(remote2Cluster.getIndexExpression(), equalTo("logs-*"));
+            assertClusterInfoSuccess(remote2Cluster, remote2NumShards);
+
+            EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER);
+            assertThat(localCluster.getIndexExpression(), equalTo("blocking"));
+            assertThat(localCluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.PARTIAL));
+
+            assertClusterMetadataInResponse(asyncResponse, responseExpectMeta, 3);
+        } finally {
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
         }
-        return runAsyncQuery(request);
     }
 
-    protected EsqlQueryResponse runAsyncQuery(EsqlQueryRequest request) {
-        try {
-            return client(LOCAL_CLUSTER).execute(EsqlQueryAction.INSTANCE, request).actionGet(30, TimeUnit.SECONDS);
-        } catch (ElasticsearchTimeoutException e) {
-            throw new AssertionError("timeout waiting for query response", e);
+    public void testStopQueryLocalNoRemotes() throws Exception {
+        setupClusters(3);
+        populateRuntimeIndex(LOCAL_CLUSTER, "pause", INDEX_WITH_RUNTIME_MAPPING);
+
+        Tuple includeCCSMetadata = randomIncludeCCSMetadata();
+        boolean responseExpectMeta = includeCCSMetadata.v2();
+
+        final String asyncExecutionId = startAsyncQuery(
+            client(),
+            "FROM blocking | STATS total=count(const) | LIMIT 1",
+            includeCCSMetadata.v1()
+        );
+
+        // wait until we know that the query against 'remote-b:blocking' has started
+        SimplePauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+
+        /* at this point:
+         *  the query against the local cluster should be running because it's blocked
+         */
+
+        // run the stop query
+        var stopRequest = new AsyncStopRequest(asyncExecutionId);
+        var stopAction = client().execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        // allow local query to proceed
+        SimplePauseFieldPlugin.allowEmitting.countDown();
+
+        try (EsqlQueryResponse asyncResponse = stopAction.actionGet(30, TimeUnit.SECONDS)) {
+            assertThat(asyncResponse.isRunning(), is(false));
+            assertThat(asyncResponse.columns().size(), equalTo(1));
+            assertThat(asyncResponse.values().hasNext(), is(true));
+            Iterator row = asyncResponse.values().next();
+            assertThat((long) row.next(), greaterThanOrEqualTo(0L));
+
+            EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
+            assertNotNull(executionInfo);
+            assertThat(executionInfo.isCrossClusterSearch(), is(false));
+        } finally {
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
         }
     }
 
-    AcknowledgedResponse deleteAsyncId(String id) {
+    public void testAsyncFailure() throws Exception {
+        Map testClusterInfo = setupClusters(2);
+        populateRuntimeIndex(REMOTE_CLUSTER_1, "pause_fail", INDEX_WITH_FAIL_MAPPING);
+
+        Tuple includeCCSMetadata = randomIncludeCCSMetadata();
+        final String asyncExecutionId = startAsyncQuery(
+            client(),
+            "FROM logs-*,cluster-a:failing | STATS total=sum(const) | LIMIT 1",
+            includeCCSMetadata.v1()
+        );
+        // wait until we know that the query against remote has started
+        FailingPauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+        // Allow to proceed
+        FailingPauseFieldPlugin.allowEmitting.countDown();
+
+        // wait until local queries have finished
         try {
-            DeleteAsyncResultRequest request = new DeleteAsyncResultRequest(id);
-            return client().execute(TransportDeleteAsyncResultAction.TYPE, request).actionGet(30, TimeUnit.SECONDS);
-        } catch (ElasticsearchTimeoutException e) {
-            throw new AssertionError("timeout waiting for DELETE response", e);
+            assertBusy(() -> assertThrows(Exception.class, () -> getAsyncResponse(client(), asyncExecutionId)));
+            // Ensure stop query fails too when get fails
+            assertThrows(
+                ElasticsearchException.class,
+                () -> client().execute(EsqlAsyncStopAction.INSTANCE, new AsyncStopRequest(asyncExecutionId)).actionGet()
+            );
+        } finally {
+            assertAcked(deleteAsyncId(client(), asyncExecutionId));
         }
     }
 
-    EsqlQueryResponse getAsyncResponse(String id) {
-        try {
-            var getResultsRequest = new GetAsyncResultRequest(id).setWaitForCompletionTimeout(timeValueMillis(1));
-            return client().execute(EsqlAsyncGetResultAction.INSTANCE, getResultsRequest).actionGet(30, TimeUnit.SECONDS);
-        } catch (ElasticsearchTimeoutException e) {
-            throw new AssertionError("timeout waiting for GET async result", e);
+    private String randomAsyncId() {
+        return AsyncExecutionId.encode(randomAlphaOfLength(10), new TaskId(randomAlphaOfLength(10), randomLong()));
+    }
+
+    public void testBadAsyncId() throws Exception {
+        setupClusters(3);
+        final AtomicReference asyncId = new AtomicReference<>();
+        try (
+            EsqlQueryResponse resp = runAsyncQuery(
+                client(),
+                "FROM logs-*,*:logs-* | STATS total=sum(const) | LIMIT 1",
+                randomBoolean(),
+                TimeValue.timeValueMillis(0)
+            )
+        ) {
+            assertTrue(resp.isRunning());
+            asyncId.set(resp.asyncExecutionId().get());
         }
+        assertBusy(() -> {
+            try (EsqlQueryResponse resp = getAsyncResponse(client(), asyncId.get())) {
+                assertFalse(resp.isRunning());
+            }
+        });
+
+        String randomAsyncIdasyncId = randomAsyncId();
+        var stopRequest = new AsyncStopRequest(randomAsyncIdasyncId);
+        var stopAction = client().execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        assertThrows(ResourceNotFoundException.class, () -> stopAction.actionGet(1000, TimeUnit.SECONDS));
+    }
+
+    private void assertClusterInfoSuccess(EsqlExecutionInfo.Cluster cluster, int numShards) {
+        assertThat(cluster.getTook().millis(), greaterThanOrEqualTo(0L));
+        assertThat(cluster.getStatus(), equalTo(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL));
+        assertThat(cluster.getTotalShards(), equalTo(numShards));
+        assertThat(cluster.getSuccessfulShards(), equalTo(numShards));
+        assertThat(cluster.getSkippedShards(), equalTo(0));
+        assertThat(cluster.getFailedShards(), equalTo(0));
+        assertThat(cluster.getFailures().size(), equalTo(0));
     }
 
     private static void assertClusterMetadataInResponse(EsqlQueryResponse resp, boolean responseExpectMeta, int numClusters) {
@@ -373,11 +565,8 @@ Map setupClusters(int numClusters) throws IOException {
         if (numClusters == 3) {
             int numShardsRemote2 = randomIntBetween(1, 5);
             populateRemoteIndices(REMOTE_CLUSTER_2, REMOTE_INDEX, numShardsRemote2);
-            populateRemoteIndicesWithRuntimeMapping(REMOTE_CLUSTER_2);
             clusterInfo.put("remote2.index", REMOTE_INDEX);
             clusterInfo.put("remote2.num_shards", numShardsRemote2);
-            clusterInfo.put("remote2.blocking_index", INDEX_WITH_RUNTIME_MAPPING);
-            clusterInfo.put("remote2.blocking_index.num_shards", 1);
         }
 
         String skipUnavailableKey = Strings.format("cluster.remote.%s.skip_unavailable", REMOTE_CLUSTER_1);
@@ -405,23 +594,26 @@ void populateLocalIndices(String indexName, int numShards) {
         localClient.admin().indices().prepareRefresh(indexName).get();
     }
 
-    void populateRemoteIndicesWithRuntimeMapping(String clusterAlias) throws IOException {
+    void populateRuntimeIndex(String clusterAlias, String langName, String indexName) throws IOException {
+        populateRuntimeIndex(clusterAlias, langName, indexName, 10);
+    }
+
+    void populateRuntimeIndex(String clusterAlias, String langName, String indexName, int count) throws IOException {
         XContentBuilder mapping = JsonXContent.contentBuilder().startObject();
         mapping.startObject("runtime");
         {
             mapping.startObject("const");
             {
                 mapping.field("type", "long");
-                mapping.startObject("script").field("source", "").field("lang", "pause").endObject();
+                mapping.startObject("script").field("source", "").field("lang", langName).endObject();
             }
             mapping.endObject();
         }
         mapping.endObject();
         mapping.endObject();
-        client(clusterAlias).admin().indices().prepareCreate(INDEX_WITH_RUNTIME_MAPPING).setMapping(mapping).get();
-        BulkRequestBuilder bulk = client(clusterAlias).prepareBulk(INDEX_WITH_RUNTIME_MAPPING)
-            .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
-        for (int i = 0; i < 10; i++) {
+        client(clusterAlias).admin().indices().prepareCreate(indexName).setMapping(mapping).get();
+        BulkRequestBuilder bulk = client(clusterAlias).prepareBulk(indexName).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+        for (int i = 0; i < count; i++) {
             bulk.add(new IndexRequest().source("foo", i));
         }
         bulk.get();
@@ -441,4 +633,26 @@ void populateRemoteIndices(String clusterAlias, String indexName, int numShards)
         }
         remoteClient.admin().indices().prepareRefresh(indexName).get();
     }
+
+    public static class CountingPauseFieldPlugin extends SimplePauseFieldPlugin {
+        public static AtomicLong count = new AtomicLong(0);
+
+        protected String scriptTypeName() {
+            return "pause_count";
+        }
+
+        public static void resetPlugin() {
+            count.set(0);
+        }
+
+        @Override
+        public boolean onWait() throws InterruptedException {
+            count.incrementAndGet();
+            return allowEmitting.await(30, TimeUnit.SECONDS);
+        }
+    }
+
+    private static List getDriverTasks(Client client) {
+        return client.admin().cluster().prepareListTasks().setActions(DriverTaskRunner.ACTION_NAME).setDetailed(true).get().getTasks();
+    }
 }
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersUsageTelemetryIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersUsageTelemetryIT.java
index cd30ab02676fc..89f7fdca79135 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersUsageTelemetryIT.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersUsageTelemetryIT.java
@@ -7,29 +7,51 @@
 
 package org.elasticsearch.xpack.esql.action;
 
+import org.elasticsearch.action.ActionFuture;
 import org.elasticsearch.action.admin.cluster.stats.CCSTelemetrySnapshot;
 import org.elasticsearch.action.admin.cluster.stats.CCSUsageTelemetry;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.support.WriteRequest;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.SkipUnavailableRule;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.json.JsonXContent;
+import org.elasticsearch.xpack.core.async.AsyncStopRequest;
+import org.junit.Before;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.elasticsearch.action.admin.cluster.stats.CCSUsageTelemetry.ASYNC_FEATURE;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse;
+import static org.elasticsearch.xpack.esql.action.EsqlAsyncTestUtils.deleteAsyncId;
 import static org.hamcrest.Matchers.equalTo;
 
 public class CrossClustersUsageTelemetryIT extends AbstractCrossClustersUsageTelemetryIT {
+    private static final String INDEX_WITH_RUNTIME_MAPPING = "blocking";
 
     @Override
     protected Collection> nodePlugins(String clusterAlias) {
         List> plugins = new ArrayList<>(super.nodePlugins(clusterAlias));
         plugins.add(EsqlPluginWithEnterpriseOrTrialLicense.class);
         plugins.add(CrossClustersQueryIT.InternalExchangePlugin.class);
+        plugins.add(SimplePauseFieldPlugin.class);
+        plugins.add(EsqlAsyncActionIT.LocalStateEsqlAsync.class); // allows the async_search DELETE action
         return plugins;
     }
 
+    @Before
+    public void resetPlugin() {
+        SimplePauseFieldPlugin.resetPlugin();
+    }
+
     public void assertPerClusterCount(CCSTelemetrySnapshot.PerClusterCCSTelemetry perCluster, long count) {
         assertThat(perCluster.getCount(), equalTo(count));
         assertThat(perCluster.getSkippedCount(), equalTo(0L));
@@ -202,6 +224,59 @@ public void testAsync() throws Exception {
         assertPerClusterCount(perCluster.get(LOCAL_CLUSTER), 2L);
     }
 
+    public void testAsyncStop() throws Exception {
+        setupClusters();
+        populateRuntimeIndex(REMOTE1, "pause", INDEX_WITH_RUNTIME_MAPPING);
+        populateRuntimeIndex(REMOTE2, "pause", INDEX_WITH_RUNTIME_MAPPING);
+
+        EsqlQueryRequest request = EsqlQueryRequest.asyncEsqlQueryRequest();
+        request.query("from logs-*,c*:logs-*,c*:blocking | eval v1=coalesce(const, v) | stats sum (v1)");
+        request.pragmas(AbstractEsqlIntegTestCase.randomPragmas());
+        request.columnar(randomBoolean());
+        request.includeCCSMetadata(randomBoolean());
+
+        AtomicReference asyncExecutionId = new AtomicReference<>();
+        assertResponse(cluster(LOCAL_CLUSTER).client(queryNode).execute(EsqlQueryAction.INSTANCE, request), resp -> {
+            if (resp.isRunning()) {
+                assertNotNull("async execution id is null", resp.asyncExecutionId());
+                asyncExecutionId.set(resp.asyncExecutionId().get());
+            }
+        });
+        SimplePauseFieldPlugin.startEmitting.await(30, TimeUnit.SECONDS);
+        AsyncStopRequest stopRequest = new AsyncStopRequest(asyncExecutionId.get());
+        ActionFuture actionFuture = cluster(LOCAL_CLUSTER).client(queryNode)
+            .execute(EsqlAsyncStopAction.INSTANCE, stopRequest);
+        // Release the pause
+        SimplePauseFieldPlugin.allowEmitting.countDown();
+        try (EsqlQueryResponse resp = actionFuture.actionGet(30, TimeUnit.SECONDS)) {
+            assertTrue(resp.getExecutionInfo().isPartial());
+
+            CCSTelemetrySnapshot telemetry = getTelemetrySnapshot(queryNode);
+
+            assertThat(telemetry.getTotalCount(), equalTo(1L));
+            assertThat(telemetry.getSuccessCount(), equalTo(1L));
+            assertThat(telemetry.getFailureReasons().size(), equalTo(0));
+            assertThat(telemetry.getTook().count(), equalTo(1L));
+            assertThat(telemetry.getTookMrtFalse().count(), equalTo(0L));
+            assertThat(telemetry.getTookMrtTrue().count(), equalTo(0L));
+            assertThat(telemetry.getRemotesPerSearchAvg(), equalTo(2.0));
+            assertThat(telemetry.getRemotesPerSearchMax(), equalTo(2L));
+            assertThat(telemetry.getSearchCountWithSkippedRemotes(), equalTo(0L));
+            assertThat(telemetry.getClientCounts().size(), equalTo(0));
+            assertThat(telemetry.getFeatureCounts().get(ASYNC_FEATURE), equalTo(1L));
+
+            var perCluster = telemetry.getByRemoteCluster();
+            assertThat(perCluster.size(), equalTo(3));
+            for (String clusterAlias : remoteClusterAlias()) {
+                assertPerClusterCount(perCluster.get(clusterAlias), 1L);
+            }
+            assertPerClusterCount(perCluster.get(LOCAL_CLUSTER), 1L);
+        } finally {
+            // Clean up
+            assertAcked(deleteAsyncId(client(), asyncExecutionId.get()));
+        }
+    }
+
     public void testNoSuchCluster() throws Exception {
         setupClusters();
         // This is not recognized as a cross-cluster search
@@ -225,4 +300,25 @@ public void testDisconnect() throws Exception {
         assertThat(telemetry.getFailureReasons(), equalTo(expectedFailure));
     }
 
+    void populateRuntimeIndex(String clusterAlias, String langName, String indexName) throws IOException {
+        XContentBuilder mapping = JsonXContent.contentBuilder().startObject();
+        mapping.startObject("runtime");
+        {
+            mapping.startObject("const");
+            {
+                mapping.field("type", "long");
+                mapping.startObject("script").field("source", "").field("lang", langName).endObject();
+            }
+            mapping.endObject();
+        }
+        mapping.endObject();
+        mapping.endObject();
+        client(clusterAlias).admin().indices().prepareCreate(indexName).setMapping(mapping).get();
+        BulkRequestBuilder bulk = client(clusterAlias).prepareBulk(indexName).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+        for (int i = 0; i < 10; i++) {
+            bulk.add(new IndexRequest().source("foo", i));
+        }
+        bulk.get();
+    }
+
 }
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java
index 8e27cfceb28e6..2d0a15436bf82 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java
@@ -362,7 +362,8 @@ private void assertCancelled(ActionFuture response) throws Ex
                     "task cancelled",
                     "request cancelled test cancel",
                     "parent task was cancelled [test cancel]",
-                    "cancelled on failure"
+                    "cancelled on failure",
+                    "task cancelled [cancelled on failure]"
                 )
             )
         );
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlAsyncTestUtils.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlAsyncTestUtils.java
new file mode 100644
index 0000000000000..d7117fb5e0750
--- /dev/null
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlAsyncTestUtils.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.action;
+
+import org.elasticsearch.ElasticsearchTimeoutException;
+import org.elasticsearch.action.support.master.AcknowledgedResponse;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.Nullable;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.transport.RemoteClusterAware;
+import org.elasticsearch.xpack.core.async.DeleteAsyncResultRequest;
+import org.elasticsearch.xpack.core.async.GetAsyncResultRequest;
+import org.elasticsearch.xpack.core.async.TransportDeleteAsyncResultAction;
+import org.elasticsearch.xpack.esql.plugin.QueryPragmas;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import static org.elasticsearch.core.TimeValue.timeValueMillis;
+import static org.elasticsearch.test.ESTestCase.assertBusy;
+import static org.elasticsearch.test.ESTestCase.assertThat;
+import static org.elasticsearch.test.ESTestCase.randomBoolean;
+import static org.elasticsearch.test.ESTestCase.randomInt;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public final class EsqlAsyncTestUtils {
+    public static String startAsyncQuery(Client client, String q, Boolean includeCCSMetadata) {
+        return startAsyncQueryWithPragmas(client, q, includeCCSMetadata, null);
+    }
+
+    public static String startAsyncQueryWithPragmas(Client client, String q, Boolean includeCCSMetadata, Map pragmas) {
+        try (EsqlQueryResponse resp = runAsyncQuery(client, q, includeCCSMetadata, TimeValue.timeValueMillis(100), pragmas)) {
+            assertTrue(resp.isRunning());
+            assertNotNull("async execution id is null", resp.asyncExecutionId());
+            // executionInfo may or may not be set on the initial response when there is a relatively low wait_for_completion_timeout
+            // so we do not check for it here
+            return resp.asyncExecutionId().get();
+        }
+    }
+
+    public static EsqlQueryResponse runAsyncQuery(Client client, String query, Boolean ccsMetadata, TimeValue waitCompletionTime) {
+        return runAsyncQuery(client, query, ccsMetadata, waitCompletionTime, null);
+    }
+
+    private static QueryPragmas randomPragmasWithOverride(@Nullable Map pragmas) {
+        if (pragmas == null || pragmas.isEmpty()) {
+            return AbstractEsqlIntegTestCase.randomPragmas();
+        }
+        Settings.Builder settings = Settings.builder();
+        settings.put(AbstractEsqlIntegTestCase.randomPragmas().getSettings());
+        settings.loadFromMap(pragmas);
+        return new QueryPragmas(settings.build());
+    }
+
+    public static EsqlQueryResponse runAsyncQuery(
+        Client client,
+        String query,
+        Boolean ccsMetadata,
+        TimeValue waitCompletionTime,
+        @Nullable Map pragmas
+    ) {
+        EsqlQueryRequest request = EsqlQueryRequest.asyncEsqlQueryRequest();
+        request.query(query);
+        request.pragmas(randomPragmasWithOverride(pragmas));
+        request.profile(randomInt(5) == 2);
+        request.columnar(randomBoolean());
+        if (ccsMetadata != null) {
+            request.includeCCSMetadata(ccsMetadata);
+        }
+        request.waitForCompletionTimeout(waitCompletionTime);
+        request.keepOnCompletion(true);
+        return runAsyncQuery(client, request);
+    }
+
+    /**
+     * Wait for the cluster to finish running the query.
+     */
+    public static void waitForCluster(Client client, String clusterName, String asyncExecutionId) throws Exception {
+        assertBusy(() -> {
+            try (EsqlQueryResponse asyncResponse = getAsyncResponse(client, asyncExecutionId)) {
+                EsqlExecutionInfo executionInfo = asyncResponse.getExecutionInfo();
+                assertNotNull(executionInfo);
+                EsqlExecutionInfo.Cluster clusterInfo = executionInfo.getCluster(clusterName);
+                // the status of the local cluster won't move to SUCCESS until the reduction pipeline is done
+                if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterName)
+                    && Objects.requireNonNullElse(clusterInfo.getTotalShards(), 0) > 0) {
+                    return;
+                }
+                assertThat(clusterInfo.getStatus(), not(equalTo(EsqlExecutionInfo.Cluster.Status.RUNNING)));
+            }
+        });
+    }
+
+    public static EsqlQueryResponse runAsyncQuery(Client client, EsqlQueryRequest request) {
+        try {
+            return client.execute(EsqlQueryAction.INSTANCE, request).actionGet(30, TimeUnit.SECONDS);
+        } catch (ElasticsearchTimeoutException e) {
+            throw new AssertionError("timeout waiting for query response", e);
+        }
+    }
+
+    public static AcknowledgedResponse deleteAsyncId(Client client, String id) {
+        try {
+            DeleteAsyncResultRequest request = new DeleteAsyncResultRequest(id);
+            return client.execute(TransportDeleteAsyncResultAction.TYPE, request).actionGet(30, TimeUnit.SECONDS);
+        } catch (ElasticsearchTimeoutException e) {
+            throw new AssertionError("timeout waiting for DELETE response", e);
+        }
+    }
+
+    public static EsqlQueryResponse getAsyncResponse(Client client, String id) {
+        try {
+            var getResultsRequest = new GetAsyncResultRequest(id).setWaitForCompletionTimeout(timeValueMillis(1));
+            return client.execute(EsqlAsyncGetResultAction.INSTANCE, getResultsRequest).actionGet(30, TimeUnit.SECONDS);
+        } catch (ElasticsearchTimeoutException e) {
+            throw new AssertionError("timeout waiting for GET async result", e);
+        }
+    }
+}
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/FailingPauseFieldPlugin.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/FailingPauseFieldPlugin.java
new file mode 100644
index 0000000000000..010931432e2e8
--- /dev/null
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/FailingPauseFieldPlugin.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.action;
+
+import org.elasticsearch.ElasticsearchException;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A plugin that provides a script language "pause_fail" that can be used to make queries fail in a predictable way.
+ */
+public class FailingPauseFieldPlugin extends AbstractPauseFieldPlugin {
+    public static CountDownLatch startEmitting = new CountDownLatch(1);
+    public static CountDownLatch allowEmitting = new CountDownLatch(1);
+
+    @Override
+    protected String scriptTypeName() {
+        return "pause_fail";
+    }
+
+    public static void resetPlugin() {
+        allowEmitting = new CountDownLatch(1);
+        startEmitting = new CountDownLatch(1);
+    }
+
+    @Override
+    public void onStartExecute() {
+        startEmitting.countDown();
+    }
+
+    @Override
+    public boolean onWait() throws InterruptedException {
+        allowEmitting.await(30, TimeUnit.SECONDS);
+        throw new ElasticsearchException("Failing query");
+    }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlAsyncStopAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlAsyncStopAction.java
new file mode 100644
index 0000000000000..2178d0bf1e9fd
--- /dev/null
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlAsyncStopAction.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.action;
+
+import org.elasticsearch.action.ActionType;
+import org.elasticsearch.xpack.core.esql.EsqlAsyncActionNames;
+
+public class EsqlAsyncStopAction extends ActionType {
+
+    public static final EsqlAsyncStopAction INSTANCE = new EsqlAsyncStopAction();
+
+    public static final String NAME = EsqlAsyncActionNames.ESQL_ASYNC_STOP_ACTION_NAME;
+
+    private EsqlAsyncStopAction() {
+        super(NAME);
+    }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlExecutionInfo.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlExecutionInfo.java
index 077eb7a721003..e4389255abc8e 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlExecutionInfo.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlExecutionInfo.java
@@ -18,6 +18,7 @@
 import org.elasticsearch.core.Predicates;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.rest.action.RestActions;
+import org.elasticsearch.transport.NoSuchRemoteClusterException;
 import org.elasticsearch.transport.RemoteClusterAware;
 import org.elasticsearch.transport.RemoteClusterService;
 import org.elasticsearch.xcontent.ParseField;
@@ -56,6 +57,7 @@ public class EsqlExecutionInfo implements ChunkedToXContentObject, Writeable {
     public static final ParseField FAILED_FIELD = new ParseField("failed");
     public static final ParseField DETAILS_FIELD = new ParseField("details");
     public static final ParseField TOOK = new ParseField("took");
+    public static final ParseField IS_PARTIAL_FIELD = new ParseField("is_partial");
 
     // Map key is clusterAlias on the primary querying cluster of a CCS minimize_roundtrips=true query
     // The Map itself is immutable after construction - all Clusters will be accounted for at the start of the search.
@@ -70,6 +72,7 @@ public class EsqlExecutionInfo implements ChunkedToXContentObject, Writeable {
     private final transient Predicate skipUnavailablePredicate;
     private final transient Long relativeStartNanos;  // start time for an ESQL query for calculating took times
     private transient TimeValue planningTookTime;  // time elapsed since start of query to calling ComputeService.execute
+    private volatile boolean isPartial; // Does this request have partial results?
 
     public EsqlExecutionInfo(boolean includeCCSMetadata) {
         this(Predicates.always(), includeCCSMetadata);  // default all clusters to skip_unavailable=true
@@ -112,6 +115,13 @@ public EsqlExecutionInfo(StreamInput in) throws IOException {
         } else {
             this.includeCCSMetadata = false;
         }
+
+        if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_RESPONSE_PARTIAL)) {
+            this.isPartial = in.readBoolean();
+        } else {
+            this.isPartial = false;
+        }
+
         this.skipUnavailablePredicate = Predicates.always();
         this.relativeStartNanos = null;
     }
@@ -127,6 +137,9 @@ public void writeTo(StreamOutput out) throws IOException {
         if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) {
             out.writeBoolean(includeCCSMetadata);
         }
+        if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_RESPONSE_PARTIAL)) {
+            out.writeBoolean(isPartial);
+        }
     }
 
     public boolean includeCCSMetadata() {
@@ -187,7 +200,7 @@ public Set clusterAliases() {
     /**
      * @param clusterAlias to lookup skip_unavailable from
      * @return skip_unavailable setting (true/false)
-     * @throws org.elasticsearch.transport.NoSuchRemoteClusterException if clusterAlias is unknown to this node's RemoteClusterService
+     * @throws NoSuchRemoteClusterException if clusterAlias is unknown to this node's RemoteClusterService
      */
     public boolean isSkipUnavailable(String clusterAlias) {
         if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias)) {
@@ -276,6 +289,24 @@ public int hashCode() {
         return Objects.hash(clusterInfo, overallTook);
     }
 
+    public boolean isPartial() {
+        return isPartial;
+    }
+
+    /**
+     * Mark the query as having partial results.
+     */
+    public void markAsPartial() {
+        isPartial = true;
+    }
+
+    /**
+     * Mark this cluster as having partial results.
+     */
+    public void markClusterAsPartial(String clusterAlias) {
+        swapCluster(clusterAlias, (k, v) -> new Cluster.Builder(v).setStatus(Cluster.Status.PARTIAL).build());
+    }
+
     /**
      * Represents the search metadata about a particular cluster involved in a cross-cluster search.
      * The Cluster object can represent either the local cluster or a remote cluster.
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java
index 4163a222b1a28..c6ab7d9ac7ddb 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java
@@ -201,6 +201,7 @@ public Iterator toXContentChunked(ToXContent.Params params
                     ? executionInfo.tookSoFar().millis()
                     : executionInfo.overallTook().millis();
                 b.field("took", tookInMillis);
+                b.field(EsqlExecutionInfo.IS_PARTIAL_FIELD.getPreferredName(), executionInfo.isPartial());
             }
             if (dropNullColumns) {
                 b.append(ResponseXContentUtils.allColumns(columns, "all_columns"))
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlStopAsyncAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlStopAsyncAction.java
new file mode 100644
index 0000000000000..c7477f738e95a
--- /dev/null
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlStopAsyncAction.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.action;
+
+import org.elasticsearch.client.internal.node.NodeClient;
+import org.elasticsearch.rest.BaseRestHandler;
+import org.elasticsearch.rest.RestRequest;
+import org.elasticsearch.rest.Scope;
+import org.elasticsearch.rest.ServerlessScope;
+import org.elasticsearch.xpack.core.async.AsyncStopRequest;
+
+import java.util.List;
+import java.util.Set;
+
+import static org.elasticsearch.rest.RestRequest.Method.POST;
+import static org.elasticsearch.xpack.esql.action.EsqlQueryResponse.DROP_NULL_COLUMNS_OPTION;
+import static org.elasticsearch.xpack.esql.formatter.TextFormat.URL_PARAM_DELIMITER;
+
+@ServerlessScope(Scope.PUBLIC)
+public class RestEsqlStopAsyncAction extends BaseRestHandler {
+    @Override
+    public List routes() {
+        return List.of(new Route(POST, "/_query/async/{id}/stop"));
+    }
+
+    @Override
+    public String getName() {
+        return "esql_async_stop";
+    }
+
+    @Override
+    protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) {
+        AsyncStopRequest stopReq = new AsyncStopRequest(request.param("id"));
+        return channel -> client.execute(EsqlAsyncStopAction.INSTANCE, stopReq, new EsqlResponseListener(channel, request));
+    }
+
+    @Override
+    protected Set responseParams() {
+        return Set.of(URL_PARAM_DELIMITER, DROP_NULL_COLUMNS_OPTION);
+    }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
index 7d99cf5988597..75619958c5228 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
@@ -55,9 +55,9 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
 import java.util.function.Supplier;
 
 import static org.elasticsearch.xpack.esql.plugin.EsqlPlugin.ESQL_WORKER_THREAD_POOL_NAME;
@@ -83,6 +83,7 @@ public class ComputeService {
     private final AtomicLong childSessionIdGenerator = new AtomicLong();
     private final DataNodeComputeHandler dataNodeComputeHandler;
     private final ClusterComputeHandler clusterComputeHandler;
+    private final ExchangeService exchangeService;
 
     @SuppressWarnings("this-escape")
     public ComputeService(
@@ -113,6 +114,7 @@ public ComputeService(
             esqlExecutor,
             dataNodeComputeHandler
         );
+        this.exchangeService = exchangeService;
     }
 
     public void execute(
@@ -195,11 +197,12 @@ public void execute(
             var exchangeSource = new ExchangeSourceHandler(
                 queryPragmas.exchangeBufferSize(),
                 transportService.getThreadPool().executor(ThreadPool.Names.SEARCH),
-                computeListener.acquireAvoid()
+                ActionListener.runBefore(computeListener.acquireAvoid(), () -> exchangeService.removeExchangeSourceHandler(sessionId))
             );
+            exchangeService.addExchangeSourceHandler(sessionId, exchangeSource);
             try (Releasable ignored = exchangeSource.addEmptySink()) {
                 // run compute on the coordinator
-                final AtomicReference localResponse = new AtomicReference<>(new ComputeResponse(List.of()));
+                final AtomicBoolean localClusterWasInterrupted = new AtomicBoolean();
                 try (
                     var localListener = new ComputeListener(
                         transportService.getThreadPool(),
@@ -207,16 +210,13 @@ public void execute(
                         computeListener.acquireCompute().delegateFailure((l, profiles) -> {
                             if (execInfo.isCrossClusterSearch() && execInfo.clusterAliases().contains(LOCAL_CLUSTER)) {
                                 var tookTime = TimeValue.timeValueNanos(System.nanoTime() - execInfo.getRelativeStartNanos());
-                                var r = localResponse.get();
-                                var merged = new ComputeResponse(
-                                    profiles,
-                                    tookTime,
-                                    r.totalShards,
-                                    r.successfulShards,
-                                    r.skippedShards,
-                                    r.failedShards
+                                var status = localClusterWasInterrupted.get()
+                                    ? EsqlExecutionInfo.Cluster.Status.PARTIAL
+                                    : EsqlExecutionInfo.Cluster.Status.SUCCESSFUL;
+                                execInfo.swapCluster(
+                                    LOCAL_CLUSTER,
+                                    (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v).setStatus(status).setTook(tookTime).build()
                                 );
-                                updateExecutionInfo(execInfo, LOCAL_CLUSTER, merged);
                             }
                             l.onResponse(profiles);
                         })
@@ -241,7 +241,17 @@ public void execute(
                             exchangeSource,
                             cancelQueryOnFailure,
                             localListener.acquireCompute().map(r -> {
-                                localResponse.set(r);
+                                localClusterWasInterrupted.set(execInfo.isPartial());
+                                if (execInfo.isCrossClusterSearch() && execInfo.clusterAliases().contains(LOCAL_CLUSTER)) {
+                                    execInfo.swapCluster(
+                                        LOCAL_CLUSTER,
+                                        (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v).setTotalShards(r.getTotalShards())
+                                            .setSuccessfulShards(r.getSuccessfulShards())
+                                            .setSkippedShards(r.getSkippedShards())
+                                            .setFailedShards(r.getFailedShards())
+                                            .build()
+                                    );
+                                }
                                 return r.getProfiles();
                             })
                         );
@@ -269,22 +279,19 @@ public void execute(
     }
 
     private void updateExecutionInfo(EsqlExecutionInfo executionInfo, String clusterAlias, ComputeResponse resp) {
-        TimeValue tookOnCluster;
-        if (resp.getTook() != null) {
-            TimeValue remoteExecutionTime = resp.getTook();
-            final long planningTime;
-            if (clusterAlias.equals(LOCAL_CLUSTER)) {
-                planningTime = 0L;
+        Function runningToSuccess = status -> {
+            if (status == EsqlExecutionInfo.Cluster.Status.RUNNING) {
+                return executionInfo.isPartial() ? EsqlExecutionInfo.Cluster.Status.PARTIAL : EsqlExecutionInfo.Cluster.Status.SUCCESSFUL;
             } else {
-                planningTime = executionInfo.planningTookTime().nanos();
+                return status;
             }
-            tookOnCluster = new TimeValue(planningTime + remoteExecutionTime.nanos(), TimeUnit.NANOSECONDS);
+        };
+        if (resp.getTook() != null) {
+            var tookTime = TimeValue.timeValueNanos(executionInfo.planningTookTime().nanos() + resp.getTook().nanos());
             executionInfo.swapCluster(
                 clusterAlias,
-                (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v)
-                    // for now ESQL doesn't return partial results, so set status to SUCCESSFUL
-                    .setStatus(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL)
-                    .setTook(tookOnCluster)
+                (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v).setStatus(runningToSuccess.apply(v.getStatus()))
+                    .setTook(tookTime)
                     .setTotalShards(resp.getTotalShards())
                     .setSuccessfulShards(resp.getSuccessfulShards())
                     .setSkippedShards(resp.getSkippedShards())
@@ -294,14 +301,11 @@ private void updateExecutionInfo(EsqlExecutionInfo executionInfo, String cluster
         } else {
             // if the cluster is an older version and does not send back took time, then calculate it here on the coordinator
             // and leave shard info unset, so it is not shown in the CCS metadata section of the JSON response
-            long remoteTook = System.nanoTime() - executionInfo.getRelativeStartNanos();
-            tookOnCluster = new TimeValue(remoteTook, TimeUnit.NANOSECONDS);
+            var tookTime = TimeValue.timeValueNanos(System.nanoTime() - executionInfo.getRelativeStartNanos());
             executionInfo.swapCluster(
                 clusterAlias,
-                (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v)
-                    // for now ESQL doesn't return partial results, so set status to SUCCESSFUL
-                    .setStatus(EsqlExecutionInfo.Cluster.Status.SUCCESSFUL)
-                    .setTook(tookOnCluster)
+                (k, v) -> new EsqlExecutionInfo.Cluster.Builder(v).setStatus(runningToSuccess.apply(v.getStatus()))
+                    .setTook(tookTime)
                     .build()
             );
         }
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
index b79dda900f39c..4379e2e8041ae 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
@@ -52,6 +52,7 @@
 import org.elasticsearch.xpack.esql.EsqlInfoTransportAction;
 import org.elasticsearch.xpack.esql.EsqlUsageTransportAction;
 import org.elasticsearch.xpack.esql.action.EsqlAsyncGetResultAction;
+import org.elasticsearch.xpack.esql.action.EsqlAsyncStopAction;
 import org.elasticsearch.xpack.esql.action.EsqlQueryAction;
 import org.elasticsearch.xpack.esql.action.EsqlQueryRequestBuilder;
 import org.elasticsearch.xpack.esql.action.EsqlResolveFieldsAction;
@@ -60,6 +61,7 @@
 import org.elasticsearch.xpack.esql.action.RestEsqlDeleteAsyncResultAction;
 import org.elasticsearch.xpack.esql.action.RestEsqlGetAsyncResultAction;
 import org.elasticsearch.xpack.esql.action.RestEsqlQueryAction;
+import org.elasticsearch.xpack.esql.action.RestEsqlStopAsyncAction;
 import org.elasticsearch.xpack.esql.enrich.EnrichLookupOperator;
 import org.elasticsearch.xpack.esql.enrich.LookupFromIndexOperator;
 import org.elasticsearch.xpack.esql.execution.PlanExecutor;
@@ -151,7 +153,8 @@ public List> getSettings() {
             new ActionHandler<>(XPackUsageFeatureAction.ESQL, EsqlUsageTransportAction.class),
             new ActionHandler<>(XPackInfoFeatureAction.ESQL, EsqlInfoTransportAction.class),
             new ActionHandler<>(EsqlResolveFieldsAction.TYPE, EsqlResolveFieldsAction.class),
-            new ActionHandler<>(EsqlSearchShardsAction.TYPE, EsqlSearchShardsAction.class)
+            new ActionHandler<>(EsqlSearchShardsAction.TYPE, EsqlSearchShardsAction.class),
+            new ActionHandler<>(EsqlAsyncStopAction.INSTANCE, TransportEsqlAsyncStopAction.class)
         );
     }
 
@@ -171,6 +174,7 @@ public List getRestHandlers(
             new RestEsqlQueryAction(),
             new RestEsqlAsyncQueryAction(),
             new RestEsqlGetAsyncResultAction(),
+            new RestEsqlStopAsyncAction(),
             new RestEsqlDeleteAsyncResultAction()
         );
     }
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncGetResultsAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncGetResultsAction.java
index 4bcebcfe64cb9..5658db0599186 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncGetResultsAction.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncGetResultsAction.java
@@ -75,7 +75,7 @@ public Writeable.Reader responseReader() {
 
     /**
      * Unwraps the exception in the case of failure. This keeps the exception types
-     * the same as the sync API, namely ParsingException and ParsingException.
+     * the same as the sync API, namely ParsingException and VerificationException.
      */
     static  ActionListener unwrapListener(ActionListener listener) {
         return new ActionListener<>() {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncStopAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncStopAction.java
new file mode 100644
index 0000000000000..a4007a520ed30
--- /dev/null
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlAsyncStopAction.java
@@ -0,0 +1,139 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.plugin;
+
+import org.elasticsearch.ResourceNotFoundException;
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.ActionListenerResponseHandler;
+import org.elasticsearch.action.support.ActionFilters;
+import org.elasticsearch.action.support.HandledTransportAction;
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.util.concurrent.EsExecutors;
+import org.elasticsearch.compute.EsqlRefCountingListener;
+import org.elasticsearch.compute.data.BlockFactory;
+import org.elasticsearch.compute.operator.exchange.ExchangeService;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.injection.guice.Inject;
+import org.elasticsearch.tasks.Task;
+import org.elasticsearch.tasks.TaskId;
+import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xpack.core.XPackPlugin;
+import org.elasticsearch.xpack.core.async.AsyncExecutionId;
+import org.elasticsearch.xpack.core.async.AsyncSearchSecurity;
+import org.elasticsearch.xpack.core.async.AsyncStopRequest;
+import org.elasticsearch.xpack.core.async.AsyncTaskIndexService;
+import org.elasticsearch.xpack.core.async.GetAsyncResultRequest;
+import org.elasticsearch.xpack.core.security.SecurityContext;
+import org.elasticsearch.xpack.esql.action.EsqlAsyncStopAction;
+import org.elasticsearch.xpack.esql.action.EsqlQueryResponse;
+import org.elasticsearch.xpack.esql.action.EsqlQueryTask;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.elasticsearch.xpack.core.ClientHelper.ASYNC_SEARCH_ORIGIN;
+
+/**
+ * This action will stop running async request and collect the results.
+ * If the request is already finished, it will do the same thing as the regular async get.
+ */
+public class TransportEsqlAsyncStopAction extends HandledTransportAction {
+
+    private final TransportEsqlQueryAction queryAction;
+    private final TransportEsqlAsyncGetResultsAction getResultsAction;
+    private final ExchangeService exchangeService;
+    private final BlockFactory blockFactory;
+    private final ClusterService clusterService;
+    private final TransportService transportService;
+    private final AsyncSearchSecurity security;
+
+    @Inject
+    public TransportEsqlAsyncStopAction(
+        TransportService transportService,
+        ClusterService clusterService,
+        ActionFilters actionFilters,
+        TransportEsqlQueryAction queryAction,
+        TransportEsqlAsyncGetResultsAction getResultsAction,
+        Client client,
+        ExchangeService exchangeService,
+        BlockFactory blockFactory
+    ) {
+        super(EsqlAsyncStopAction.NAME, transportService, actionFilters, AsyncStopRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
+        this.queryAction = queryAction;
+        this.getResultsAction = getResultsAction;
+        this.exchangeService = exchangeService;
+        this.blockFactory = blockFactory;
+        this.transportService = transportService;
+        this.clusterService = clusterService;
+        this.security = new AsyncSearchSecurity(
+            XPackPlugin.ASYNC_RESULTS_INDEX,
+            new SecurityContext(clusterService.getSettings(), client.threadPool().getThreadContext()),
+            client,
+            ASYNC_SEARCH_ORIGIN
+        );
+    }
+
+    @Override
+    protected void doExecute(Task task, AsyncStopRequest request, ActionListener listener) {
+        AsyncExecutionId searchId = AsyncExecutionId.decode(request.getId());
+        DiscoveryNode node = clusterService.state().nodes().get(searchId.getTaskId().getNodeId());
+        if (clusterService.localNode().getId().equals(searchId.getTaskId().getNodeId()) || node == null) {
+            stopQueryAndReturnResult(task, searchId, listener);
+        } else {
+            transportService.sendRequest(
+                node,
+                EsqlAsyncStopAction.NAME,
+                request,
+                new ActionListenerResponseHandler<>(listener, EsqlQueryResponse.reader(blockFactory), EsExecutors.DIRECT_EXECUTOR_SERVICE)
+            );
+        }
+    }
+
+    /**
+    * Returns the ID for stored compute session. See {@link TransportEsqlQueryAction#sessionID(Task)}
+    */
+    private String sessionID(AsyncExecutionId asyncId) {
+        return new TaskId(clusterService.localNode().getId(), asyncId.getTaskId().getId()).toString();
+    }
+
+    private void stopQueryAndReturnResult(Task task, AsyncExecutionId asyncId, ActionListener listener) {
+        String asyncIdStr = asyncId.getEncoded();
+        TransportEsqlQueryAction.EsqlQueryListener asyncListener = queryAction.getAsyncListener(asyncIdStr);
+        if (asyncListener == null) {
+            // This should mean one of the two things: either bad request ID, or the query has already finished
+            // In both cases, let regular async get deal with it.
+            var getAsyncResultRequest = new GetAsyncResultRequest(asyncIdStr);
+            // TODO: this should not be happening, but if the listener is not registered and the query is not finished,
+            // we give it some time to finish
+            getAsyncResultRequest.setWaitForCompletionTimeout(new TimeValue(1, TimeUnit.SECONDS));
+            getResultsAction.execute(task, getAsyncResultRequest, listener);
+            return;
+        }
+        try {
+            EsqlQueryTask asyncTask = AsyncTaskIndexService.getTask(taskManager, asyncId, EsqlQueryTask.class);
+            if (false == security.currentUserHasAccessToTask(asyncTask)) {
+                throw new ResourceNotFoundException(asyncId + " not found");
+            }
+        } catch (IOException e) {
+            throw new ResourceNotFoundException(asyncId + " not found", e);
+        }
+        // Here we will wait for both the response to become available and for the finish operation to complete
+        var responseHolder = new AtomicReference();
+        try (var refs = new EsqlRefCountingListener(listener.map(unused -> responseHolder.get()))) {
+            asyncListener.addListener(refs.acquire().map(r -> {
+                responseHolder.set(r);
+                return null;
+            }));
+            asyncListener.markAsPartial();
+            exchangeService.finishSessionEarly(sessionID(asyncId), refs.acquire());
+        }
+    }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
index d83239545c383..a32b4591943f4 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
@@ -13,12 +13,14 @@
 import org.elasticsearch.action.admin.cluster.stats.CCSUsageTelemetry;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.HandledTransportAction;
+import org.elasticsearch.action.support.SubscribableListener;
 import org.elasticsearch.client.internal.Client;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.compute.data.BlockFactory;
 import org.elasticsearch.compute.operator.exchange.ExchangeService;
@@ -81,6 +83,8 @@ public class TransportEsqlQueryAction extends HandledTransportAction asyncListeners = ConcurrentCollections.newConcurrentMap();
 
     @Inject
     @SuppressWarnings("this-escape")
@@ -179,11 +183,41 @@ private void doExecuteForked(Task task, EsqlQueryRequest request, ActionListener
         }
     }
 
+    // Subscribable listener that can keep track of the EsqlExecutionInfo
+    // Used to mark an async query as partial if it is stopped
+    public static class EsqlQueryListener extends SubscribableListener {
+        private EsqlExecutionInfo executionInfo;
+
+        public EsqlQueryListener(EsqlExecutionInfo executionInfo) {
+            this.executionInfo = executionInfo;
+        }
+
+        public EsqlExecutionInfo getExecutionInfo() {
+            return executionInfo;
+        }
+
+        public void markAsPartial() {
+            if (executionInfo != null) {
+                executionInfo.markAsPartial();
+            }
+        }
+    }
+
     @Override
     public void execute(EsqlQueryRequest request, EsqlQueryTask task, ActionListener listener) {
         // set EsqlExecutionInfo on async-search task so that it is accessible to GET _query/async while the query is still running
         task.setExecutionInfo(createEsqlExecutionInfo(request));
-        ActionListener.run(listener, l -> innerExecute(task, request, l));
+        // Since the request is async here, we need to wrap the listener in a SubscribableListener so that we can collect the results from
+        // other endpoints, such as _query/async/stop
+        EsqlQueryListener subListener = new EsqlQueryListener(task.executionInfo());
+        String asyncExecutionId = task.getExecutionId().getEncoded();
+        subListener.addListener(ActionListener.runAfter(listener, () -> asyncListeners.remove(asyncExecutionId)));
+        asyncListeners.put(asyncExecutionId, subListener);
+        ActionListener.run(subListener, l -> innerExecute(task, request, l));
+    }
+
+    public EsqlQueryListener getAsyncListener(String executionId) {
+        return asyncListeners.get(executionId);
     }
 
     private void innerExecute(Task task, EsqlQueryRequest request, ActionListener listener) {
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java
index 8cafc9e38ea01..b1e62f52428c0 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java
@@ -518,30 +518,66 @@ static EsqlQueryResponse fromXContent(XContentParser parser) {
         }
     }
 
+    public static int clusterDetailsSize(int numClusters) {
+        /* Example:
+        "_clusters" : {
+            "total" : 2,
+            "successful" : 2,
+            "running" : 0,
+            "skipped" : 0,
+            "partial" : 0,
+            "failed" : 0,
+            "details" : {
+                "(local)" : {
+                    "status" : "successful",
+                    "indices" : "logs-1",
+                    "took" : 4444,
+                    "_shards" : {
+                      "total" : 10,
+                      "successful" : 10,
+                      "skipped" : 3,
+                      "failed" : 0
+                    }
+                },
+                "remote1" : {
+                    "status" : "successful",
+                    "indices" : "remote1:logs-1",
+                    "took" : 4999,
+                    "_shards" : {
+                      "total" : 12,
+                      "successful" : 12,
+                      "skipped" : 5,
+                      "failed" : 0
+                    }
+                }
+            }
+         }
+         */
+        return numClusters * 4 + 6;
+    }
+
     public void testChunkResponseSizeColumnar() {
-        int sizeClusterDetails = 14;
         try (EsqlQueryResponse resp = randomResponse(true, null)) {
             int columnCount = resp.pages().get(0).getBlockCount();
             int bodySize = resp.pages().stream().mapToInt(p -> p.getPositionCount() * p.getBlockCount()).sum() + columnCount * 2;
-            assertChunkCount(resp, r -> 5 + sizeClusterDetails + bodySize);
+            assertChunkCount(resp, r -> 6 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize);
         }
 
         try (EsqlQueryResponse resp = randomResponseAsync(true, null, true)) {
             int columnCount = resp.pages().get(0).getBlockCount();
             int bodySize = resp.pages().stream().mapToInt(p -> p.getPositionCount() * p.getBlockCount()).sum() + columnCount * 2;
-            assertChunkCount(resp, r -> 7 + sizeClusterDetails + bodySize); // is_running
+            assertChunkCount(resp, r -> 8 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize); // is_running
         }
     }
 
     public void testChunkResponseSizeRows() {
-        int sizeClusterDetails = 14;
         try (EsqlQueryResponse resp = randomResponse(false, null)) {
             int bodySize = resp.pages().stream().mapToInt(Page::getPositionCount).sum();
-            assertChunkCount(resp, r -> 5 + sizeClusterDetails + bodySize);
+            assertChunkCount(resp, r -> 6 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize);
         }
         try (EsqlQueryResponse resp = randomResponseAsync(false, null, true)) {
             int bodySize = resp.pages().stream().mapToInt(Page::getPositionCount).sum();
-            assertChunkCount(resp, r -> 7 + sizeClusterDetails + bodySize);
+            assertChunkCount(resp, r -> 8 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize);
         }
     }
 
diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/CrossClusterEsqlRCS1MissingIndicesIT.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/CrossClusterEsqlRCS1MissingIndicesIT.java
index 8d8629db96fc6..23f33b2351c21 100644
--- a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/CrossClusterEsqlRCS1MissingIndicesIT.java
+++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/CrossClusterEsqlRCS1MissingIndicesIT.java
@@ -78,6 +78,7 @@ record ExpectedCluster(String clusterAlias, String indexExpression, String statu
     void assertExpectedClustersForMissingIndicesTests(Map responseMap, List expected) {
         Map clusters = (Map) responseMap.get("_clusters");
         assertThat((int) responseMap.get("took"), greaterThan(0));
+        assertThat((boolean) responseMap.get("is_partial"), is(false));
 
         Map detailsMap = (Map) clusters.get("details");
         assertThat(detailsMap.size(), is(expected.size()));
diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityEsqlIT.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityEsqlIT.java
index 41f2eab6a00e8..42d03838ed8d6 100644
--- a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityEsqlIT.java
+++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityEsqlIT.java
@@ -18,6 +18,7 @@
 import org.elasticsearch.common.UUIDs;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.CheckedConsumer;
+import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Strings;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.test.cluster.ElasticsearchCluster;
@@ -330,6 +331,19 @@ public void populateData() throws Exception {
         assertOK(adminClient().performRequest(putUserRequest));
     }
 
+    private static String populateOtherUser() throws IOException {
+        String otherUser = REMOTE_SEARCH_USER + "_other";
+
+        final var putUserRequest = new Request("PUT", "/_security/user/" + otherUser);
+        putUserRequest.setJsonEntity("""
+            {
+              "password": "x-pack-test-password",
+              "roles" : ["remote_search"]
+            }""");
+        assertOK(adminClient().performRequest(putUserRequest));
+        return otherUser;
+    }
+
     @After
     public void wipeData() throws Exception {
         CheckedConsumer wipe = client -> {
@@ -1198,7 +1212,116 @@ public void testSearchesAgainstNonMatchingIndices() throws Exception {
         }
     }
 
+    public void testCrossClusterAsyncQuery() throws Exception {
+        assumeTrue("delay() is only available in snapshot builds", Build.current().isSnapshot());
+        configureRemoteCluster();
+        populateData();
+        String otherUser = populateOtherUser();
+
+        // Adding a delay there so that the async query is not completed before we check the status
+        Request request = esqlRequestAsync("""
+            FROM employees, *:employees
+            | SORT emp_id ASC
+            | LIMIT 10
+            | WHERE delay(10ms)
+            | KEEP emp_id, department""");
+        Response response = performRequestWithRemoteSearchUser(request);
+        assertOK(response);
+        Map responseAsMap = entityAsMap(response);
+        assumeTrue("Query finished too fast, can not test", (boolean) responseAsMap.get("is_running"));
+
+        String asyncId = (String) responseAsMap.get("id");
+        response = performRequestWithRemoteSearchUser(esqlAsyncGetRequest(asyncId));
+        assertOK(response);
+        responseAsMap = entityAsMap(response);
+        assertThat(responseAsMap.get("is_running"), equalTo(true));
+
+        // Other user can't see the async query
+        ResponseException error = expectThrows(
+            ResponseException.class,
+            () -> performRequestWithUser(esqlAsyncGetRequest(asyncId), otherUser)
+        );
+        assertThat(error.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+        assertThat(error.getMessage(), containsString("resource_not_found_exception"));
+
+        // Clean up
+        response = performRequestWithRemoteSearchUser(esqlAsyncDeleteRequest(asyncId));
+        assertOK(response);
+    }
+
+    public void testCrossClusterAsyncQueryStop() throws Exception {
+        assumeTrue("delay() is only available in snapshot builds", Build.current().isSnapshot());
+        configureRemoteCluster();
+        populateData();
+        String otherUser = populateOtherUser();
+
+        // query remote cluster only
+        Request request = esqlRequestAsync("""
+            FROM employees, *:employees
+            | SORT emp_id ASC
+            | LIMIT 10
+            | WHERE delay(10ms)
+            | KEEP emp_id, department""");
+        Response response = performRequestWithRemoteSearchUser(request);
+        assertOK(response);
+        Map responseAsMap = entityAsMap(response);
+        assertThat(responseAsMap.get("is_running"), equalTo(true));
+        String asyncId = (String) responseAsMap.get("id");
+
+        response = performRequestWithRemoteSearchUser(esqlAsyncGetRequest(asyncId));
+        assertOK(response);
+        responseAsMap = entityAsMap(response);
+        assertThat(responseAsMap.get("is_running"), equalTo(true));
+
+        // Other user can't see the async query
+        ResponseException error = expectThrows(
+            ResponseException.class,
+            () -> performRequestWithUser(esqlAsyncStopRequest(asyncId), otherUser)
+        );
+        assertThat(error.getResponse().getStatusLine().getStatusCode(), equalTo(404));
+        assertThat(error.getMessage(), containsString("resource_not_found_exception"));
+
+        response = performRequestWithRemoteSearchUser(esqlAsyncStopRequest(asyncId));
+        assertOK(response);
+        responseAsMap = entityAsMap(response);
+        assertThat(responseAsMap.get("is_running"), equalTo(false));
+
+        // Clean up
+        response = performRequestWithRemoteSearchUser(esqlAsyncDeleteRequest(asyncId));
+        assertOK(response);
+    }
+
     protected Request esqlRequest(String command) throws IOException {
+        XContentBuilder body = getBody(command, null);
+        Request request = new Request("POST", "_query");
+        request.setJsonEntity(org.elasticsearch.common.Strings.toString(body));
+        return request;
+    }
+
+    protected Request esqlRequestAsync(String command) throws IOException {
+        XContentBuilder body = getBody(command, Map.of("wait_for_completion_timeout", "1ms"));
+        Request request = new Request("POST", "_query/async");
+        request.setJsonEntity(org.elasticsearch.common.Strings.toString(body));
+        return request;
+    }
+
+    protected Request esqlAsyncGetRequest(String asyncID) {
+        Request request = new Request("GET", "_query/async/" + asyncID);
+        request.addParameter("wait_for_completion_timeout", "1ms");
+        return request;
+    }
+
+    protected Request esqlAsyncStopRequest(String asyncID) {
+        Request request = new Request("POST", "_query/async/" + asyncID + "/stop");
+        return request;
+    }
+
+    protected Request esqlAsyncDeleteRequest(String asyncID) {
+        Request request = new Request("DELETE", "_query/async/" + asyncID);
+        return request;
+    }
+
+    private static XContentBuilder getBody(String command, @Nullable Map extraParams) throws IOException {
         XContentBuilder body = JsonXContent.contentBuilder();
         body.startObject();
         body.field("query", command);
@@ -1224,10 +1347,17 @@ protected Request esqlRequest(String command) throws IOException {
                 body.endObject();
             }
         }
+        if (extraParams != null) {
+            extraParams.forEach((name, value) -> {
+                try {
+                    body.field(name, value);
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            });
+        }
         body.endObject();
-        Request request = new Request("POST", "_query");
-        request.setJsonEntity(org.elasticsearch.common.Strings.toString(body));
-        return request;
+        return body;
     }
 
     private Response performRequestWithRemoteSearchUser(final Request request) throws IOException {
@@ -1237,6 +1367,11 @@ private Response performRequestWithRemoteSearchUser(final Request request) throw
         return client().performRequest(request);
     }
 
+    private Response performRequestWithUser(final Request request, String user) throws IOException {
+        request.setOptions(RequestOptions.DEFAULT.toBuilder().addHeader("Authorization", headerFromRandomAuthMethod(user, PASS)));
+        return client().performRequest(request);
+    }
+
     private Response performRequestWithRemoteSearchUserViaAPIKey(Request request, String encodedApiKey) throws IOException {
         request.setOptions(RequestOptions.DEFAULT.toBuilder().addHeader("Authorization", "ApiKey " + encodedApiKey));
         return client().performRequest(request);
diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java
index 62687e42b0912..274587318d555 100644
--- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java
+++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java
@@ -568,6 +568,7 @@ public class Constants {
         "indices:data/read/eql/async/get",
         "indices:data/read/esql",
         "indices:data/read/esql/async/get",
+        "indices:data/read/esql/async/stop",
         "indices:data/read/esql/resolve_fields",
         "indices:data/read/esql/search_shards",
         "indices:data/read/explain",
diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java
index bdd1384bd9d22..1623ff091a222 100644
--- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java
+++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java
@@ -1029,6 +1029,7 @@ private static boolean isAsyncRelatedAction(String action) {
             || action.equals(TransportDeleteAsyncResultAction.TYPE.name())
             || action.equals(EqlAsyncActionNames.EQL_ASYNC_GET_RESULT_ACTION_NAME)
             || action.equals(EsqlAsyncActionNames.ESQL_ASYNC_GET_RESULT_ACTION_NAME)
+            || action.equals(EsqlAsyncActionNames.ESQL_ASYNC_STOP_ACTION_NAME)
             || action.equals(SqlAsyncActionNames.SQL_ASYNC_GET_RESULT_ACTION_NAME);
     }
 
diff --git a/x-pack/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java
index 32ca1cb7221de..80f2959fe55cb 100644
--- a/x-pack/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java
+++ b/x-pack/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java
@@ -26,7 +26,6 @@
 import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.rest.action.search.RestSearchAction;
-import org.elasticsearch.test.MapMatcher;
 import org.elasticsearch.test.StreamsUtils;
 import org.elasticsearch.test.rest.ESRestTestCase;
 import org.elasticsearch.test.rest.RestTestLegacyFeatures;
@@ -53,8 +52,6 @@
 import java.util.stream.Collectors;
 
 import static org.elasticsearch.core.TimeValue.timeValueSeconds;
-import static org.elasticsearch.test.MapMatcher.assertMap;
-import static org.elasticsearch.test.MapMatcher.matchesMap;
 import static org.elasticsearch.upgrades.FullClusterRestartIT.assertNumHits;
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsString;
@@ -1064,16 +1061,10 @@ public void testDisableFieldNameField() throws IOException {
             // {"columns":[{"name":"dv","type":"keyword"},{"name":"no_dv","type":"keyword"}],"values":[["test",null]]}
             try {
                 Map result = entityAsMap(client().performRequest(esql));
-                MapMatcher mapMatcher = matchesMap();
-                if (result.get("took") != null) {
-                    mapMatcher = mapMatcher.entry("took", ((Integer) result.get("took")).intValue());
-                }
-                assertMap(
+                assertResultMap(
                     result,
-                    mapMatcher.entry(
-                        "columns",
-                        List.of(Map.of("name", "dv", "type", "keyword"), Map.of("name", "no_dv", "type", "keyword"))
-                    ).entry("values", List.of(List.of("test", "test")))
+                    List.of(Map.of("name", "dv", "type", "keyword"), Map.of("name", "no_dv", "type", "keyword")),
+                    List.of(List.of("test", "test"))
                 );
             } catch (ResponseException e) {
                 logger.error(