From 90de2846177db56603589ef23d0d68e0c326d748 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 25 Mar 2025 13:15:48 -0400 Subject: [PATCH 01/13] ESQL: Add `found_documents` to task and profile Adds the `found_documents` and `values_loaded` fields to the tasks and profile for ESQL: ``` "drivers" : [ { "description" : "data", "cluster_name" : "runTask", "node_name" : "runTask-0", "start_millis" : 1742923173077, "stop_millis" : 1742923173087, "took_nanos" : 9557014, "cpu_nanos" : 9091340, "documents_found" : 5, <---- THESE "values_loaded" : 15, <---- THESE "iterations" : 6, ... ``` --- .../org/elasticsearch/TransportVersions.java | 1 + .../compute/data/CompositeBlock.java | 6 ++- .../compute/lucene/LuceneOperator.java | 5 +++ .../lucene/ValuesSourceReaderOperator.java | 41 +++++++++++++++---- .../compute/operator/DriverProfile.java | 2 + .../compute/operator/DriverStatus.java | 20 +++++++++ .../compute/operator/Operator.java | 18 +++++++- .../compute/operator/OperatorStatus.java | 22 ++++++++++ ...ValuesSourceReaderOperatorStatusTests.java | 10 +++-- .../compute/operator/DriverProfileTests.java | 2 + .../compute/operator/DriverStatusTests.java | 2 + .../xpack/esql/action/EsqlActionTaskIT.java | 14 +++++++ .../xpack/esql/action/EsqlCapabilities.java | 6 +++ .../esql/action/EsqlQueryResponseTests.java | 2 + .../rest-api-spec/test/esql/120_profile.yml | 30 ++++++++++++-- 15 files changed, 165 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 0005cf11f7265..289abd55b80b3 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -200,6 +200,7 @@ static TransportVersion def(int id) { public static final TransportVersion INDEXING_STATS_INCLUDES_RECENT_WRITE_LOAD = def(9_034_0_00); public static final TransportVersion ESQL_AGGREGATE_METRIC_DOUBLE_LITERAL = def(9_035_0_00); public static final TransportVersion INDEX_METADATA_INCLUDES_RECENT_WRITE_LOAD = def(9_036_0_00); + public static final TransportVersion ESQL_VALUES_LOADED = def(9_037_0_00); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/CompositeBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/CompositeBlock.java index 8d0b8a27f5833..10b91c578a6d6 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/CompositeBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/CompositeBlock.java @@ -83,7 +83,11 @@ public int getPositionCount() { @Override public int getTotalValueCount() { - throw new UnsupportedOperationException("Composite block"); + int totalValueCount = 0; + for (Block b : blocks) { + totalValueCount += b.getTotalValueCount(); + } + return totalValueCount; } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 7547e2da3e184..4bf23d59055b8 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -417,6 +417,11 @@ public long rowsEmitted() { return rowsEmitted; } + @Override + public long documentsFound() { + return rowsEmitted; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java index 841789e8ada3c..1eb50e83d8c68 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -47,6 +47,8 @@ import java.util.function.IntFunction; import java.util.function.Supplier; +import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; + /** * Operator that extracts doc_values from a Lucene index out of pages that have been produced by {@link LuceneSourceOperator} * and outputs them to a new column. @@ -113,6 +115,7 @@ public record ShardContext(IndexReader reader, Supplier newSourceL private final BlockFactory blockFactory; private final Map readersBuilt = new TreeMap<>(); + private long valuesLoaded; int lastShard = -1; int lastSegment = -1; @@ -165,6 +168,9 @@ public int get(int i) { } } success = true; + for (Block b : blocks) { + valuesLoaded += b.getTotalValueCount(); + } return page.appendBlocks(blocks); } catch (IOException e) { throw new UncheckedIOException(e); @@ -547,7 +553,7 @@ public String toString() { @Override protected Status status(long processNanos, int pagesProcessed, long rowsReceived, long rowsEmitted) { - return new Status(new TreeMap<>(readersBuilt), processNanos, pagesProcessed, rowsReceived, rowsEmitted); + return new Status(new TreeMap<>(readersBuilt), processNanos, pagesProcessed, rowsReceived, rowsEmitted, valuesLoaded); } public static class Status extends AbstractPageMappingOperator.Status { @@ -558,21 +564,38 @@ public static class Status extends AbstractPageMappingOperator.Status { ); private final Map readersBuilt; - - Status(Map readersBuilt, long processNanos, int pagesProcessed, long rowsReceived, long rowsEmitted) { + private final long valuesLoaded; + + Status( + Map readersBuilt, + long processNanos, + int pagesProcessed, + long rowsReceived, + long rowsEmitted, + long valuesLoaded + ) { super(processNanos, pagesProcessed, rowsReceived, rowsEmitted); this.readersBuilt = readersBuilt; + this.valuesLoaded = valuesLoaded; } Status(StreamInput in) throws IOException { super(in); readersBuilt = in.readOrderedMap(StreamInput::readString, StreamInput::readVInt); + if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + valuesLoaded = in.readVLong(); + } else { + valuesLoaded = 0; + } } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeMap(readersBuilt, StreamOutput::writeVInt); + if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + out.writeVLong(valuesLoaded); + } } @Override @@ -584,6 +607,11 @@ public Map readersBuilt() { return readersBuilt; } + @Override + public long valuesLoaded() { + return valuesLoaded; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); @@ -592,6 +620,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(e.getKey(), e.getValue()); } builder.endObject(); + builder.field("values_loaded", valuesLoaded); innerToXContent(builder); return builder.endObject(); } @@ -600,12 +629,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public boolean equals(Object o) { if (super.equals(o) == false) return false; Status status = (Status) o; - return readersBuilt.equals(status.readersBuilt); + return readersBuilt.equals(status.readersBuilt) && valuesLoaded == status.valuesLoaded; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), readersBuilt); + return Objects.hash(super.hashCode(), readersBuilt, valuesLoaded); } @Override @@ -704,6 +733,4 @@ public BlockLoader.AggregateMetricDoubleBuilder aggregateMetricDoubleBuilder(int return factory.newAggregateMetricDoubleBlockBuilder(count); } } - - // TODO tests that mix source loaded fields and doc values in the same block } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverProfile.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverProfile.java index 1abe3087b4d93..1e75b913fb99d 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverProfile.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverProfile.java @@ -104,6 +104,8 @@ public Iterator toXContentChunked(ToXContent.Params params if (b.humanReadable()) { b.field("cpu_time", TimeValue.timeValueNanos(cpuNanos)); } + b.field("documents_found", operators.stream().mapToLong(OperatorStatus::documentsFound).sum()); + b.field("values_loaded", operators.stream().mapToLong(OperatorStatus::valuesLoaded).sum()); b.field("iterations", iterations); return b; }), diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java index d919cc1c18697..c0351f27ecb2c 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java @@ -124,6 +124,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (builder.humanReadable()) { builder.field("cpu_time", TimeValue.timeValueNanos(cpuNanos)); } + builder.field("documents_found", documentsFound()); + builder.field("values_loaded", valuesLoaded()); builder.field("iterations", iterations); builder.field("status", status, params); builder.startArray("completed_operators"); @@ -145,6 +147,24 @@ public String toString() { return Strings.toString(this); } + /** + * The number of documents found by this driver. + */ + public long documentsFound() { + long documentsFound = completedOperators.stream().mapToLong(OperatorStatus::documentsFound).sum(); + documentsFound += activeOperators.stream().mapToLong(OperatorStatus::documentsFound).sum(); + return documentsFound; + } + + /** + * The number of values loaded by this operator. + */ + public long valuesLoaded() { + long valuesLoaded = completedOperators.stream().mapToLong(OperatorStatus::valuesLoaded).sum(); + valuesLoaded += activeOperators.stream().mapToLong(OperatorStatus::valuesLoaded).sum(); + return valuesLoaded; + } + public enum Status implements Writeable, ToXContentFragment { QUEUED, STARTING, diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java index 46e85bec693e8..0a382a40c809c 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java @@ -105,5 +105,21 @@ interface OperatorFactory extends Describable { /** * Status of an {@link Operator} to be returned by the tasks API. */ - interface Status extends ToXContentObject, VersionedNamedWriteable {} + interface Status extends ToXContentObject, VersionedNamedWriteable { + /** + * The number of documents found by this operator. Most operators + * don't find documents and will return {@code 0} here. + */ + default long documentsFound() { + return 0; + } + + /** + * The number of values loaded by this operator. Most operators + * don't load values and will return {@code 0} here. + */ + default long valuesLoaded() { + return 0; + } + } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OperatorStatus.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OperatorStatus.java index 6d83338faf7c5..14d3da12ad5d4 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OperatorStatus.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OperatorStatus.java @@ -49,4 +49,26 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public String toString() { return Strings.toString(this); } + + /** + * The number of documents found by this operator. Most operators + * don't find documents and will return {@code 0} here. + */ + public long documentsFound() { + if (status == null) { + return 0; + } + return status.documentsFound(); + } + + /** + * The number of values loaded by this operator. Most operators + * don't load values and will return {@code 0} here. + */ + public long valuesLoaded() { + if (status == null) { + return 0; + } + return status.valuesLoaded(); + } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java index 4303137f74bb3..af1463b88c62c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java @@ -20,7 +20,7 @@ public class ValuesSourceReaderOperatorStatusTests extends AbstractWireSerializingTestCase { public static ValuesSourceReaderOperator.Status simple() { - return new ValuesSourceReaderOperator.Status(Map.of("ReaderType", 3), 1022323, 123, 111, 222); + return new ValuesSourceReaderOperator.Status(Map.of("ReaderType", 3), 1022323, 123, 111, 222, 1000); } public static String simpleToJson() { @@ -29,6 +29,7 @@ public static String simpleToJson() { "readers_built" : { "ReaderType" : 3 }, + "values_loaded" : 1000, "process_nanos" : 1022323, "process_time" : "1ms", "pages_processed" : 123, @@ -53,6 +54,7 @@ public ValuesSourceReaderOperator.Status createTestInstance() { randomNonNegativeLong(), randomNonNegativeInt(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong() ); } @@ -73,14 +75,16 @@ protected ValuesSourceReaderOperator.Status mutateInstance(ValuesSourceReaderOpe int pagesProcessed = instance.pagesProcessed(); long rowsReceived = instance.rowsReceived(); long rowsEmitted = instance.rowsEmitted(); - switch (between(0, 4)) { + long valuesLoaded = instance.valuesLoaded(); + switch (between(0, 5)) { case 0 -> readersBuilt = randomValueOtherThan(readersBuilt, this::randomReadersBuilt); case 1 -> processNanos = randomValueOtherThan(processNanos, ESTestCase::randomNonNegativeLong); case 2 -> pagesProcessed = randomValueOtherThan(pagesProcessed, ESTestCase::randomNonNegativeInt); case 3 -> rowsReceived = randomValueOtherThan(rowsReceived, ESTestCase::randomNonNegativeLong); case 4 -> rowsEmitted = randomValueOtherThan(rowsEmitted, ESTestCase::randomNonNegativeLong); + case 5 -> valuesLoaded = randomValueOtherThan(valuesLoaded, ESTestCase::randomNonNegativeLong); default -> throw new UnsupportedOperationException(); } - return new ValuesSourceReaderOperator.Status(readersBuilt, processNanos, pagesProcessed, rowsReceived, rowsEmitted); + return new ValuesSourceReaderOperator.Status(readersBuilt, processNanos, pagesProcessed, rowsReceived, rowsEmitted, valuesLoaded); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverProfileTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverProfileTests.java index 08087f249c19f..c8f8094f69c27 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverProfileTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverProfileTests.java @@ -58,6 +58,8 @@ public void testToXContent() { "took_time" : "10micros", "cpu_nanos" : 10000, "cpu_time" : "10micros", + "documents_found" : 222, + "values_loaded" : 1000, "iterations" : 12, "operators" : [ { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java index 3915c9d6a37b8..df3583d0c99bd 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java @@ -59,6 +59,8 @@ public void testToXContent() { "last_updated" : "1973-11-29T09:27:23.214Z", "cpu_nanos" : 123213, "cpu_time" : "123.2micros", + "documents_found" : 222, + "values_loaded" : 1000, "iterations" : 55, "status" : "running", "completed_operators" : [ 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 0294df0bfe5a4..5ba3648a02f32 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 @@ -135,6 +135,7 @@ public void testTaskContents() throws Exception { matchesMap().entry("pause_me:column_at_a_time:ScriptLongs", greaterThanOrEqualTo(1)) ); assertThat(oStatus.pagesProcessed(), greaterThanOrEqualTo(1)); + assertThat(oStatus.valuesLoaded(), greaterThanOrEqualTo(1L)); valuesSourceReaders++; continue; } @@ -181,6 +182,19 @@ public void testTaskContents() throws Exception { \\_ProjectOperator[projection = [0]] \\_LimitOperator[limit = 1000] \\_OutputOperator[columns = [sum(pause_me)]]""")); + + for (TaskInfo task : dataTasks(foundTasks)) { + assertThat(((DriverStatus) task.status()).documentsFound(), greaterThan(0L)); + assertThat(((DriverStatus) task.status()).valuesLoaded(), greaterThan(0L)); + } + for (TaskInfo task : nodeReduceTasks(foundTasks)) { + assertThat(((DriverStatus) task.status()).documentsFound(), equalTo(0L)); + assertThat(((DriverStatus) task.status()).valuesLoaded(), equalTo(0L)); + } + for (TaskInfo task : coordinatorTasks(foundTasks)) { + assertThat(((DriverStatus) task.status()).documentsFound(), equalTo(0L)); + assertThat(((DriverStatus) task.status()).valuesLoaded(), equalTo(0L)); + } } finally { scriptPermits.release(numberOfDocs()); try (EsqlQueryResponse esqlResponse = response.get()) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 6e3d1ba37a05f..abf7dccb152b3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -919,6 +919,12 @@ public enum Cap { */ METRICS_COMMAND(Build.current().isSnapshot()), + /** + * Are the {@code documents_found} and {@code values_loaded} fields available + * in the profile? + */ + PROFILE_DOCUMENTS_FOUND, + /** * Index component selector syntax (my-data-stream-name::failures) */ 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 95b0abc31062f..a5d0f2b7b2c99 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 @@ -763,6 +763,8 @@ public void testProfileXContent() { "stop_millis" : 1723489819929, "took_nanos" : 20021, "cpu_nanos" : 20000, + "documents_found" : 0, + "values_loaded" : 0, "iterations" : 12, "operators" : [ { diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml index 17034de677b8d..c15e05b1ac551 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml @@ -1,9 +1,5 @@ --- setup: - - requires: - cluster_features: ["gte_v8.12.0"] - reason: "profile option added in 8.12" - test_runner_features: warnings - do: indices.create: index: test @@ -140,3 +136,29 @@ avg 8.14 or after: - gte: {profile.drivers.1.took_nanos: 0} - gte: {profile.drivers.1.cpu_nanos: 0} # It's hard to assert much about these because they don't come back in any particular order. + +--- +documents found: + - requires: + test_runner_features: [capabilities, contains] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [profile_documents_found] + reason: "checks for documents_found and values_loaded" + + - do: + esql.query: + body: + query: 'FROM test | LIMIT 1' + profile: true + + - length: {profile.drivers: 3} + - match: {profile.drivers.0.operators.0.operator: /ExchangeSourceOperator|LuceneSourceOperator.+/} + - gte: {profile.drivers.0.documents_found: 0} + - gte: {profile.drivers.0.values_loaded: 0} + - gte: {profile.drivers.1.documents_found: 0} + - gte: {profile.drivers.1.values_loaded: 0} + - gte: {profile.drivers.2.documents_found: 0} + - gte: {profile.drivers.2.values_loaded: 0} From 4875c5ecf3fa61295efa31cd00f0a41019ecab37 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 25 Mar 2025 13:29:51 -0400 Subject: [PATCH 02/13] More composite tests --- .../compute/data/CompositeBlockTests.java | 60 ++++++++++++++++--- 1 file changed, 52 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java index ed7eded6eeda4..540adf0f2c1bf 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java @@ -12,6 +12,7 @@ import java.util.Arrays; import java.util.List; +import java.util.function.Supplier; import static org.hamcrest.Matchers.equalTo; @@ -21,19 +22,29 @@ public class CompositeBlockTests extends ComputeTestCase { .filter(e -> e != ElementType.COMPOSITE && e != ElementType.UNKNOWN && e != ElementType.DOC) .toList(); - public static CompositeBlock randomCompositeBlock(BlockFactory blockFactory, int numBlocks, int positionCount) { + public static CompositeBlock randomCompositeBlock( + BlockFactory blockFactory, + Supplier randomElementType, + boolean nullAllowed, + int numBlocks, + int positionCount, + int minValuesPerPosition, + int maxValuesPerPosition, + int minDupsPerPosition, + int maxDupsPerPosition + ) { Block[] blocks = new Block[numBlocks]; for (int b = 0; b < numBlocks; b++) { - ElementType elementType = randomFrom(supportedSubElementTypes); + ElementType elementType = randomElementType.get(); blocks[b] = RandomBlock.randomBlock( blockFactory, elementType, positionCount, - elementType == ElementType.NULL || randomBoolean(), - 0, - between(1, 2), - 0, - between(1, 2) + nullAllowed && (elementType == ElementType.NULL || randomBoolean()), + minValuesPerPosition, + maxValuesPerPosition, + minDupsPerPosition, + maxDupsPerPosition ).block(); } return new CompositeBlock(blocks); @@ -43,7 +54,19 @@ public void testFilter() { final BlockFactory blockFactory = blockFactory(); int numBlocks = randomIntBetween(1, 1000); int positionCount = randomIntBetween(1, 1000); - try (CompositeBlock origComposite = randomCompositeBlock(blockFactory, numBlocks, positionCount)) { + try ( + CompositeBlock origComposite = randomCompositeBlock( + blockFactory, + () -> randomFrom(supportedSubElementTypes), + true, + numBlocks, + positionCount, + 0, + between(1, 2), + 0, + between(1, 2) + ) + ) { int[] selected = new int[randomIntBetween(0, positionCount * 3)]; for (int i = 0; i < selected.length; i++) { selected[i] = randomIntBetween(0, positionCount - 1); @@ -59,4 +82,25 @@ public void testFilter() { } } } + + public void testTotalValueCount() { + final BlockFactory blockFactory = blockFactory(); + int numBlocks = randomIntBetween(1, 1000); + int positionCount = randomIntBetween(1, 1000); + try ( + CompositeBlock composite = randomCompositeBlock( + blockFactory, + () -> randomValueOtherThan(ElementType.NULL, () -> randomFrom(supportedSubElementTypes)), + false, + numBlocks, + positionCount, + 1, + 1, + 0, + 0 + ) + ) { + assertThat(composite.getTotalValueCount(), equalTo(numBlocks * positionCount)); + } + } } From fb36e7e84c99dba1cc5fe401719ddcf297763017 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 25 Mar 2025 16:22:56 -0400 Subject: [PATCH 03/13] Plumbing --- .../operator/DriverCompletionInfo.java | 113 ++++++++++++++++++ .../xpack/esql/action/EsqlQueryResponse.java | 45 ++++++- .../xpack/esql/action/EsqlQueryTask.java | 3 +- .../esql/plugin/ClusterComputeHandler.java | 15 +-- .../xpack/esql/plugin/ComputeListener.java | 17 ++- .../xpack/esql/plugin/ComputeResponse.java | 39 +++--- .../xpack/esql/plugin/ComputeService.java | 41 ++++--- .../esql/plugin/DataNodeComputeHandler.java | 16 +-- .../esql/plugin/DataNodeComputeResponse.java | 43 ++++--- .../esql/plugin/DataNodeRequestSender.java | 41 ++++--- .../esql/plugin/TransportEsqlQueryAction.java | 19 ++- .../xpack/esql/session/EsqlCCSUtils.java | 3 +- .../xpack/esql/session/EsqlSession.java | 15 ++- .../xpack/esql/session/Result.java | 8 +- .../esql/plugin/ComputeListenerTests.java | 5 +- 15 files changed, 314 insertions(+), 109 deletions(-) create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java new file mode 100644 index 0000000000000..56bb2a856172a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java @@ -0,0 +1,113 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Information returned when one of more {@link Driver}s is completed. + * @param documentsFound The number of documents found by all lucene queries performed by these drivers. + * @param valuesLoaded The number of values loaded from lucene for all drivers. + * @param collectedProfiles {@link DriverProfile}s from each driver. These are fairly cheap to build but + * not free so this will be empty if the {@code profile} option was not set in + * the request. + */ +public record DriverCompletionInfo(long documentsFound, long valuesLoaded, List collectedProfiles) implements Writeable { + /** + * Completion info we use when we didn't properly complete any drivers. + * Usually this is returned with an error, but it's also used when receiving + * responses from very old nodes. + */ + public static final DriverCompletionInfo EMPTY = new DriverCompletionInfo(0, 0, List.of()); + + /** + * Build a {@link DriverCompletionInfo} for many drivers including their profile output. + */ + public static DriverCompletionInfo includingProfiles(List drivers) { + long documentsFound = 0; + long valuesLoaded = 0; + List collectedProfiles = new ArrayList<>(drivers.size()); + for (Driver d : drivers) { + DriverProfile p = d.profile(); + for (OperatorStatus o : p.operators()) { + documentsFound += o.documentsFound(); + valuesLoaded += o.valuesLoaded(); + } + collectedProfiles.add(p); + } + return new DriverCompletionInfo(documentsFound, valuesLoaded, collectedProfiles); + } + + /** + * Build a {@link DriverCompletionInfo} for many drivers excluding their profile output. + */ + public static DriverCompletionInfo excludingProfiles(List drivers) { + long documentsFound = 0; + long valuesLoaded = 0; + for (Driver d : drivers) { + DriverStatus s = d.status(); + assert s.status() == DriverStatus.Status.DONE; + for (OperatorStatus o : s.completedOperators()) { + documentsFound += o.documentsFound(); + valuesLoaded += o.valuesLoaded(); + } + } + return new DriverCompletionInfo(documentsFound, valuesLoaded, List.of()); + } + + public DriverCompletionInfo(StreamInput in) throws IOException { + this(in.readVLong(), in.readVLong(), in.readCollectionAsImmutableList(DriverProfile::readFrom)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(documentsFound); + out.writeVLong(valuesLoaded); + out.writeCollection(collectedProfiles, (o, v) -> v.writeTo(o)); + } + + public static class Accumulator { + private long documentsFound; + private long valuesLoaded; + private final List collectedProfiles = new ArrayList<>(); + + public void accumulate(DriverCompletionInfo info) { + this.documentsFound += info.documentsFound; + this.valuesLoaded += info.valuesLoaded; + this.collectedProfiles.addAll(info.collectedProfiles); + } + + public DriverCompletionInfo finish() { + return new DriverCompletionInfo(documentsFound, valuesLoaded, collectedProfiles); + } + } + + public static class AtomicAccumulator { + private final AtomicLong documentsFound = new AtomicLong(); + private final AtomicLong valuesLoaded = new AtomicLong(); + private final List collectedProfiles = Collections.synchronizedList(new ArrayList<>()); + + public void accumulate(DriverCompletionInfo info) { + this.documentsFound.addAndGet(info.documentsFound); + this.valuesLoaded.addAndGet(info.valuesLoaded); + this.collectedProfiles.addAll(info.collectedProfiles); + } + + public DriverCompletionInfo finish() { + return new DriverCompletionInfo(documentsFound.get(), valuesLoaded.get(), collectedProfiles); + } + } +} 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 5e36eee7364c7..1aa9a0c765b49 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 @@ -35,6 +35,8 @@ import java.util.Objects; import java.util.Optional; +import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; + public class EsqlQueryResponse extends org.elasticsearch.xpack.core.esql.action.EsqlQueryResponse implements ChunkedToXContentObject, @@ -47,6 +49,8 @@ public class EsqlQueryResponse extends org.elasticsearch.xpack.core.esql.action. private final List columns; private final List pages; + private final long documentsFound; + private final long valuesLoaded; private final Profile profile; private final boolean columnar; private final String asyncExecutionId; @@ -58,6 +62,8 @@ public class EsqlQueryResponse extends org.elasticsearch.xpack.core.esql.action. public EsqlQueryResponse( List columns, List pages, + long documentsFound, + long valuesLoaded, @Nullable Profile profile, boolean columnar, @Nullable String asyncExecutionId, @@ -67,6 +73,8 @@ public EsqlQueryResponse( ) { this.columns = columns; this.pages = pages; + this.valuesLoaded = valuesLoaded; + this.documentsFound = documentsFound; this.profile = profile; this.columnar = columnar; this.asyncExecutionId = asyncExecutionId; @@ -78,12 +86,14 @@ public EsqlQueryResponse( public EsqlQueryResponse( List columns, List pages, + long documentsFound, + long valuesLoaded, @Nullable Profile profile, boolean columnar, boolean isAsync, EsqlExecutionInfo executionInfo ) { - this(columns, pages, profile, columnar, null, false, isAsync, executionInfo); + this(columns, pages, documentsFound, valuesLoaded, profile, columnar, null, false, isAsync, executionInfo); } /** @@ -109,6 +119,8 @@ static EsqlQueryResponse deserialize(BlockStreamInput in) throws IOException { } List columns = in.readCollectionAsList(ColumnInfoImpl::new); List pages = in.readCollectionAsList(Page::new); + long documentsFound = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVInt() : 0; + long valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVInt() : 0; if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { profile = in.readOptionalWriteable(Profile::new); } @@ -117,7 +129,18 @@ static EsqlQueryResponse deserialize(BlockStreamInput in) throws IOException { if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) { executionInfo = in.readOptionalWriteable(EsqlExecutionInfo::new); } - return new EsqlQueryResponse(columns, pages, profile, columnar, asyncExecutionId, isRunning, isAsync, executionInfo); + return new EsqlQueryResponse( + columns, + pages, + documentsFound, + valuesLoaded, + profile, + columnar, + asyncExecutionId, + isRunning, + isAsync, + executionInfo + ); } @Override @@ -129,6 +152,10 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeCollection(columns); out.writeCollection(pages); + if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + out.writeVLong(documentsFound); + out.writeVLong(valuesLoaded); + } if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { out.writeOptionalWriteable(profile); } @@ -218,6 +245,12 @@ public Iterator toXContentChunked(ToXContent.Params params tookTime = Collections.emptyIterator(); } + Iterator meta = ChunkedToXContentHelper.chunk((builder, p) -> { + builder.field("documents_found", documentsFound); + builder.field("values_loaded", valuesLoaded); + return builder; + }); + Iterator columnHeadings = dropNullColumns ? Iterators.concat( ResponseXContentUtils.allColumns(columns, "all_columns"), @@ -235,6 +268,7 @@ public Iterator toXContentChunked(ToXContent.Params params ChunkedToXContentHelper.startObject(), asyncPropertiesOrEmpty(), tookTime, + meta, columnHeadings, ChunkedToXContentHelper.array("values", valuesIt), executionInfoRender, @@ -275,6 +309,8 @@ public boolean equals(Object o) { && Objects.equals(isRunning, that.isRunning) && columnar == that.columnar && Iterators.equals(values(), that.values(), (row1, row2) -> Iterators.equals(row1, row2, Objects::equals)) + && documentsFound == that.documentsFound + && valuesLoaded == that.valuesLoaded && Objects.equals(profile, that.profile) && Objects.equals(executionInfo, that.executionInfo); } @@ -285,8 +321,11 @@ public int hashCode() { asyncExecutionId, isRunning, columns, - Iterators.hashCode(values(), row -> Iterators.hashCode(row, Objects::hashCode)), columnar, + Iterators.hashCode(values(), row -> Iterators.hashCode(row, Objects::hashCode)), + documentsFound, + valuesLoaded, + profile, executionInfo ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryTask.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryTask.java index f896a25317102..4d7565a5d7863 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryTask.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryTask.java @@ -44,6 +44,7 @@ public EsqlExecutionInfo executionInfo() { @Override public EsqlQueryResponse getCurrentResult() { - return new EsqlQueryResponse(List.of(), List.of(), null, false, getExecutionId().getEncoded(), true, true, executionInfo); + // TODO it'd be nice to have the number of documents we've read from completed drivers here + return new EsqlQueryResponse(List.of(), List.of(), 0, 0, null, false, getExecutionId().getEncoded(), true, true, executionInfo); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java index 5ad81177a6a44..72d1c2fbc967c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.exchange.ExchangeService; import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; @@ -75,7 +76,7 @@ void startComputeOnRemoteCluster( RemoteCluster cluster, Runnable cancelQueryOnFailure, EsqlExecutionInfo executionInfo, - ActionListener> listener + ActionListener listener ) { var queryPragmas = configuration.pragmas(); listener = ActionListener.runBefore(listener, exchangeSource.addEmptySink()::close); @@ -87,10 +88,10 @@ void startComputeOnRemoteCluster( final boolean receivedResults = finalResponse.get() != null || pagesFetched.get(); if (receivedResults == false && EsqlCCSUtils.shouldIgnoreRuntimeError(executionInfo, clusterAlias, e)) { EsqlCCSUtils.markClusterWithFinalStateAndNoShards(executionInfo, clusterAlias, EsqlExecutionInfo.Cluster.Status.SKIPPED, e); - l.onResponse(List.of()); + l.onResponse(DriverCompletionInfo.EMPTY); } else if (configuration.allowPartialResults()) { EsqlCCSUtils.markClusterWithFinalStateAndNoShards(executionInfo, clusterAlias, EsqlExecutionInfo.Cluster.Status.PARTIAL, e); - l.onResponse(List.of()); + l.onResponse(DriverCompletionInfo.EMPTY); } else { l.onFailure(e); } @@ -118,9 +119,9 @@ void startComputeOnRemoteCluster( onGroupFailure = computeService.cancelQueryOnFailure(groupTask); l = ActionListener.runAfter(l, () -> transportService.getTaskManager().unregister(groupTask)); } - try (var computeListener = new ComputeListener(transportService.getThreadPool(), onGroupFailure, l.map(profiles -> { + try (var computeListener = new ComputeListener(transportService.getThreadPool(), onGroupFailure, l.map(completionInfo -> { updateExecutionInfo(executionInfo, clusterAlias, finalResponse.get()); - return profiles; + return completionInfo; }))) { var remoteSink = exchangeService.newRemoteSink(groupTask, childSessionId, transportService, cluster.connection); exchangeSource.addRemoteSink( @@ -134,7 +135,7 @@ void startComputeOnRemoteCluster( var clusterRequest = new ClusterComputeRequest(clusterAlias, childSessionId, configuration, remotePlan); final ActionListener clusterListener = computeListener.acquireCompute().map(r -> { finalResponse.set(r); - return r.getProfiles(); + return r.getCompletionInfo(); }); transportService.sendChildRequest( cluster.connection, @@ -287,7 +288,7 @@ void runComputeOnRemoteCluster( cancelQueryOnFailure, computeListener.acquireCompute().map(r -> { finalResponse.set(r); - return r.getProfiles(); + return r.getCompletionInfo(); }) ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java index c8b8e84fd2478..9e0c8a6853282 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.compute.EsqlRefCountingListener; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.ResponseHeadersCollector; import org.elasticsearch.core.Releasable; @@ -18,6 +19,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; /** * A variant of {@link RefCountingListener} with the following differences: @@ -27,19 +29,18 @@ * 4. Collects failures and returns the most appropriate exception to the caller. */ final class ComputeListener implements Releasable { + private final DriverCompletionInfo.AtomicAccumulator completionInfoAccumulator = new DriverCompletionInfo.AtomicAccumulator(); private final EsqlRefCountingListener refs; - private final List collectedProfiles; private final ResponseHeadersCollector responseHeaders; private final Runnable runOnFailure; - ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener> delegate) { + ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener delegate) { this.runOnFailure = runOnFailure; this.responseHeaders = new ResponseHeadersCollector(threadPool.getThreadContext()); - this.collectedProfiles = Collections.synchronizedList(new ArrayList<>()); // listener that executes after all the sub-listeners refs (created via acquireCompute) have completed this.refs = new EsqlRefCountingListener(delegate.delegateFailure((l, ignored) -> { responseHeaders.finish(); - delegate.onResponse(collectedProfiles.stream().toList()); + delegate.onResponse(completionInfoAccumulator.finish()); })); } @@ -60,13 +61,11 @@ ActionListener acquireAvoid() { /** * Acquires a new listener that collects compute result. This listener will also collect warnings emitted during compute */ - ActionListener> acquireCompute() { + ActionListener acquireCompute() { final ActionListener delegate = acquireAvoid(); - return ActionListener.wrap(profiles -> { + return ActionListener.wrap(info -> { responseHeaders.collect(); - if (profiles != null && profiles.isEmpty() == false) { - collectedProfiles.addAll(profiles); - } + completionInfoAccumulator.accumulate(info); delegate.onResponse(null); }, e -> { responseHeaders.collect(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java index 289186ae38e6e..715da6ef0079e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.TimeValue; import org.elasticsearch.transport.TransportResponse; @@ -18,11 +19,13 @@ import java.io.IOException; import java.util.List; +import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; + /** * The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest} */ final class ComputeResponse extends TransportResponse { - private final List profiles; + private final DriverCompletionInfo completionInfo; // for use with ClusterComputeRequests (cross-cluster searches) private final TimeValue took; // overall took time for a specific cluster in a cross-cluster search @@ -32,12 +35,12 @@ final class ComputeResponse extends TransportResponse { public final int failedShards; public final List failures; - ComputeResponse(List profiles) { - this(profiles, null, null, null, null, null, List.of()); + ComputeResponse(DriverCompletionInfo completionInfo) { + this(completionInfo, null, null, null, null, null, List.of()); } ComputeResponse( - List profiles, + DriverCompletionInfo completionInfo, TimeValue took, Integer totalShards, Integer successfulShards, @@ -45,7 +48,7 @@ final class ComputeResponse extends TransportResponse { Integer failedShards, List failures ) { - this.profiles = profiles; + this.completionInfo = completionInfo; this.took = took; this.totalShards = totalShards == null ? 0 : totalShards.intValue(); this.successfulShards = successfulShards == null ? 0 : successfulShards.intValue(); @@ -55,14 +58,16 @@ final class ComputeResponse extends TransportResponse { } ComputeResponse(StreamInput in) throws IOException { - if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { + if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + completionInfo = new DriverCompletionInfo(in); + } else if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { if (in.readBoolean()) { - profiles = in.readCollectionAsImmutableList(DriverProfile::readFrom); + completionInfo = new DriverCompletionInfo(0, 0, in.readCollectionAsImmutableList(DriverProfile::readFrom)); } else { - profiles = null; + completionInfo = DriverCompletionInfo.EMPTY; } } else { - profiles = null; + completionInfo = DriverCompletionInfo.EMPTY; } if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) { this.took = in.readOptionalTimeValue(); @@ -87,13 +92,11 @@ final class ComputeResponse extends TransportResponse { @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { - if (profiles == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeCollection(profiles); - } + if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + completionInfo.writeTo(out); + } else if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { + out.writeBoolean(true); + out.writeCollection(completionInfo.collectedProfiles()); } if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) { out.writeOptionalTimeValue(took); @@ -108,8 +111,8 @@ public void writeTo(StreamOutput out) throws IOException { } } - public List getProfiles() { - return profiles; + public DriverCompletionInfo getCompletionInfo() { + return completionInfo; } public TimeValue getTook() { 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 293be0eb3c2b0..3d5b61b5bf7ce 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 @@ -17,6 +17,7 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.DriverTaskRunner; import org.elasticsearch.compute.operator.exchange.ExchangeService; @@ -208,10 +209,14 @@ public void execute( ); updateShardCountForCoordinatorOnlyQuery(execInfo); try ( - var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { - updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result(physicalPlan.output(), collectedPages, profiles, execInfo); - })) + var computeListener = new ComputeListener( + transportService.getThreadPool(), + cancelQueryOnFailure, + listener.map(completionInfo -> { + updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); + return new Result(physicalPlan.output(), collectedPages, completionInfo, execInfo); + }) + ) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); return; @@ -240,10 +245,16 @@ public void execute( ); listener = ActionListener.runBefore(listener, () -> exchangeService.removeExchangeSourceHandler(sessionId)); exchangeService.addExchangeSourceHandler(sessionId, exchangeSource); - try (var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { - execInfo.markEndQuery(); // TODO: revisit this time recording model as part of INLINESTATS improvements - return new Result(outputAttributes, collectedPages, profiles, execInfo); - }))) { + try ( + var computeListener = new ComputeListener( + transportService.getThreadPool(), + cancelQueryOnFailure, + listener.map(completionInfo -> { + execInfo.markEndQuery(); // TODO: revisit this time recording model as part of INLINESTATS improvements + return new Result(outputAttributes, collectedPages, completionInfo, execInfo); + }) + ) + ) { try (Releasable ignored = exchangeSource.addEmptySink()) { // run compute on the coordinator final AtomicBoolean localClusterWasInterrupted = new AtomicBoolean(); @@ -251,7 +262,7 @@ public void execute( var localListener = new ComputeListener( transportService.getThreadPool(), cancelQueryOnFailure, - computeListener.acquireCompute().delegateFailure((l, profiles) -> { + computeListener.acquireCompute().delegateFailure((l, completionInfo) -> { if (execInfo.clusterInfo.containsKey(LOCAL_CLUSTER)) { execInfo.swapCluster(LOCAL_CLUSTER, (k, v) -> { var tookTime = TimeValue.timeValueNanos(System.nanoTime() - execInfo.getRelativeStartNanos()); @@ -266,7 +277,7 @@ public void execute( return builder.build(); }); } - l.onResponse(profiles); + l.onResponse(completionInfo); }) ) ) { @@ -309,7 +320,7 @@ public void execute( .setFailures(r.failures) .build() ); - dataNodesListener.onResponse(r.getProfiles()); + dataNodesListener.onResponse(r.getCompletionInfo()); }, e -> { if (configuration.allowPartialResults()) { execInfo.swapCluster( @@ -318,7 +329,7 @@ public void execute( EsqlExecutionInfo.Cluster.Status.PARTIAL ).setFailures(List.of(new ShardSearchFailure(e))).build() ); - dataNodesListener.onResponse(List.of()); + dataNodesListener.onResponse(DriverCompletionInfo.EMPTY); } else { dataNodesListener.onFailure(e); } @@ -378,7 +389,7 @@ private static void updateExecutionInfoAfterCoordinatorOnlyQuery(EsqlExecutionIn } } - void runCompute(CancellableTask task, ComputeContext context, PhysicalPlan plan, ActionListener> listener) { + void runCompute(CancellableTask task, ComputeContext context, PhysicalPlan plan, ActionListener listener) { listener = ActionListener.runBefore(listener, () -> Releasables.close(context.searchContexts())); List contexts = new ArrayList<>(context.searchContexts().size()); for (int i = 0; i < context.searchContexts().size(); i++) { @@ -435,9 +446,9 @@ public SourceProvider createSourceProvider() { } ActionListener listenerCollectingStatus = listener.map(ignored -> { if (context.configuration().profile()) { - return drivers.stream().map(Driver::profile).toList(); + return DriverCompletionInfo.includingProfiles(drivers); } else { - return List.of(); + return DriverCompletionInfo.excludingProfiles(drivers); } }); listenerCollectingStatus = ActionListener.releaseAfter(listenerCollectingStatus, () -> Releasables.close(drivers)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeHandler.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeHandler.java index bea9c7b7a5db9..4fb62ac21471d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeHandler.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeHandler.java @@ -16,7 +16,7 @@ import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.compute.operator.DriverProfile; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.exchange.ExchangeService; import org.elasticsearch.compute.operator.exchange.ExchangeSink; import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; @@ -191,7 +191,7 @@ protected void sendRequest( TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(computeListener.acquireCompute().map(r -> { nodeResponseRef.set(r); - return r.profiles(); + return r.completionInfo(); }), DataNodeComputeResponse::new, esqlExecutor) ); } @@ -251,15 +251,15 @@ private void runBatch(int startBatchIndex) { final int endBatchIndex = Math.min(startBatchIndex + maxConcurrentShards, request.shardIds().size()); final AtomicInteger pagesProduced = new AtomicInteger(); List shardIds = request.shardIds().subList(startBatchIndex, endBatchIndex); - ActionListener> batchListener = new ActionListener<>() { - final ActionListener> ref = computeListener.acquireCompute(); + ActionListener batchListener = new ActionListener<>() { + final ActionListener ref = computeListener.acquireCompute(); @Override - public void onResponse(List result) { + public void onResponse(DriverCompletionInfo info) { try { onBatchCompleted(endBatchIndex); } finally { - ref.onResponse(result); + ref.onResponse(info); } } @@ -269,7 +269,7 @@ public void onFailure(Exception e) { for (ShardId shardId : shardIds) { addShardLevelFailure(shardId, e); } - onResponse(List.of()); + onResponse(DriverCompletionInfo.EMPTY); } else { // TODO: add these to fatal failures so we can continue processing other shards. try { @@ -283,7 +283,7 @@ public void onFailure(Exception e) { acquireSearchContexts(clusterAlias, shardIds, configuration, request.aliasFilters(), ActionListener.wrap(searchContexts -> { assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH, ESQL_WORKER_THREAD_POOL_NAME); if (searchContexts.isEmpty()) { - batchListener.onResponse(List.of()); + batchListener.onResponse(DriverCompletionInfo.EMPTY); return; } var computeContext = new ComputeContext( diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java index 1313db9e70449..f8e054c579348 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportResponse; @@ -18,43 +19,55 @@ import java.util.Map; import java.util.Objects; +import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; + /** * The compute result of {@link DataNodeRequest} */ final class DataNodeComputeResponse extends TransportResponse { - private final List profiles; + private final DriverCompletionInfo completionInfo; private final Map shardLevelFailures; - DataNodeComputeResponse(List profiles, Map shardLevelFailures) { - this.profiles = profiles; + DataNodeComputeResponse(DriverCompletionInfo completionInfo, Map shardLevelFailures) { + this.completionInfo = completionInfo; this.shardLevelFailures = shardLevelFailures; } DataNodeComputeResponse(StreamInput in) throws IOException { + if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + this.completionInfo = new DriverCompletionInfo(in); + this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); + return; + } if (DataNodeComputeHandler.supportShardLevelRetryFailure(in.getTransportVersion())) { - this.profiles = in.readCollectionAsImmutableList(DriverProfile::readFrom); + this.completionInfo = new DriverCompletionInfo(0, 0, in.readCollectionAsImmutableList(DriverProfile::readFrom)); this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); - } else { - this.profiles = Objects.requireNonNullElse(new ComputeResponse(in).getProfiles(), List.of()); - this.shardLevelFailures = Map.of(); + return; } + this.completionInfo = new ComputeResponse(in).getCompletionInfo(); + this.shardLevelFailures = Map.of(); } @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + completionInfo.writeTo(out); + out.writeMap(shardLevelFailures, (o, v) -> v.writeTo(o), StreamOutput::writeException); + return; + } if (DataNodeComputeHandler.supportShardLevelRetryFailure(out.getTransportVersion())) { - out.writeCollection(profiles, (o, v) -> v.writeTo(o)); + out.writeCollection(completionInfo.collectedProfiles(), (o, v) -> v.writeTo(o)); out.writeMap(shardLevelFailures, (o, v) -> v.writeTo(o), StreamOutput::writeException); - } else { - if (shardLevelFailures.isEmpty() == false) { - throw new IllegalStateException("shard level failures are not supported in old versions"); - } - new ComputeResponse(profiles).writeTo(out); + return; + } + if (shardLevelFailures.isEmpty() == false) { + throw new IllegalStateException("shard level failures are not supported in old versions"); } + new ComputeResponse(completionInfo).writeTo(out); } - List profiles() { - return profiles; + public DriverCompletionInfo completionInfo() { + return completionInfo; } Map shardLevelFailures() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java index 7abc0ba40af76..b823746ba1ec7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.FailureCollector; import org.elasticsearch.core.TimeValue; @@ -113,17 +114,23 @@ final void startComputeOnDataNodes( ) { final long startTimeInNanos = System.nanoTime(); searchShards(rootTask, clusterAlias, requestFilter, concreteIndices, originalIndices, ActionListener.wrap(targetShards -> { - try (var computeListener = new ComputeListener(transportService.getThreadPool(), runOnTaskFailure, listener.map(profiles -> { - return new ComputeResponse( - profiles, - TimeValue.timeValueNanos(System.nanoTime() - startTimeInNanos), - targetShards.totalShards(), - targetShards.totalShards() - shardFailures.size() - skippedShards.get(), - targetShards.skippedShards() + skippedShards.get(), - shardFailures.size(), - selectFailures() - ); - }))) { + try ( + var computeListener = new ComputeListener( + transportService.getThreadPool(), + runOnTaskFailure, + listener.map(completionInfo -> { + return new ComputeResponse( + completionInfo, + TimeValue.timeValueNanos(System.nanoTime() - startTimeInNanos), + targetShards.totalShards(), + targetShards.totalShards() - shardFailures.size() - skippedShards.get(), + targetShards.skippedShards() + skippedShards.get(), + shardFailures.size(), + selectFailures() + ); + }) + ) + ) { for (TargetShard shard : targetShards.shards.values()) { for (DiscoveryNode node : shard.remainingNodes) { nodePermits.putIfAbsent(node, new Semaphore(1)); @@ -236,15 +243,15 @@ private List selectFailures() { } private void sendOneNodeRequest(TargetShards targetShards, ComputeListener computeListener, NodeRequest request) { - final ActionListener> listener = computeListener.acquireCompute(); + final ActionListener listener = computeListener.acquireCompute(); sendRequest(request.node, request.shardIds, request.aliasFilters, new NodeListener() { - void onAfter(List profiles) { + void onAfter(DriverCompletionInfo info) { nodePermits.get(request.node).release(); if (concurrentRequests != null) { concurrentRequests.release(); } trySendingRequestsForPendingShards(targetShards, computeListener); - listener.onResponse(profiles); + listener.onResponse(info); } @Override @@ -260,7 +267,7 @@ public void onResponse(DataNodeComputeResponse response) { trackShardLevelFailure(shardId, false, e.getValue()); pendingShardIds.add(shardId); } - onAfter(response.profiles()); + onAfter(response.completionInfo()); } @Override @@ -269,7 +276,7 @@ public void onFailure(Exception e, boolean receivedData) { trackShardLevelFailure(shardId, receivedData, e); pendingShardIds.add(shardId); } - onAfter(List.of()); + onAfter(DriverCompletionInfo.EMPTY); } @Override @@ -278,7 +285,7 @@ public void onSkip() { if (rootTask.isCancelled()) { onFailure(new TaskCancelledException("null"), true); } else { - onResponse(new DataNodeComputeResponse(List.of(), Map.of())); + onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of())); } } }); 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 65562b9e65c27..9f571c657992f 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 @@ -315,7 +315,9 @@ private EsqlExecutionInfo createEsqlExecutionInfo(EsqlQueryRequest request) { private EsqlQueryResponse toResponse(Task task, EsqlQueryRequest request, Configuration configuration, Result result) { List columns = result.schema().stream().map(c -> new ColumnInfoImpl(c.name(), c.dataType().outputType())).toList(); - EsqlQueryResponse.Profile profile = configuration.profile() ? new EsqlQueryResponse.Profile(result.profiles()) : null; + EsqlQueryResponse.Profile profile = configuration.profile() + ? new EsqlQueryResponse.Profile(result.completionInfo().collectedProfiles()) + : null; threadPool.getThreadContext().addResponseHeader(AsyncExecutionId.ASYNC_EXECUTION_IS_RUNNING_HEADER, "?0"); if (task instanceof EsqlQueryTask asyncTask && request.keepOnCompletion()) { String asyncExecutionId = asyncTask.getExecutionId().getEncoded(); @@ -323,6 +325,8 @@ private EsqlQueryResponse toResponse(Task task, EsqlQueryRequest request, Config return new EsqlQueryResponse( columns, result.pages(), + result.completionInfo().documentsFound(), + result.completionInfo().documentsFound(), profile, request.columnar(), asyncExecutionId, @@ -331,7 +335,16 @@ private EsqlQueryResponse toResponse(Task task, EsqlQueryRequest request, Config result.executionInfo() ); } - return new EsqlQueryResponse(columns, result.pages(), profile, request.columnar(), request.async(), result.executionInfo()); + return new EsqlQueryResponse( + columns, + result.pages(), + result.completionInfo().documentsFound(), + result.completionInfo().valuesLoaded(), + profile, + request.columnar(), + request.async(), + result.executionInfo() + ); } /** @@ -383,6 +396,8 @@ public EsqlQueryResponse initialResponse(EsqlQueryTask task) { return new EsqlQueryResponse( List.of(), List.of(), + 0, + 0, null, false, asyncExecutionId, diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java index 3a2db609d6c8a..bf847a3023c7e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.core.Nullable; import org.elasticsearch.indices.IndicesExpressionGrouper; import org.elasticsearch.license.XPackLicenseState; @@ -75,7 +76,7 @@ abstract static class CssPartialErrorsActionListener implements ActionListener 0) { // code-path to execute subplans - executeSubPlan(new ArrayList<>(), physicalPlan, iterator, executionInfo, runner, listener); + executeSubPlan(new DriverCompletionInfo.Accumulator(), physicalPlan, iterator, executionInfo, runner, listener); } else { // execute main plan runner.run(physicalPlan, listener); @@ -249,7 +250,7 @@ private void executeSubPlans( } private void executeSubPlan( - List profileAccumulator, + DriverCompletionInfo.Accumulator completionInfoAccumulator, PhysicalPlan plan, Iterator subPlanIterator, EsqlExecutionInfo executionInfo, @@ -260,7 +261,7 @@ private void executeSubPlan( runner.run(tuple.physical, listener.delegateFailureAndWrap((next, result) -> { try { - profileAccumulator.addAll(result.profiles()); + completionInfoAccumulator.accumulate(result.completionInfo()); LocalRelation resultWrapper = resultToPlan(tuple.logical, result); // replace the original logical plan with the backing result @@ -296,12 +297,14 @@ private void executeSubPlan( if (subPlanIterator.hasNext() == false) { runner.run(newPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { - profileAccumulator.addAll(finalResult.profiles()); - finalListener.onResponse(new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, executionInfo)); + completionInfoAccumulator.accumulate(finalResult.completionInfo()); + finalListener.onResponse( + new Result(finalResult.schema(), finalResult.pages(), completionInfoAccumulator.finish(), executionInfo) + ); })); } else { // continue executing the subplans - executeSubPlan(profileAccumulator, newPlan, subPlanIterator, executionInfo, runner, next); + executeSubPlan(completionInfoAccumulator, newPlan, subPlanIterator, executionInfo, runner, next); } } finally { Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks))); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java index 4f90893c759b8..38f8303a44b7d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java @@ -9,6 +9,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.Nullable; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; @@ -23,10 +24,7 @@ * that was run. Each {@link Page} contains a {@link Block} of values for each * attribute in this list. * @param pages Actual values produced by running the ESQL. - * @param profiles {@link DriverProfile}s from all drivers that ran to produce the output. These - * are quite cheap to build, so we build them for all ESQL runs, regardless of if - * users have asked for them. But we only include them in the results if users ask - * for them. + * @param completionInfo Information collected from drivers after they've been completed. * @param executionInfo Metadata about the execution of this query. Used for cross cluster queries. */ -public record Result(List schema, List pages, List profiles, @Nullable EsqlExecutionInfo executionInfo) {} +public record Result(List schema, List pages, DriverCompletionInfo completionInfo, @Nullable EsqlExecutionInfo executionInfo) {} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java index 5ec78fd8efbee..f5183eb43a842 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.DriverSleeps; import org.elasticsearch.core.TimeValue; @@ -79,12 +80,12 @@ private List randomProfiles() { } public void testEmpty() { - PlainActionFuture> results = new PlainActionFuture<>(); + PlainActionFuture results = new PlainActionFuture<>(); try (var ignored = new ComputeListener(threadPool, () -> {}, results)) { assertFalse(results.isDone()); } assertTrue(results.isDone()); - assertThat(results.actionGet(10, TimeUnit.SECONDS), empty()); + assertThat(results.actionGet(10, TimeUnit.SECONDS).collectedProfiles(), empty()); } public void testCollectComputeResults() { From b5bf239a31939b666b8f988e1f94157458b03a8d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 25 Mar 2025 18:47:33 -0400 Subject: [PATCH 04/13] Add to response ``` { "took" : 194, "is_partial" : false, "documents_found" : 100000, "values_loaded" : 200000, "columns" : [ { "name" : "a", "type" : "long" }, { "name" : "b", "type" : "long" } ], "values" : [[10, 1]] } ``` --- .../org/elasticsearch/common/Strings.java | 2 +- .../operator/DriverCompletionInfo.java | 1 + .../xpack/esql/action/EsqlCapabilities.java | 4 +- .../xpack/esql/action/EsqlQueryResponse.java | 12 +- .../esql/plugin/ClusterComputeHandler.java | 1 - .../xpack/esql/plugin/ComputeListener.java | 6 - .../xpack/esql/plugin/ComputeService.java | 1 - .../esql/plugin/DataNodeComputeResponse.java | 2 - .../esql/plugin/DataNodeRequestSender.java | 1 - .../xpack/esql/session/EsqlSession.java | 1 - .../xpack/esql/session/Result.java | 8 +- .../elasticsearch/xpack/esql/CsvTests.java | 6 +- .../esql/action/EsqlQueryResponseTests.java | 328 ++++++++++++++---- .../xpack/esql/formatter/TextFormatTests.java | 8 +- .../esql/formatter/TextFormatterTests.java | 6 + .../esql/plugin/ComputeListenerTests.java | 47 ++- .../plugin/DataNodeRequestSenderTests.java | 25 +- .../esql/querylog/EsqlQueryLogTests.java | 3 +- .../rest-api-spec/test/esql/10_basic.yml | 55 ++- .../rest-api-spec/test/esql/120_profile.yml | 2 +- 20 files changed, 391 insertions(+), 128 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/Strings.java b/server/src/main/java/org/elasticsearch/common/Strings.java index b87112386b440..a9cedb7cf7f50 100644 --- a/server/src/main/java/org/elasticsearch/common/Strings.java +++ b/server/src/main/java/org/elasticsearch/common/Strings.java @@ -822,7 +822,7 @@ public static String toString(ChunkedToXContent chunkedToXContent, boolean prett * Allows to configure the params. * Allows to control whether the outputted json needs to be pretty printed and human readable. */ - private static String toString(ToXContent toXContent, ToXContent.Params params, boolean pretty, boolean human) { + public static String toString(ToXContent toXContent, ToXContent.Params params, boolean pretty, boolean human) { try { XContentBuilder builder = createBuilder(pretty, human); if (toXContent.isFragment()) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java index 56bb2a856172a..1e928945da741 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java @@ -26,6 +26,7 @@ * the request. */ public record DriverCompletionInfo(long documentsFound, long valuesLoaded, List collectedProfiles) implements Writeable { + /** * Completion info we use when we didn't properly complete any drivers. * Usually this is returned with an error, but it's also used when receiving diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index abf7dccb152b3..0f1cd2e1466a2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -921,9 +921,9 @@ public enum Cap { /** * Are the {@code documents_found} and {@code values_loaded} fields available - * in the profile? + * in the response and profile? */ - PROFILE_DOCUMENTS_FOUND, + DOCUMENTS_FOUND, /** * Index component selector syntax (my-data-stream-name::failures) 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 1aa9a0c765b49..a7a86da2f6d8d 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 @@ -119,8 +119,8 @@ static EsqlQueryResponse deserialize(BlockStreamInput in) throws IOException { } List columns = in.readCollectionAsList(ColumnInfoImpl::new); List pages = in.readCollectionAsList(Page::new); - long documentsFound = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVInt() : 0; - long valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVInt() : 0; + long documentsFound = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVLong() : 0; + long valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVLong() : 0; if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { profile = in.readOptionalWriteable(Profile::new); } @@ -188,6 +188,14 @@ public Iterator column(int columnIndex) { return ResponseValueUtils.valuesForColumn(columnIndex, columns.get(columnIndex).type(), pages); } + public long documentsFound() { + return documentsFound; + } + + public long valuesLoaded() { + return valuesLoaded; + } + public Profile profile() { return profile; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java index 72d1c2fbc967c..6938fa300022c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ClusterComputeHandler.java @@ -12,7 +12,6 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.exchange.ExchangeService; import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; import org.elasticsearch.core.Releasable; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java index 9e0c8a6853282..856f131cb5645 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java @@ -11,16 +11,10 @@ import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.ResponseHeadersCollector; import org.elasticsearch.core.Releasable; import org.elasticsearch.threadpool.ThreadPool; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - /** * A variant of {@link RefCountingListener} with the following differences: * 1. Automatically cancels sub tasks on failure (via runOnTaskFailure) 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 3d5b61b5bf7ce..6e70136e3c47c 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 @@ -18,7 +18,6 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Driver; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.DriverTaskRunner; import org.elasticsearch.compute.operator.exchange.ExchangeService; import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java index f8e054c579348..1c4d06b9372fe 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java @@ -15,9 +15,7 @@ import org.elasticsearch.transport.TransportResponse; import java.io.IOException; -import java.util.List; import java.util.Map; -import java.util.Objects; import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java index b823746ba1ec7..717d3d0cf9747 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSender.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.FailureCollector; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index 4d890efbda543..79a8bd9980bf7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -18,7 +18,6 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexMode; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java index 38f8303a44b7d..5da8a53e53f15 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java @@ -10,7 +10,6 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.DriverCompletionInfo; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.Nullable; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.core.expression.Attribute; @@ -27,4 +26,9 @@ * @param completionInfo Information collected from drivers after they've been completed. * @param executionInfo Metadata about the execution of this query. Used for cross cluster queries. */ -public record Result(List schema, List pages, DriverCompletionInfo completionInfo, @Nullable EsqlExecutionInfo executionInfo) {} +public record Result( + List schema, + List pages, + DriverCompletionInfo completionInfo, + @Nullable EsqlExecutionInfo executionInfo +) {} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index 90d171bd9796a..0a41a98230533 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.operator.DriverRunner; import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; @@ -710,6 +711,9 @@ protected void start(Driver driver, ActionListener driverListener) { } }; listener = ActionListener.releaseAfter(listener, () -> Releasables.close(drivers)); - runner.runToCompletion(drivers, listener.map(ignore -> new Result(physicalPlan.output(), collectedPages, List.of(), null))); + runner.runToCompletion( + drivers, + listener.map(ignore -> new Result(physicalPlan.output(), collectedPages, DriverCompletionInfo.EMPTY, null)) + ); } } 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 a5d0f2b7b2c99..02cbf8bd0a95f 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 @@ -41,6 +41,7 @@ import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.test.AbstractChunkedSerializingTestCase; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.xcontent.InstantiatingObjectParser; import org.elasticsearch.xcontent.ObjectParser; @@ -128,7 +129,18 @@ EsqlQueryResponse randomResponseAsync(boolean columnar, EsqlQueryResponse.Profil id = randomAlphaOfLengthBetween(1, 16); isRunning = randomBoolean(); } - return new EsqlQueryResponse(columns, values, profile, columnar, id, isRunning, async, createExecutionInfo()); + return new EsqlQueryResponse( + columns, + values, + randomNonNegativeLong(), + randomNonNegativeLong(), + profile, + columnar, + id, + isRunning, + async, + createExecutionInfo() + ); } EsqlExecutionInfo createExecutionInfo() { @@ -249,55 +261,41 @@ protected EsqlQueryResponse mutateInstance(EsqlQueryResponse instance) { allNull = false; } } - return switch (allNull ? between(0, 2) : between(0, 3)) { + List columns = instance.columns(); + List pages = deepCopyOfPages(instance); + long documentsFound = instance.documentsFound(); + long valuesLoaded = instance.valuesLoaded(); + EsqlQueryResponse.Profile profile = instance.profile(); + boolean columnar = instance.columnar(); + boolean isAsync = instance.isAsync(); + EsqlExecutionInfo executionInfo = instance.getExecutionInfo(); + switch (allNull ? between(0, 4) : between(0, 5)) { case 0 -> { int mutCol = between(0, instance.columns().size() - 1); - List cols = new ArrayList<>(instance.columns()); + columns = new ArrayList<>(instance.columns()); // keep the type the same so the values are still valid but change the name - cols.set(mutCol, new ColumnInfoImpl(cols.get(mutCol).name() + "mut", cols.get(mutCol).type())); - yield new EsqlQueryResponse( - cols, - deepCopyOfPages(instance), - instance.profile(), - instance.columnar(), - instance.isAsync(), - instance.getExecutionInfo() - ); + columns.set(mutCol, new ColumnInfoImpl(columns.get(mutCol).name() + "mut", columns.get(mutCol).type())); } - case 1 -> new EsqlQueryResponse( - instance.columns(), - deepCopyOfPages(instance), - instance.profile(), - false == instance.columnar(), - instance.isAsync(), - instance.getExecutionInfo() - ); - case 2 -> new EsqlQueryResponse( - instance.columns(), - deepCopyOfPages(instance), - randomValueOtherThan(instance.profile(), this::randomProfile), - instance.columnar(), - instance.isAsync(), - instance.getExecutionInfo() - ); - case 3 -> { + case 1 -> documentsFound = randomValueOtherThan(documentsFound, ESTestCase::randomNonNegativeLong); + case 2 -> valuesLoaded = randomValueOtherThan(valuesLoaded, ESTestCase::randomNonNegativeLong); + case 3 -> columnar = false == columnar; + case 4 -> profile = randomValueOtherThan(profile, this::randomProfile); + case 5 -> { + assert allNull == false + : "can't replace values while preserving types if all pages are null - the only valid values are null"; int noPages = instance.pages().size(); List differentPages = List.of(); do { differentPages.forEach(p -> Releasables.closeExpectNoException(p::releaseBlocks)); differentPages = randomList(noPages, noPages, () -> randomPage(instance.columns())); } while (differentPages.equals(instance.pages())); - yield new EsqlQueryResponse( - instance.columns(), - differentPages, - instance.profile(), - instance.columnar(), - instance.isAsync(), - instance.getExecutionInfo() - ); + pages.forEach(Page::releaseBlocks); + pages = differentPages; } default -> throw new IllegalArgumentException(); - }; + } + ; + return new EsqlQueryResponse(columns, pages, documentsFound, valuesLoaded, profile, columnar, isAsync, executionInfo); } private List deepCopyOfPages(EsqlQueryResponse response) { @@ -349,6 +347,8 @@ public static class ResponseBuilder { ObjectParser.ValueType.BOOLEAN_OR_NULL ); parser.declareInt(constructorArg(), new ParseField("took")); + parser.declareLong(constructorArg(), new ParseField("documents_found")); + parser.declareLong(constructorArg(), new ParseField("values_loaded")); parser.declareObjectArray(constructorArg(), (p, c) -> ColumnInfoImpl.fromXContent(p), new ParseField("columns")); parser.declareField(constructorArg(), (p, c) -> p.list(), new ParseField("values"), ObjectParser.ValueType.OBJECT_ARRAY); parser.declareObject(optionalConstructorArg(), (p, c) -> parseClusters(p), new ParseField("_clusters")); @@ -363,6 +363,8 @@ public ResponseBuilder( @Nullable String asyncExecutionId, Boolean isRunning, Integer took, + long documentsFound, + long valuesLoaded, List columns, List> values, EsqlExecutionInfo executionInfo @@ -371,6 +373,8 @@ public ResponseBuilder( this.response = new EsqlQueryResponse( columns, List.of(valuesToPage(TestBlockFactory.getNonBreakingInstance(), columns, values)), + documentsFound, + valuesLoaded, null, false, asyncExecutionId, @@ -565,62 +569,154 @@ public void testChunkResponseSizeColumnar() { 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 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + 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 -> 6 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize); // is_running + assertChunkCount(resp, r -> 7 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize); // is_running } } public void testChunkResponseSizeRows() { try (EsqlQueryResponse resp = randomResponse(false, null)) { int bodySize = resp.pages().stream().mapToInt(Page::getPositionCount).sum(); - assertChunkCount(resp, r -> 5 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + 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 -> 6 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize); + assertChunkCount(resp, r -> 7 + clusterDetailsSize(resp.getExecutionInfo().clusterInfo.size()) + bodySize); } } public void testSimpleXContentColumnar() { try (EsqlQueryResponse response = simple(true)) { - assertThat(Strings.toString(wrapAsToXContent(response)), equalTo(""" - {"columns":[{"name":"foo","type":"integer"}],"values":[[40,80]]}""")); + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" + { + "documents_found" : 3, + "values_loaded" : 100, + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40, + 80 + ] + ] + }""")); } } public void testSimpleXContentColumnarDropNulls() { try (EsqlQueryResponse response = simple(true)) { assertThat( - Strings.toString(wrapAsToXContent(response), new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true"))), + Strings.toString( + wrapAsToXContent(response), + new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true")), + true, + false + ), equalTo(""" - {"all_columns":[{"name":"foo","type":"integer"}],"columns":[{"name":"foo","type":"integer"}],"values":[[40,80]]}""") + { + "documents_found" : 3, + "values_loaded" : 100, + "all_columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40, + 80 + ] + ] + }""") ); } } public void testSimpleXContentColumnarAsync() { try (EsqlQueryResponse response = simple(true, true)) { - assertThat(Strings.toString(wrapAsToXContent(response)), equalTo(""" - {"is_running":false,"columns":[{"name":"foo","type":"integer"}],"values":[[40,80]]}""")); + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" + { + "is_running" : false, + "documents_found" : 3, + "values_loaded" : 100, + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40, + 80 + ] + ] + }""")); } } public void testSimpleXContentRows() { try (EsqlQueryResponse response = simple(false)) { - assertThat(Strings.toString(wrapAsToXContent(response)), equalTo(""" - {"columns":[{"name":"foo","type":"integer"}],"values":[[40],[80]]}""")); + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" + { + "documents_found" : 3, + "values_loaded" : 100, + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40 + ], + [ + 80 + ] + ] + }""")); } } public void testSimpleXContentRowsAsync() { try (EsqlQueryResponse response = simple(false, true)) { - assertThat(Strings.toString(wrapAsToXContent(response)), equalTo(""" - {"is_running":false,"columns":[{"name":"foo","type":"integer"}],"values":[[40],[80]]}""")); + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" + { + "is_running" : false, + "documents_found" : 3, + "values_loaded" : 100, + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40 + ], + [ + 80 + ] + ] + }""")); } } @@ -629,6 +725,8 @@ public void testBasicXContentIdAndRunning() { EsqlQueryResponse response = new EsqlQueryResponse( List.of(new ColumnInfoImpl("foo", "integer")), List.of(new Page(blockFactory.newIntArrayVector(new int[] { 40, 80 }, 2).asBlock())), + 10, + 99, null, false, "id-123", @@ -637,8 +735,27 @@ public void testBasicXContentIdAndRunning() { null ) ) { - assertThat(Strings.toString(response), equalTo(""" - {"id":"id-123","is_running":true,"columns":[{"name":"foo","type":"integer"}],"values":[[40],[80]]}""")); + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" + { + "id" : "id-123", + "is_running" : true, + "documents_found" : 10, + "values_loaded" : 99, + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40 + ], + [ + 80 + ] + ] + }""")); } } @@ -647,6 +764,8 @@ public void testNullColumnsXContentDropNulls() { EsqlQueryResponse response = new EsqlQueryResponse( List.of(new ColumnInfoImpl("foo", "integer"), new ColumnInfoImpl("all_null", "integer")), List.of(new Page(blockFactory.newIntArrayVector(new int[] { 40, 80 }, 2).asBlock(), blockFactory.newConstantNullBlock(2))), + 1, + 3, null, false, null, @@ -656,11 +775,41 @@ public void testNullColumnsXContentDropNulls() { ) ) { assertThat( - Strings.toString(wrapAsToXContent(response), new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true"))), - equalTo("{" + """ - "all_columns":[{"name":"foo","type":"integer"},{"name":"all_null","type":"integer"}],""" + """ - "columns":[{"name":"foo","type":"integer"}],""" + """ - "values":[[40],[80]]}""") + Strings.toString( + wrapAsToXContent(response), + new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true")), + true, + false + ), + equalTo(""" + { + "documents_found" : 1, + "values_loaded" : 3, + "all_columns" : [ + { + "name" : "foo", + "type" : "integer" + }, + { + "name" : "all_null", + "type" : "integer" + } + ], + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40 + ], + [ + 80 + ] + ] + }""") ); } } @@ -677,6 +826,8 @@ public void testNullColumnsFromBuilderXContentDropNulls() { EsqlQueryResponse response = new EsqlQueryResponse( List.of(new ColumnInfoImpl("foo", "integer"), new ColumnInfoImpl("all_null", "integer")), List.of(new Page(blockFactory.newIntArrayVector(new int[] { 40, 80 }, 2).asBlock(), b.build())), + 1, + 3, null, false, null, @@ -686,11 +837,41 @@ public void testNullColumnsFromBuilderXContentDropNulls() { ) ) { assertThat( - Strings.toString(wrapAsToXContent(response), new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true"))), - equalTo("{" + """ - "all_columns":[{"name":"foo","type":"integer"},{"name":"all_null","type":"integer"}],""" + """ - "columns":[{"name":"foo","type":"integer"}],""" + """ - "values":[[40],[80]]}""") + Strings.toString( + wrapAsToXContent(response), + new ToXContent.MapParams(Map.of(DROP_NULL_COLUMNS_OPTION, "true")), + true, + false + ), + equalTo(""" + { + "documents_found" : 1, + "values_loaded" : 3, + "all_columns" : [ + { + "name" : "foo", + "type" : "integer" + }, + { + "name" : "all_null", + "type" : "integer" + } + ], + "columns" : [ + { + "name" : "foo", + "type" : "integer" + } + ], + "values" : [ + [ + 40 + ], + [ + 80 + ] + ] + }""") ); } } @@ -704,6 +885,8 @@ private EsqlQueryResponse simple(boolean columnar, boolean async) { return new EsqlQueryResponse( List.of(new ColumnInfoImpl("foo", "integer")), List.of(new Page(blockFactory.newIntArrayVector(new int[] { 40, 80 }, 2).asBlock())), + 3, + 100, null, columnar, async, @@ -716,6 +899,8 @@ public void testProfileXContent() { EsqlQueryResponse response = new EsqlQueryResponse( List.of(new ColumnInfoImpl("foo", "integer")), List.of(new Page(blockFactory.newIntArrayVector(new int[] { 40, 80 }, 2).asBlock())), + 10, + 100, new EsqlQueryResponse.Profile( List.of( new DriverProfile( @@ -737,8 +922,10 @@ public void testProfileXContent() { null ); ) { - assertThat(Strings.toString(response, true, false), equalTo(""" + assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" { + "documents_found" : 10, + "values_loaded" : 100, "columns" : [ { "name" : "foo", @@ -802,7 +989,7 @@ public void testColumns() { var longBlk2 = blockFactory.newLongArrayVector(new long[] { 300L, 400L, 500L }, 3).asBlock(); var columnInfo = List.of(new ColumnInfoImpl("foo", "integer"), new ColumnInfoImpl("bar", "long")); var pages = List.of(new Page(intBlk1, longBlk1), new Page(intBlk2, longBlk2)); - try (var response = new EsqlQueryResponse(columnInfo, pages, null, false, null, false, false, null)) { + try (var response = new EsqlQueryResponse(columnInfo, pages, 0, 0, null, false, null, false, false, null)) { assertThat(columnValues(response.column(0)), contains(10, 20, 30, 40, 50)); assertThat(columnValues(response.column(1)), contains(100L, 200L, 300L, 400L, 500L)); expectThrows(IllegalArgumentException.class, () -> response.column(-1)); @@ -814,7 +1001,7 @@ public void testColumnsIllegalArg() { var intBlk1 = blockFactory.newIntArrayVector(new int[] { 10 }, 1).asBlock(); var columnInfo = List.of(new ColumnInfoImpl("foo", "integer")); var pages = List.of(new Page(intBlk1)); - try (var response = new EsqlQueryResponse(columnInfo, pages, null, false, null, false, false, null)) { + try (var response = new EsqlQueryResponse(columnInfo, pages, 0, 0, null, false, null, false, false, null)) { expectThrows(IllegalArgumentException.class, () -> response.column(-1)); expectThrows(IllegalArgumentException.class, () -> response.column(1)); } @@ -833,7 +1020,7 @@ public void testColumnsWithNull() { } var columnInfo = List.of(new ColumnInfoImpl("foo", "integer")); var pages = List.of(new Page(blk1), new Page(blk2), new Page(blk3)); - try (var response = new EsqlQueryResponse(columnInfo, pages, null, false, null, false, false, null)) { + try (var response = new EsqlQueryResponse(columnInfo, pages, 0, 0, null, false, null, false, false, null)) { assertThat(columnValues(response.column(0)), contains(10, null, 30, null, null, 60, null, 80, 90, null)); expectThrows(IllegalArgumentException.class, () -> response.column(-1)); expectThrows(IllegalArgumentException.class, () -> response.column(2)); @@ -853,7 +1040,7 @@ public void testColumnsWithMultiValue() { } var columnInfo = List.of(new ColumnInfoImpl("foo", "integer")); var pages = List.of(new Page(blk1), new Page(blk2), new Page(blk3)); - try (var response = new EsqlQueryResponse(columnInfo, pages, null, false, null, false, false, null)) { + try (var response = new EsqlQueryResponse(columnInfo, pages, 0, 0, null, false, null, false, false, null)) { assertThat(columnValues(response.column(0)), contains(List.of(10, 20), null, List.of(40, 50), null, 70, 80, null)); expectThrows(IllegalArgumentException.class, () -> response.column(-1)); expectThrows(IllegalArgumentException.class, () -> response.column(2)); @@ -866,7 +1053,7 @@ public void testRowValues() { List columns = randomList(numColumns, numColumns, this::randomColumnInfo); int noPages = randomIntBetween(1, 20); List pages = randomList(noPages, noPages, () -> randomPage(columns)); - try (var resp = new EsqlQueryResponse(columns, pages, null, false, "", false, false, null)) { + try (var resp = new EsqlQueryResponse(columns, pages, 0, 0, null, false, "", false, false, null)) { var rowValues = getValuesList(resp.rows()); var valValues = getValuesList(resp.values()); for (int i = 0; i < rowValues.size(); i++) { @@ -964,5 +1151,4 @@ static Page valuesToPage(BlockFactory blockFactory, List columns } return new Page(results.stream().map(Block.Builder::build).toArray(Block[]::new)); } - } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java index f3746db2b38a6..178a5e13d6e19 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java @@ -246,7 +246,7 @@ public void testPlainTextEmptyCursorWithColumns() { public void testPlainTextEmptyCursorWithoutColumns() { assertEquals( StringUtils.EMPTY, - getTextBodyContent(PLAIN_TEXT.format(req(), new EsqlQueryResponse(emptyList(), emptyList(), null, false, false, null))) + getTextBodyContent(PLAIN_TEXT.format(req(), new EsqlQueryResponse(emptyList(), emptyList(), 0, 0, null, false, false, null))) ); } @@ -269,7 +269,7 @@ public void testTsvFormatWithDropNullColumns() { } private static EsqlQueryResponse emptyData() { - return new EsqlQueryResponse(singletonList(new ColumnInfoImpl("name", "keyword")), emptyList(), null, false, false, null); + return new EsqlQueryResponse(singletonList(new ColumnInfoImpl("name", "keyword")), emptyList(), 0, 0, null, false, false, null); } private static EsqlQueryResponse regularData() { @@ -303,7 +303,7 @@ private static EsqlQueryResponse regularData() { ) ); - return new EsqlQueryResponse(headers, values, null, false, false, null); + return new EsqlQueryResponse(headers, values, 0, 0, null, false, false, null); } private static EsqlQueryResponse escapedData() { @@ -324,7 +324,7 @@ private static EsqlQueryResponse escapedData() { ) ); - return new EsqlQueryResponse(headers, values, null, false, false, null); + return new EsqlQueryResponse(headers, values, 0, 0, null, false, false, null); } private static RestRequest req() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java index 6ca63dfb84f37..d9f50c2229f93 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java @@ -79,6 +79,8 @@ public class TextFormatterTests extends ESTestCase { blockFactory.newConstantNullBlock(2) ) ), + 0, + 0, null, randomBoolean(), randomBoolean(), @@ -181,6 +183,8 @@ public void testFormatWithoutHeader() { blockFactory.newConstantNullBlock(2) ) ), + 0, + 0, null, randomBoolean(), randomBoolean(), @@ -222,6 +226,8 @@ public void testVeryLongPadding() { .build() ) ), + 0, + 0, null, randomBoolean(), randomBoolean(), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java index f5183eb43a842..cac20924ed3b4 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ComputeListenerTests.java @@ -35,6 +35,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -57,7 +58,7 @@ public void shutdownTransportService() { terminate(threadPool); } - private List randomProfiles() { + private DriverCompletionInfo randomCompletionInfo() { int numProfiles = randomIntBetween(0, 2); List profiles = new ArrayList<>(numProfiles); for (int i = 0; i < numProfiles; i++) { @@ -76,7 +77,7 @@ private List randomProfiles() { ) ); } - return profiles; + return new DriverCompletionInfo(randomNonNegativeLong(), randomNonNegativeLong(), profiles); } public void testEmpty() { @@ -89,7 +90,9 @@ public void testEmpty() { } public void testCollectComputeResults() { - PlainActionFuture> future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); + long documentsFound = 0; + long valuesLoaded = 0; List allProfiles = new ArrayList<>(); AtomicInteger onFailure = new AtomicInteger(); try (var computeListener = new ComputeListener(threadPool, onFailure::incrementAndGet, future)) { @@ -103,20 +106,24 @@ public void testCollectComputeResults() { threadPool.generic() ); } else { - var profiles = randomProfiles(); - allProfiles.addAll(profiles); - ActionListener> subListener = computeListener.acquireCompute(); + var info = randomCompletionInfo(); + documentsFound += info.documentsFound(); + valuesLoaded += info.valuesLoaded(); + allProfiles.addAll(info.collectedProfiles()); + ActionListener subListener = computeListener.acquireCompute(); threadPool.schedule( - ActionRunnable.wrap(subListener, l -> l.onResponse(profiles)), + ActionRunnable.wrap(subListener, l -> l.onResponse(info)), TimeValue.timeValueNanos(between(0, 100)), threadPool.generic() ); } } } - List profiles = future.actionGet(10, TimeUnit.SECONDS); + DriverCompletionInfo actual = future.actionGet(10, TimeUnit.SECONDS); + assertThat(actual.documentsFound(), equalTo(documentsFound)); + assertThat(actual.valuesLoaded(), equalTo(valuesLoaded)); assertThat( - profiles.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), + actual.collectedProfiles().stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), equalTo(allProfiles.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum))) ); assertThat(onFailure.get(), equalTo(0)); @@ -130,13 +137,13 @@ public void testCancelOnFailure() { ); int successTasks = between(1, 50); int failedTasks = between(1, 100); - PlainActionFuture> rootListener = new PlainActionFuture<>(); + PlainActionFuture rootListener = new PlainActionFuture<>(); final AtomicInteger onFailure = new AtomicInteger(); try (var computeListener = new ComputeListener(threadPool, onFailure::incrementAndGet, rootListener)) { for (int i = 0; i < successTasks; i++) { - ActionListener> subListener = computeListener.acquireCompute(); + ActionListener subListener = computeListener.acquireCompute(); threadPool.schedule( - ActionRunnable.wrap(subListener, l -> l.onResponse(randomProfiles())), + ActionRunnable.wrap(subListener, l -> l.onResponse(randomCompletionInfo())), TimeValue.timeValueNanos(between(0, 100)), threadPool.generic() ); @@ -161,13 +168,17 @@ public void testCancelOnFailure() { } public void testCollectWarnings() throws Exception { + AtomicLong documentsFound = new AtomicLong(); + AtomicLong valuesLoaded = new AtomicLong(); List allProfiles = new ArrayList<>(); Map> allWarnings = new HashMap<>(); - ActionListener> rootListener = new ActionListener<>() { + ActionListener rootListener = new ActionListener<>() { @Override - public void onResponse(List result) { + public void onResponse(DriverCompletionInfo result) { + assertThat(result.documentsFound(), equalTo(documentsFound.get())); + assertThat(result.valuesLoaded(), equalTo(valuesLoaded.get())); assertThat( - result.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), + result.collectedProfiles().stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), equalTo(allProfiles.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum))) ); Map> responseHeaders = threadPool.getThreadContext() @@ -202,8 +213,10 @@ public void onFailure(Exception e) { threadPool.generic() ); } else { - var resp = randomProfiles(); - allProfiles.addAll(resp); + var resp = randomCompletionInfo(); + documentsFound.addAndGet(resp.documentsFound()); + valuesLoaded.addAndGet(resp.valuesLoaded()); + allProfiles.addAll(resp.collectedProfiles()); int numWarnings = randomIntBetween(1, 5); Map warnings = new HashMap<>(); for (int i = 0; i < numWarnings; i++) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSenderTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSenderTests.java index 92c77f7bd47c7..8503f17e2270b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSenderTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSenderTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.compute.test.ComputeTestCase; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; @@ -121,7 +122,7 @@ public void testOnePass() { Queue sent = ConcurrentCollections.newQueue(); var future = sendRequests(targetShards, randomBoolean(), -1, (node, shardIds, aliasFilters, listener) -> { sent.add(new NodeRequest(node, shardIds, aliasFilters)); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); }); safeGet(future); assertThat(sent.size(), equalTo(2)); @@ -140,7 +141,7 @@ public void testMissingShards() { var targetShards = List.of(targetShard(shard1, node1), targetShard(shard3), targetShard(shard4, node2, node3)); var future = sendRequests(targetShards, true, -1, (node, shardIds, aliasFilters, listener) -> { assertThat(shard3, not(in(shardIds))); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); }); ComputeResponse resp = safeGet(future); assertThat(resp.totalShards, equalTo(3)); @@ -171,7 +172,7 @@ public void testRetryThenSuccess() { if (node.equals(node4) && shardIds.contains(shard2)) { failures.put(shard2, new IOException("test")); } - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), failures))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, failures))); }); try { future.actionGet(1, TimeUnit.MINUTES); @@ -200,7 +201,7 @@ public void testRetryButFail() { if (shardIds.contains(shard5)) { failures.put(shard5, new IOException("test failure for shard5")); } - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), failures))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, failures))); }); var error = expectThrows(Exception.class, future::actionGet); assertNotNull(ExceptionsHelper.unwrap(error, IOException.class)); @@ -225,7 +226,7 @@ public void testDoNotRetryOnRequestLevelFailure() { if (node1.equals(node) && failed.compareAndSet(false, true)) { runWithDelay(() -> listener.onFailure(new IOException("test request level failure"), true)); } else { - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); } }); Exception exception = expectThrows(Exception.class, future::actionGet); @@ -245,7 +246,7 @@ public void testAllowPartialResults() { if (node1.equals(node) && failed.compareAndSet(false, true)) { runWithDelay(() -> listener.onFailure(new IOException("test request level failure"), true)); } else { - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); } }); ComputeResponse resp = safeGet(future); @@ -266,7 +267,7 @@ public void testNonFatalErrorIsRetriedOnAnotherShard() { if (Objects.equals(node1, node)) { runWithDelay(() -> listener.onFailure(new RuntimeException("test request level non fatal failure"), false)); } else { - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); } })); assertThat(response.totalShards, equalTo(1)); @@ -324,7 +325,7 @@ public void testLimitConcurrentNodes() { sent.add(new NodeRequest(node, shardIds, aliasFilters)); runWithDelay(() -> { concurrentRequests.decrementAndGet(); - listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of())); + listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of())); }); })); assertThat(sent.size(), equalTo(5)); @@ -347,7 +348,7 @@ public void testSkipNodes() { var response = safeGet(sendRequests(targetShards, randomBoolean(), 1, (node, shardIds, aliasFilters, listener) -> { runWithDelay(() -> { if (processed.incrementAndGet() == 1) { - listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of())); + listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of())); } else { listener.onSkip(); } @@ -369,7 +370,7 @@ public void testSkipRemovesPriorNonFatalErrors() { if (Objects.equals(node.getId(), node1.getId()) && shardIds.equals(List.of(shard1))) { listener.onFailure(new RuntimeException("test request level non fatal failure"), false); } else if (Objects.equals(node.getId(), node3.getId()) && shardIds.equals(List.of(shard2))) { - listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of())); + listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of())); } else if (Objects.equals(node.getId(), node2.getId()) && shardIds.equals(List.of(shard1))) { listener.onSkip(); } @@ -394,7 +395,7 @@ public void testQueryHotShardsFirst() { var sent = Collections.synchronizedList(new ArrayList()); safeGet(sendRequests(targetShards, randomBoolean(), -1, (node, shardIds, aliasFilters, listener) -> { sent.add(node.getId()); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); })); assertThat(sent, equalTo(List.of("node-1", "node-2", "node-3", "node-4"))); } @@ -407,7 +408,7 @@ public void testQueryHotShardsFirstWhenIlmMovesShard() { var sent = ConcurrentCollections.newQueue(); safeGet(sendRequests(targetShards, randomBoolean(), -1, (node, shardIds, aliasFilters, listener) -> { sent.add(new NodeRequest(node, shardIds, aliasFilters)); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(DriverCompletionInfo.EMPTY, Map.of()))); })); assertThat(groupRequests(sent, 1), equalTo(Map.of(node1, List.of(shard1)))); assertThat(groupRequests(sent, 1), anyOf(equalTo(Map.of(node2, List.of(shard2))), equalTo(Map.of(warmNode2, List.of(shard2))))); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querylog/EsqlQueryLogTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querylog/EsqlQueryLogTests.java index add3bf77efb00..b23517dd14088 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querylog/EsqlQueryLogTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querylog/EsqlQueryLogTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.compute.operator.DriverCompletionInfo; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.SlowLogFieldProvider; @@ -105,7 +106,7 @@ public void testPrioritiesOnSuccess() { for (int i = 0; i < actualTook.length; i++) { EsqlExecutionInfo warnQuery = getEsqlExecutionInfo(actualTook[i], actualPlanningTook[i]); - queryLog.onQueryPhase(new Result(List.of(), List.of(), List.of(), warnQuery), query); + queryLog.onQueryPhase(new Result(List.of(), List.of(), DriverCompletionInfo.EMPTY, warnQuery), query); if (expectedLevel[i] != null) { assertThat(appender.lastEvent(), is(not(nullValue()))); var msg = (ESLogMessage) appender.lastMessage(); diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml index 96145e84ad2cd..64cc90df4c358 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml @@ -169,7 +169,7 @@ setup: - match: {values.0.1: 40} --- -"Basic ESQL query": +basic: - do: esql.query: body: @@ -181,12 +181,63 @@ setup: - match: {values.0: [1, 1]} --- -"Test From Eval Sort Limit": +basic with documents_found: + - requires: + test_runner_features: [capabilities, contains] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [documents_found] + reason: "checks for documents_found and values_loaded" + + - do: + esql.query: + body: + query: 'from test | keep data | sort data | limit 2' + columnar: true + + - match: {documents_found: 10} # two documents per shard + - match: {values_loaded: 10} # one per document + - match: {columns.0.name: "data"} + - match: {columns.0.type: "long"} + - match: {values.0: [1, 1]} + +--- +FROM EVAL SORT LIMIT: + - do: + esql.query: + body: + query: 'from test | eval x = count + 7 | sort x | limit 1' + + - match: {columns.0.name: "color"} + - match: {columns.1.name: "count"} + - match: {columns.2.name: "count_d"} + - match: {columns.3.name: "data"} + - match: {columns.4.name: "data_d"} + - match: {columns.5.name: "time"} + - match: {columns.6.name: "x"} + - match: {values.0.6: 47} + - length: {values: 1} + +--- +FROM EVAL SORT LIMIT with documents_found: + - requires: + test_runner_features: [capabilities, contains] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [documents_found] + reason: "checks for documents_found and values_loaded" + - do: esql.query: body: query: 'from test | eval x = count + 7 | sort x | limit 1' + - match: {documents_found: 40} + - match: {values_loaded: 65} - match: {columns.0.name: "color"} - match: {columns.1.name: "count"} - match: {columns.2.name: "count_d"} diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml index c15e05b1ac551..8a90915caa8c6 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml @@ -145,7 +145,7 @@ documents found: - method: POST path: /_query parameters: [] - capabilities: [profile_documents_found] + capabilities: [documents_found] reason: "checks for documents_found and values_loaded" - do: From 7d306c07051a1d872bfb1d5e12fcf3e5385a5ba1 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 25 Mar 2025 19:05:08 -0400 Subject: [PATCH 05/13] Update docs/changelog/125631.yaml --- docs/changelog/125631.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/125631.yaml diff --git a/docs/changelog/125631.yaml b/docs/changelog/125631.yaml new file mode 100644 index 0000000000000..32917bb1da060 --- /dev/null +++ b/docs/changelog/125631.yaml @@ -0,0 +1,5 @@ +pr: 125631 +summary: Add `documents_found` and `values_loaded` +area: ES|QL +type: enhancement +issues: [] From 75cef177f1dc70725a6fce9b7a73fe7c97d608b0 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 26 Mar 2025 10:54:54 -0400 Subject: [PATCH 06/13] Tets --- .../org/elasticsearch/test/rest/ESRestTestCase.java | 10 +++++++++- .../xpack/esql/qa/single_node/RestEsqlIT.java | 7 +++++-- .../xpack/esql/qa/rest/RestEsqlTestCase.java | 11 ++++++++--- 3 files changed, 22 insertions(+), 6 deletions(-) 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 ceceea5140236..16332e6cb98c7 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 @@ -2667,6 +2667,8 @@ public static Request newXContentRequest(HttpMethod method, String endpoint, ToX protected static MapMatcher getResultMatcher(boolean includeMetadata, boolean includePartial) { MapMatcher mapMatcher = matchesMap(); + mapMatcher = mapMatcher.entry("documents_found", greaterThanOrEqualTo(0)); + mapMatcher = mapMatcher.entry("values_loaded", greaterThanOrEqualTo(0)); if (includeMetadata) { mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); } @@ -2688,7 +2690,13 @@ protected static MapMatcher getResultMatcher(Map result) { * 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)); + assertMap( + result, + getResultMatcher(result).entry("columns", columnMatcher) + .entry("values", valuesMatcher) + .entry("documents_found", greaterThanOrEqualTo(0)) + .entry("values_loaded", greaterThanOrEqualTo(0)) + ); } protected static void assertResultMap(Map result, Object columnMatcher, Object valuesMatcher) { 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 1b44536eed508..5656be25d00f5 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 @@ -659,7 +659,9 @@ private MapMatcher commonProfile() { .entry("cpu_nanos", greaterThan(0L)) .entry("took_nanos", greaterThan(0L)) .entry("operators", instanceOf(List.class)) - .entry("sleeps", matchesMap().extraOk()); + .entry("sleeps", matchesMap().extraOk()) + .entry("documents_found", greaterThanOrEqualTo(0)) + .entry("values_loaded", greaterThanOrEqualTo(0)); } /** @@ -688,7 +690,8 @@ private String checkOperatorProfile(Map o) { .entry("rows_emitted", greaterThan(0)) .entry("process_nanos", greaterThan(0)) .entry("processed_queries", List.of("*:*")); - case "ValuesSourceReaderOperator" -> basicProfile().entry("readers_built", matchesMap().extraOk()); + case "ValuesSourceReaderOperator" -> basicProfile().entry("values_loaded", greaterThanOrEqualTo(0)) + .entry("readers_built", matchesMap().extraOk()); case "AggregationOperator" -> matchesMap().entry("pages_processed", greaterThan(0)) .entry("rows_received", greaterThan(0)) .entry("rows_emitted", greaterThan(0)) 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 b25f57b77f074..cb165906cee7e 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,6 +59,7 @@ 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; @@ -266,12 +267,16 @@ public static RequestObjectBuilder jsonBuilder() throws IOException { public void testGetAnswer() throws IOException { Map answer = runEsql(requestObjectBuilder().query("row a = 1, b = 2")); - assertEquals(4, answer.size()); + assertEquals(6, 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"); - assertEquals(List.of(colA, colB), answer.get("columns")); - assertEquals(List.of(List.of(1, 2)), answer.get("values")); + assertMap(answer, matchesMap().entry("took", greaterThanOrEqualTo(0)) + .entry("is_partial", any(Boolean.class)) + .entry("documents_found", 0) + .entry("values_loaded", 0) + .entry("columns", List.of(colA, colB)) + .entry("values", List.of(List.of(1, 2)))); } public void testUseUnknownIndex() throws IOException { From 12aafc47897d788d6a4a303841b6c24e36bca29a Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 26 Mar 2025 11:25:36 -0400 Subject: [PATCH 07/13] fix more --- .../java/org/elasticsearch/test/rest/ESRestTestCase.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) 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 16332e6cb98c7..1bc1300fb0219 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 @@ -2690,13 +2690,7 @@ protected static MapMatcher getResultMatcher(Map result) { * 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) - .entry("documents_found", greaterThanOrEqualTo(0)) - .entry("values_loaded", greaterThanOrEqualTo(0)) - ); + assertMap(result, getResultMatcher(result).entry("columns", columnMatcher).entry("values", valuesMatcher)); } protected static void assertResultMap(Map result, Object columnMatcher, Object valuesMatcher) { From 62b5008059cf1d58364b6424c78ee2eb07cb5bf8 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 26 Mar 2025 15:35:33 +0000 Subject: [PATCH 08/13] [CI] Auto commit changes from spotless --- .../xpack/esql/qa/rest/RestEsqlTestCase.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 cb165906cee7e..93c1afc37743a 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 @@ -271,12 +271,15 @@ public void testGetAnswer() throws IOException { assertThat(((Integer) answer.get("took")).intValue(), greaterThanOrEqualTo(0)); Map colA = Map.of("name", "a", "type", "integer"); Map colB = Map.of("name", "b", "type", "integer"); - assertMap(answer, matchesMap().entry("took", greaterThanOrEqualTo(0)) - .entry("is_partial", any(Boolean.class)) - .entry("documents_found", 0) - .entry("values_loaded", 0) - .entry("columns", List.of(colA, colB)) - .entry("values", List.of(List.of(1, 2)))); + assertMap( + answer, + matchesMap().entry("took", greaterThanOrEqualTo(0)) + .entry("is_partial", any(Boolean.class)) + .entry("documents_found", 0) + .entry("values_loaded", 0) + .entry("columns", List.of(colA, colB)) + .entry("values", List.of(List.of(1, 2))) + ); } public void testUseUnknownIndex() throws IOException { From 9335c3edd372478f537d4222392b57d225694ca4 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 26 Mar 2025 12:03:29 -0400 Subject: [PATCH 09/13] Fixup --- .../xpack/esql/formatter/TextFormatTests.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java index b8cd6e771395d..9c45c998ac96a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java @@ -269,11 +269,16 @@ public void testTsvFormatWithDropNullColumns() { } private static EsqlQueryResponse emptyData() { -<<<<<<< HEAD - return new EsqlQueryResponse(singletonList(new ColumnInfoImpl("name", "keyword")), emptyList(), 0, 0, null, false, false, null); -======= - return new EsqlQueryResponse(singletonList(new ColumnInfoImpl("name", "keyword", null)), emptyList(), null, false, false, null); ->>>>>>> main + return new EsqlQueryResponse( + singletonList(new ColumnInfoImpl("name", "keyword", null)), + emptyList(), + 0, + 0, + null, + false, + false, + null + ); } private static EsqlQueryResponse regularData() { From 5067e281b9e56e5a9caf115e54ba3e17add3d1b8 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 26 Mar 2025 12:21:20 -0400 Subject: [PATCH 10/13] Fixup --- .../xpack/esql/qa/rest/RestEsqlTestCase.java | 15 +++++++++------ .../xpack/esql/action/EsqlQueryResponseTests.java | 4 ++-- 2 files changed, 11 insertions(+), 8 deletions(-) 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 cb165906cee7e..93c1afc37743a 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 @@ -271,12 +271,15 @@ public void testGetAnswer() throws IOException { assertThat(((Integer) answer.get("took")).intValue(), greaterThanOrEqualTo(0)); Map colA = Map.of("name", "a", "type", "integer"); Map colB = Map.of("name", "b", "type", "integer"); - assertMap(answer, matchesMap().entry("took", greaterThanOrEqualTo(0)) - .entry("is_partial", any(Boolean.class)) - .entry("documents_found", 0) - .entry("values_loaded", 0) - .entry("columns", List.of(colA, colB)) - .entry("values", List.of(List.of(1, 2)))); + assertMap( + answer, + matchesMap().entry("took", greaterThanOrEqualTo(0)) + .entry("is_partial", any(Boolean.class)) + .entry("documents_found", 0) + .entry("values_loaded", 0) + .entry("columns", List.of(colA, colB)) + .entry("values", List.of(List.of(1, 2))) + ); } public void testUseUnknownIndex() throws IOException { 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 9305129dea957..ff40c4a9e10b8 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 @@ -783,8 +783,8 @@ public void testXContentOriginalTypes() { ) { assertThat(Strings.toString(wrapAsToXContent(response), true, false), equalTo(""" { - "documents_found": 1, - "values_loaded": 1, + "documents_found" : 1, + "values_loaded" : 1, "columns" : [ { "name" : "foo", From 28e9fa44772cd7a30913b1e9a8c0ec3d292676a0 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 1 Apr 2025 14:28:21 -0400 Subject: [PATCH 11/13] Rename --- .../org/elasticsearch/TransportVersions.java | 2 +- .../lucene/ValuesSourceReaderOperator.java | 10 +++------- .../compute/operator/DriverCompletionInfo.java | 5 ++++- .../compute/operator/DriverStatus.java | 18 ++++++++++++++---- .../xpack/esql/action/EsqlCapabilities.java | 2 +- .../xpack/esql/action/EsqlQueryResponse.java | 8 ++++---- .../xpack/esql/plugin/ComputeResponse.java | 6 +++--- .../esql/plugin/DataNodeComputeResponse.java | 6 +++--- .../rest-api-spec/test/esql/10_basic.yml | 4 ++-- .../rest-api-spec/test/esql/120_profile.yml | 2 +- 10 files changed, 36 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 53261b36e5247..bdf904c353358 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -209,7 +209,7 @@ static TransportVersion def(int id) { public static final TransportVersion INDEX_STATS_AND_METADATA_INCLUDE_PEAK_WRITE_LOAD = def(9_041_0_00); public static final TransportVersion REPOSITORIES_METADATA_AS_PROJECT_CUSTOM = def(9_042_0_00); public static final TransportVersion BATCHED_QUERY_PHASE_VERSION = def(9_043_0_00); - public static final TransportVersion ESQL_VALUES_LOADED = def(9_044_0_00); + public static final TransportVersion ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED = def(9_044_0_00); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java index 1eb50e83d8c68..f8ebeefb74106 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -47,7 +47,7 @@ import java.util.function.IntFunction; import java.util.function.Supplier; -import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; +import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED; /** * Operator that extracts doc_values from a Lucene index out of pages that have been produced by {@link LuceneSourceOperator} @@ -582,18 +582,14 @@ public static class Status extends AbstractPageMappingOperator.Status { Status(StreamInput in) throws IOException { super(in); readersBuilt = in.readOrderedMap(StreamInput::readString, StreamInput::readVInt); - if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { - valuesLoaded = in.readVLong(); - } else { - valuesLoaded = 0; - } + valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED) ? in.readVLong() : 0; } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeMap(readersBuilt, StreamOutput::writeVInt); - if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { out.writeVLong(valuesLoaded); } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java index 1e928945da741..bafcc076a46e8 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverCompletionInfo.java @@ -20,7 +20,10 @@ /** * Information returned when one of more {@link Driver}s is completed. * @param documentsFound The number of documents found by all lucene queries performed by these drivers. - * @param valuesLoaded The number of values loaded from lucene for all drivers. + * @param valuesLoaded The number of values loaded from lucene for all drivers. This is + * roughly the number of documents times the number of + * fields per document. Except {@code null} values don't count. + * And multivalued fields count as many times as there are values. * @param collectedProfiles {@link DriverProfile}s from each driver. These are fairly cheap to build but * not free so this will be empty if the {@code profile} option was not set in * the request. diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java index c0351f27ecb2c..c96ac2c4c80b8 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java @@ -151,8 +151,13 @@ public String toString() { * The number of documents found by this driver. */ public long documentsFound() { - long documentsFound = completedOperators.stream().mapToLong(OperatorStatus::documentsFound).sum(); - documentsFound += activeOperators.stream().mapToLong(OperatorStatus::documentsFound).sum(); + long documentsFound = 0; + for (OperatorStatus s : completedOperators) { + documentsFound += s.documentsFound(); + } + for (OperatorStatus s : activeOperators) { + documentsFound += s.documentsFound(); + } return documentsFound; } @@ -160,8 +165,13 @@ public long documentsFound() { * The number of values loaded by this operator. */ public long valuesLoaded() { - long valuesLoaded = completedOperators.stream().mapToLong(OperatorStatus::valuesLoaded).sum(); - valuesLoaded += activeOperators.stream().mapToLong(OperatorStatus::valuesLoaded).sum(); + long valuesLoaded = 0; + for (OperatorStatus s : completedOperators) { + valuesLoaded += s.valuesLoaded(); + } + for (OperatorStatus s : activeOperators) { + valuesLoaded += s.valuesLoaded(); + } return valuesLoaded; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 794a87b4da1f2..1a59f3841df2b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -933,7 +933,7 @@ public enum Cap { * Are the {@code documents_found} and {@code values_loaded} fields available * in the response and profile? */ - DOCUMENTS_FOUND, + DOCUMENTS_FOUND_AND_VALUES_LOADED, /** * Index component selector syntax (my-data-stream-name::failures) 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 a7a86da2f6d8d..08101d5d78262 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 @@ -35,7 +35,7 @@ import java.util.Objects; import java.util.Optional; -import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; +import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED; public class EsqlQueryResponse extends org.elasticsearch.xpack.core.esql.action.EsqlQueryResponse implements @@ -119,8 +119,8 @@ static EsqlQueryResponse deserialize(BlockStreamInput in) throws IOException { } List columns = in.readCollectionAsList(ColumnInfoImpl::new); List pages = in.readCollectionAsList(Page::new); - long documentsFound = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVLong() : 0; - long valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED) ? in.readVLong() : 0; + long documentsFound = in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED) ? in.readVLong() : 0; + long valuesLoaded = in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED) ? in.readVLong() : 0; if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { profile = in.readOptionalWriteable(Profile::new); } @@ -152,7 +152,7 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeCollection(columns); out.writeCollection(pages); - if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { out.writeVLong(documentsFound); out.writeVLong(valuesLoaded); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java index 715da6ef0079e..1a9b211d7a487 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.List; -import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; +import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED; /** * The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest} @@ -58,7 +58,7 @@ final class ComputeResponse extends TransportResponse { } ComputeResponse(StreamInput in) throws IOException { - if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { completionInfo = new DriverCompletionInfo(in); } else if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { if (in.readBoolean()) { @@ -92,7 +92,7 @@ final class ComputeResponse extends TransportResponse { @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { completionInfo.writeTo(out); } else if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { out.writeBoolean(true); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java index 1c4d06b9372fe..4de26e4034d81 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java @@ -17,7 +17,7 @@ import java.io.IOException; import java.util.Map; -import static org.elasticsearch.TransportVersions.ESQL_VALUES_LOADED; +import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED; /** * The compute result of {@link DataNodeRequest} @@ -32,7 +32,7 @@ final class DataNodeComputeResponse extends TransportResponse { } DataNodeComputeResponse(StreamInput in) throws IOException { - if (in.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { this.completionInfo = new DriverCompletionInfo(in); this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); return; @@ -48,7 +48,7 @@ final class DataNodeComputeResponse extends TransportResponse { @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getTransportVersion().onOrAfter(ESQL_VALUES_LOADED)) { + if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) { completionInfo.writeTo(out); out.writeMap(shardLevelFailures, (o, v) -> v.writeTo(o), StreamOutput::writeException); return; diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml index 64cc90df4c358..b7fe3ae355ccf 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml @@ -188,7 +188,7 @@ basic with documents_found: - method: POST path: /_query parameters: [] - capabilities: [documents_found] + capabilities: [documents_found_and_values_loaded] reason: "checks for documents_found and values_loaded" - do: @@ -228,7 +228,7 @@ FROM EVAL SORT LIMIT with documents_found: - method: POST path: /_query parameters: [] - capabilities: [documents_found] + capabilities: [documents_found_and_values_loaded] reason: "checks for documents_found and values_loaded" - do: diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml index 8a90915caa8c6..a7b0c48b93ebe 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/120_profile.yml @@ -145,7 +145,7 @@ documents found: - method: POST path: /_query parameters: [] - capabilities: [documents_found] + capabilities: [documents_found_and_values_loaded] reason: "checks for documents_found and values_loaded" - do: From fbbe29cb82d7ae0d6c86eca0b5947b240d197735 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 15 Apr 2025 16:16:31 -0400 Subject: [PATCH 12/13] Update --- .../org/elasticsearch/test/rest/ESRestTestCase.java | 11 +++++++---- .../elasticsearch/xpack/esql/ccq/MultiClustersIT.java | 6 +++++- 2 files changed, 12 insertions(+), 5 deletions(-) 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 506398d4a5302..aaa1a92ee16a0 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 @@ -2672,10 +2672,13 @@ protected static MapMatcher getProfileMatcher() { .entry("drivers", instanceOf(List.class)); } - protected static MapMatcher getResultMatcher(boolean includeMetadata, boolean includePartial) { + protected static MapMatcher getResultMatcher(boolean includeMetadata, boolean includePartial, boolean includeDocumentsFound) { MapMatcher mapMatcher = matchesMap(); - mapMatcher = mapMatcher.entry("documents_found", greaterThanOrEqualTo(0)); - mapMatcher = mapMatcher.entry("values_loaded", greaterThanOrEqualTo(0)); + if (includeDocumentsFound) { + // Older versions may not return documents_found and values_loaded. + mapMatcher = mapMatcher.entry("documents_found", greaterThanOrEqualTo(0)); + mapMatcher = mapMatcher.entry("values_loaded", greaterThanOrEqualTo(0)); + } if (includeMetadata) { mapMatcher = mapMatcher.entry("took", greaterThanOrEqualTo(0)); } @@ -2690,7 +2693,7 @@ protected static MapMatcher getResultMatcher(boolean includeMetadata, boolean in * 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")); + return getResultMatcher(result.containsKey("took"), result.containsKey("is_partial"), result.containsKey("documents_found")); } /** 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 791f5dacdce64..4ea413e4fcd3b 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 @@ -159,7 +159,11 @@ 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")); + MapMatcher mapMatcher = getResultMatcher( + ccsMetadataAvailable(), + result.containsKey("is_partial"), + result.containsKey("documents_found") + ); if (includeCCSMetadata) { mapMatcher = mapMatcher.entry("_clusters", any(Map.class)); } From 4e3136a4abeafa70d13f71fdfa5577eceabdaf4b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Apr 2025 10:08:43 -0400 Subject: [PATCH 13/13] Shift assertion --- .../resources/rest-api-spec/test/esql/10_basic.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml index b7fe3ae355ccf..99beec034b0f7 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml @@ -237,7 +237,10 @@ FROM EVAL SORT LIMIT with documents_found: query: 'from test | eval x = count + 7 | sort x | limit 1' - match: {documents_found: 40} - - match: {values_loaded: 65} + # We can't be sure quite how many values we'll load. It's at least + # one per document in the index. And one per top document. But we + # might load more values because we run in more threads. + - gte: {values_loaded: 45} - match: {columns.0.name: "color"} - match: {columns.1.name: "count"} - match: {columns.2.name: "count_d"}