From 19b95b4f2166d80c010dd01c3df0d309d24958d4 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Thu, 13 Mar 2025 13:51:36 -0400 Subject: [PATCH 01/20] top level plumbing; work in progress --- .../org/elasticsearch/TransportVersions.java | 1 + .../xpack/esql/action/EsqlQueryResponse.java | 11 +++- .../xpack/esql/planner/PlannerProfile.java | 50 +++++++++++++++++++ .../xpack/esql/plugin/ComputeService.java | 4 +- .../esql/plugin/TransportEsqlQueryAction.java | 4 +- .../xpack/esql/session/EsqlCCSUtils.java | 5 +- .../xpack/esql/session/EsqlSession.java | 8 ++- .../xpack/esql/session/Result.java | 28 +++++++---- .../elasticsearch/xpack/esql/CsvTests.java | 2 +- 9 files changed, 97 insertions(+), 16 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 350e93698e607..42184af7d8480 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -180,6 +180,7 @@ static TransportVersion def(int id) { public static final TransportVersion MAX_OPERATION_SIZE_REJECTIONS_ADDED = def(9_024_0_00); public static final TransportVersion RETRY_ILM_ASYNC_ACTION_REQUIRE_ERROR = def(9_025_0_00); public static final TransportVersion ESQL_SERIALIZE_BLOCK_TYPE_CODE = def(9_026_0_00); + public static final TransportVersion ESQL_PLANNER_PROFILE = def(9_027_0_00); /* * STOP! READ THIS FIRST! No, really, 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 7c5e2538e2e2c..dca1992c4eb5e 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 @@ -27,6 +27,7 @@ import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xpack.core.esql.action.EsqlResponse; import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.io.IOException; import java.util.Collections; @@ -348,13 +349,21 @@ public EsqlResponse responseInternal() { public static class Profile implements Writeable, ChunkedToXContentObject { private final List drivers; + private final PlannerProfile plannerProfile; - public Profile(List drivers) { + public Profile(List drivers, PlannerProfile plannerProfile) { this.drivers = drivers; + this.plannerProfile = plannerProfile; } public Profile(StreamInput in) throws IOException { this.drivers = in.readCollectionAsImmutableList(DriverProfile::readFrom); + if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + this.plannerProfile = in.readNamedWriteable(PlannerProfile.class); + } else { + this.plannerProfile = PlannerProfile.EMPTY; + } + } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java new file mode 100644 index 0000000000000..90d94fd4ae7af --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -0,0 +1,50 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner; + +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ChunkedToXContentObject; +import org.elasticsearch.xcontent.ToXContent; + +import java.io.IOException; +import java.util.Iterator; + +public class PlannerProfile implements NamedWriteable, ChunkedToXContentObject { + + public static final PlannerProfile EMPTY = new PlannerProfile(); + + public PlannerProfile() { + // NOCOMMIT + throw new UnsupportedOperationException(); + } + + public PlannerProfile(StreamInput in) throws IOException { + // NOCOMMIT + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + // NOCOMMIT + throw new UnsupportedOperationException(); + } + + @Override + public Iterator toXContentChunked(ToXContent.Params params) { + // NOCOMMIT + throw new UnsupportedOperationException(); + } + + @Override + public String getWriteableName() { + // NOCOMMIT + throw new UnsupportedOperationException(); + } +} 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 8ce0b36cdc391..52c0f5c42817b 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 @@ -172,7 +172,7 @@ public void execute( try ( var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result(physicalPlan.output(), collectedPages, profiles, execInfo); + return new Result(physicalPlan.output(), collectedPages, profiles, , execInfo); })) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); @@ -204,7 +204,7 @@ public void execute( 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); + return new Result(outputAttributes, collectedPages, profiles, , execInfo); }))) { try (Releasable ignored = exchangeSource.addEmptySink()) { // run compute on the coordinator 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..973b7e5f20a68 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.profiles(), result.plannerProfile()) + : null; threadPool.getThreadContext().addResponseHeader(AsyncExecutionId.ASYNC_EXECUTION_IS_RUNNING_HEADER, "?0"); if (task instanceof EsqlQueryTask asyncTask && request.keepOnCompletion()) { String asyncExecutionId = asyncTask.getExecutionId().getEncoded(); 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 89cd4b3d4d7cd..c07ba002524ef 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 @@ -31,6 +31,7 @@ import org.elasticsearch.xpack.esql.analysis.TableInfo; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.Collections; import java.util.HashMap; @@ -76,7 +77,9 @@ abstract static class CssPartialErrorsActionListener implements ActionListener { profileAccumulator.addAll(finalResult.profiles()); - finalListener.onResponse(new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, executionInfo)); + finalListener.onResponse( + new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, plannerProfile, executionInfo) + ); })); } else { // continue executing the subplans 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..57a84c4f6cf39 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 @@ -14,19 +14,29 @@ import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.List; /** * Results from running a chunk of ESQL. - * @param schema "Schema" of the {@link Attribute}s that are produced by the {@link LogicalPlan} - * 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 schema "Schema" of the {@link Attribute}s that are produced by the {@link LogicalPlan} + * 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 plannerProfile {@link PlannerProfile} for this query. Like the driver profile, this is only + * included if the user asks for it. * @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, + List profiles, + PlannerProfile plannerProfile, + @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 1347b8430f3a4..11cfd7343a6da 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 @@ -692,6 +692,6 @@ 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, List.of(), , null))); } } From 1e795fc8d47f48de2c9e55eb93c2a7e4688e2d6f Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Fri, 14 Mar 2025 11:22:18 -0400 Subject: [PATCH 02/20] collected profiles object. WIP, this still doesn't compile --- .../xpack/esql/planner/PlannerProfile.java | 2 + .../xpack/esql/plugin/ComputeListener.java | 37 ++++++++++++++++--- .../xpack/esql/plugin/ComputeService.java | 2 +- .../esql/plugin/TransportEsqlQueryAction.java | 2 +- .../xpack/esql/session/EsqlSession.java | 5 ++- .../xpack/esql/session/Result.java | 24 ++++++------ 6 files changed, 52 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 90d94fd4ae7af..0e50d45c1973b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -20,6 +20,8 @@ public class PlannerProfile implements NamedWriteable, ChunkedToXContentObject { public static final PlannerProfile EMPTY = new PlannerProfile(); + private final boolean isLocalPlanning; + public PlannerProfile() { // NOCOMMIT throw new UnsupportedOperationException(); 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..d9b8543788dce 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 @@ -14,6 +14,7 @@ import org.elasticsearch.compute.operator.ResponseHeadersCollector; import org.elasticsearch.core.Releasable; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.ArrayList; import java.util.Collections; @@ -29,17 +30,38 @@ final class ComputeListener implements Releasable { private final EsqlRefCountingListener refs; private final List collectedProfiles; + private final List collectedPlannerProfiles; private final ResponseHeadersCollector responseHeaders; private final Runnable runOnFailure; - ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener> delegate) { + public static class CollectedProfiles { + private List driverProfiles; + private List plannerProfiles; + + CollectedProfiles(List driverProfiles, List plannerProfiles) { + this.driverProfiles = driverProfiles; + this.plannerProfiles = plannerProfiles; + } + + public List getDriverProfiles() { + return driverProfiles; + } + + public List getPlannerProfiles() { + return plannerProfiles; + } + + } + + ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener delegate) { this.runOnFailure = runOnFailure; this.responseHeaders = new ResponseHeadersCollector(threadPool.getThreadContext()); this.collectedProfiles = Collections.synchronizedList(new ArrayList<>()); + this.collectedPlannerProfiles = 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(new CollectedProfiles(collectedProfiles.stream().toList(), collectedPlannerProfiles.stream().toList())); })); } @@ -60,12 +82,17 @@ 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 -> { responseHeaders.collect(); - if (profiles != null && profiles.isEmpty() == false) { - collectedProfiles.addAll(profiles); + if (profiles != null) { + if (profiles.getDriverProfiles().isEmpty() == false) { + collectedProfiles.addAll(profiles.getDriverProfiles()); + } + if (profiles.getPlannerProfiles().isEmpty() == false) { + collectedPlannerProfiles.addAll(profiles.getPlannerProfiles()); + } } delegate.onResponse(null); }, e -> { 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 52c0f5c42817b..3fabc166ff238 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 @@ -172,7 +172,7 @@ public void execute( try ( var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result(physicalPlan.output(), collectedPages, profiles, , execInfo); + return new Result(physicalPlan.output(), collectedPages, profiles.getDriverProfiles(), profiles.getPlannerProfiles(), execInfo); })) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); 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 973b7e5f20a68..41d4b76c35fee 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 @@ -316,7 +316,7 @@ 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(), result.plannerProfile()) + ? new EsqlQueryResponse.Profile(result.profiles(), result.plannerProfiles()) : null; threadPool.getThreadContext().addResponseHeader(AsyncExecutionId.ASYNC_EXECUTION_IS_RUNNING_HEADER, "?0"); if (task instanceof EsqlQueryTask asyncTask && request.keepOnCompletion()) { 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 6d4158cf84da3..30eeb87135f06 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 @@ -260,6 +260,8 @@ private void executeSubPlan( ActionListener listener ) { PlanTuple tuple = subPlanIterator.next(); + List plannerProfiles = new ArrayList<>(); + plannerProfiles.add(plannerProfile); runner.run(tuple.physical, listener.delegateFailureAndWrap((next, result) -> { try { @@ -301,8 +303,9 @@ private void executeSubPlan( // In the production path, this is runner.run calls ComputeService.execute runner.run(newPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { profileAccumulator.addAll(finalResult.profiles()); + plannerProfiles.addAll(finalResult.plannerProfiles()); finalListener.onResponse( - new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, plannerProfile, executionInfo) + new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, plannerProfiles, executionInfo) ); })); } else { 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 57a84c4f6cf39..70acb963e2e0f 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 @@ -21,22 +21,22 @@ /** * Results from running a chunk of ESQL. * - * @param schema "Schema" of the {@link Attribute}s that are produced by the {@link LogicalPlan} - * 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 plannerProfile {@link PlannerProfile} for this query. Like the driver profile, this is only - * included if the user asks for it. - * @param executionInfo Metadata about the execution of this query. Used for cross cluster queries. + * @param schema "Schema" of the {@link Attribute}s that are produced by the {@link LogicalPlan} + * 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 plannerProfiles {@link PlannerProfile} for this query. Like the driver profile, this is only + * included if the user asks for it. + * @param executionInfo Metadata about the execution of this query. Used for cross cluster queries. */ public record Result( List schema, List pages, List profiles, - PlannerProfile plannerProfile, + List plannerProfiles, @Nullable EsqlExecutionInfo executionInfo ) {} From fa82be2ee952deb58e5b444116cf333b2b16dafb Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Fri, 14 Mar 2025 13:35:54 -0400 Subject: [PATCH 03/20] make collected profiles writable. Still WIP --- .../xpack/esql/action/EsqlQueryResponse.java | 2 +- .../xpack/esql/planner/PlannerProfile.java | 9 ++---- .../xpack/esql/plugin/ComputeListener.java | 18 ++++++++++-- .../xpack/esql/plugin/ComputeResponse.java | 20 +++++++++---- .../xpack/esql/plugin/ComputeService.java | 2 +- .../esql/plugin/DataNodeComputeHandler.java | 8 ++--- .../esql/plugin/DataNodeComputeResponse.java | 29 +++++++++++++++---- .../esql/plugin/DataNodeRequestSender.java | 6 ++-- 8 files changed, 64 insertions(+), 30 deletions(-) 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 dca1992c4eb5e..a852798c568b8 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 @@ -359,7 +359,7 @@ public Profile(List drivers, PlannerProfile plannerProfile) { public Profile(StreamInput in) throws IOException { this.drivers = in.readCollectionAsImmutableList(DriverProfile::readFrom); if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.plannerProfile = in.readNamedWriteable(PlannerProfile.class); + this.plannerProfile = new PlannerProfile(in); } else { this.plannerProfile = PlannerProfile.EMPTY; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 0e50d45c1973b..3cdaf55f96f5e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -7,16 +7,16 @@ package org.elasticsearch.xpack.esql.planner; -import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ChunkedToXContentObject; import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; import java.util.Iterator; -public class PlannerProfile implements NamedWriteable, ChunkedToXContentObject { +public class PlannerProfile implements Writeable, ChunkedToXContentObject { public static final PlannerProfile EMPTY = new PlannerProfile(); @@ -44,9 +44,4 @@ public Iterator toXContentChunked(ToXContent.Params params throw new UnsupportedOperationException(); } - @Override - public String getWriteableName() { - // NOCOMMIT - throw new UnsupportedOperationException(); - } } 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 d9b8543788dce..a82e53eedaed4 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 @@ -9,6 +9,9 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.ResponseHeadersCollector; @@ -16,6 +19,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.esql.planner.PlannerProfile; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -34,7 +38,9 @@ final class ComputeListener implements Releasable { private final ResponseHeadersCollector responseHeaders; private final Runnable runOnFailure; - public static class CollectedProfiles { + public static class CollectedProfiles implements Writeable { + public static final CollectedProfiles EMPTY = new CollectedProfiles(List.of(), List.of()); + private List driverProfiles; private List plannerProfiles; @@ -43,6 +49,15 @@ public static class CollectedProfiles { this.plannerProfiles = plannerProfiles; } + CollectedProfiles(StreamInput in) throws IOException { + + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + + } + public List getDriverProfiles() { return driverProfiles; } @@ -50,7 +65,6 @@ public List getDriverProfiles() { public List getPlannerProfiles() { return plannerProfiles; } - } ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener delegate) { 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 72d5187bfd7cd..744bfae62890c 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 @@ -21,7 +21,7 @@ * The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest} */ final class ComputeResponse extends TransportResponse { - private final List profiles; + private final ComputeListener.CollectedProfiles profiles; // for use with ClusterComputeRequests (cross-cluster searches) private final TimeValue took; // overall took time for a specific cluster in a cross-cluster search @@ -30,12 +30,12 @@ final class ComputeResponse extends TransportResponse { public final int skippedShards; public final int failedShards; - ComputeResponse(List profiles) { + ComputeResponse(ComputeListener.CollectedProfiles profiles) { this(profiles, null, null, null, null, null); } ComputeResponse( - List profiles, + ComputeListener.CollectedProfiles profiles, TimeValue took, Integer totalShards, Integer successfulShards, @@ -54,7 +54,11 @@ final class ComputeResponse extends TransportResponse { super(in); if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { if (in.readBoolean()) { - profiles = in.readCollectionAsImmutableList(DriverProfile::readFrom); + if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + profiles = new ComputeListener.CollectedProfiles(in); + } else { + profiles = new ComputeListener.CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + } } else { profiles = null; } @@ -83,7 +87,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(false); } else { out.writeBoolean(true); - out.writeCollection(profiles); + if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + profiles.writeTo(out); + } else { + out.writeCollection(profiles.getDriverProfiles()); + } } } if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) { @@ -95,7 +103,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - public List getProfiles() { + public ComputeListener.CollectedProfiles getProfiles() { return profiles; } 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 3fabc166ff238..d5ded9ec57571 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 @@ -339,7 +339,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++) { 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 464aaca686249..ba3d09db051be 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 @@ -238,11 +238,11 @@ 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(ComputeListener.CollectedProfiles result) { try { onBatchCompleted(endBatchIndex); } finally { @@ -256,7 +256,7 @@ public void onFailure(Exception e) { for (ShardId shardId : shardIds) { addShardLevelFailure(shardId, e); } - onResponse(List.of()); + onResponse(ComputeListener.CollectedProfiles.EMPTY); } else { // TODO: add these to fatal failures so we can continue processing other shards. try { 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..bfaf0b92f887e 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 @@ -7,11 +7,13 @@ package org.elasticsearch.xpack.esql.plugin; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.io.IOException; import java.util.List; @@ -22,20 +24,31 @@ * The compute result of {@link DataNodeRequest} */ final class DataNodeComputeResponse extends TransportResponse { - private final List profiles; + private final ComputeListener.CollectedProfiles profiles; private final Map shardLevelFailures; - DataNodeComputeResponse(List profiles, Map shardLevelFailures) { + DataNodeComputeResponse(ComputeListener.CollectedProfiles profiles, Map shardLevelFailures) { this.profiles = profiles; this.shardLevelFailures = shardLevelFailures; } DataNodeComputeResponse(StreamInput in) throws IOException { if (DataNodeComputeHandler.supportShardLevelRetryFailure(in.getTransportVersion())) { - this.profiles = in.readCollectionAsImmutableList(DriverProfile::readFrom); + if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + this.profiles = new ComputeListener.CollectedProfiles(in); + } else { + this.profiles = new ComputeListener.CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + } this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); } else { - this.profiles = Objects.requireNonNullElse(new ComputeResponse(in).getProfiles(), List.of()); + if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + this.profiles = new ComputeListener.CollectedProfiles(in); + } else { + this.profiles = new ComputeListener.CollectedProfiles( + Objects.requireNonNullElse(new ComputeResponse(in).getProfiles().getDriverProfiles(), List.of()), + List.of() + ); + } this.shardLevelFailures = Map.of(); } } @@ -43,7 +56,11 @@ final class DataNodeComputeResponse extends TransportResponse { @Override public void writeTo(StreamOutput out) throws IOException { if (DataNodeComputeHandler.supportShardLevelRetryFailure(out.getTransportVersion())) { - out.writeCollection(profiles, (o, v) -> v.writeTo(o)); + if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + profiles.writeTo(out); + } else { + out.writeCollection(profiles.getDriverProfiles(), (o, v) -> v.writeTo(o)); + } out.writeMap(shardLevelFailures, (o, v) -> v.writeTo(o), StreamOutput::writeException); } else { if (shardLevelFailures.isEmpty() == false) { @@ -53,7 +70,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - List profiles() { + ComputeListener.CollectedProfiles profiles() { return profiles; } 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 310d9906dc4b7..ea96c1577b7e6 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 @@ -155,9 +155,9 @@ private void reportFailures(ComputeListener computeListener) { } 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(ComputeListener.CollectedProfiles profiles) { nodePermits.get(request.node).release(); trySendingRequestsForPendingShards(targetShards, computeListener); listener.onResponse(profiles); @@ -185,7 +185,7 @@ public void onFailure(Exception e, boolean receivedData) { trackShardLevelFailure(shardId, receivedData, e); pendingShardIds.add(shardId); } - onAfter(List.of()); + onAfter(ComputeListener.CollectedProfiles.EMPTY); } }); } From 838cf28622fec6d81ab083f340ca7a036ed23083 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Fri, 14 Mar 2025 13:54:49 -0400 Subject: [PATCH 04/20] clean up a few more usages --- .../xpack/esql/planner/PlannerProfile.java | 4 ++++ .../esql/plugin/ClusterComputeHandler.java | 7 +++---- .../xpack/esql/plugin/ComputeListener.java | 3 +++ .../xpack/esql/plugin/ComputeService.java | 20 ++++++++++++++----- .../esql/plugin/DataNodeComputeHandler.java | 3 +-- .../esql/plugin/ComputeListenerTests.java | 14 +++++++------ 6 files changed, 34 insertions(+), 17 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 3cdaf55f96f5e..fd2ddeed63b4c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -16,6 +16,10 @@ import java.io.IOException; import java.util.Iterator; +/** + * Stores profiling information about the query plan. This can be the top level planning on the coordinating node, or the local + * planning on the data nodes. + */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { public static final PlannerProfile EMPTY = new PlannerProfile(); 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 d6395fdefd7e2..9f0978b1164a6 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,7 +11,6 @@ import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.ChannelActionListener; -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; @@ -75,7 +74,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 +86,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(ComputeListener.CollectedProfiles.EMPTY); } else if (configuration.allowPartialResults()) { EsqlCCSUtils.markClusterWithFinalStateAndNoShards(executionInfo, clusterAlias, EsqlExecutionInfo.Cluster.Status.PARTIAL, e); - l.onResponse(List.of()); + l.onResponse(ComputeListener.CollectedProfiles.EMPTY); } else { l.onFailure(e); } 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 a82e53eedaed4..03db158b7ae13 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 @@ -38,6 +38,9 @@ final class ComputeListener implements Releasable { private final ResponseHeadersCollector responseHeaders; private final Runnable runOnFailure; + /** + * Holds the collection of driver profiles and query planning profiles from the individual nodes processing the query. + */ public static class CollectedProfiles implements Writeable { public static final CollectedProfiles EMPTY = new CollectedProfiles(List.of(), List.of()); 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 d5ded9ec57571..594dea0d49f8e 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,7 +17,6 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Driver; -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; @@ -172,7 +171,13 @@ public void execute( try ( var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result(physicalPlan.output(), collectedPages, profiles.getDriverProfiles(), profiles.getPlannerProfiles(), execInfo); + return new Result( + physicalPlan.output(), + collectedPages, + profiles.getDriverProfiles(), + profiles.getPlannerProfiles(), + execInfo + ); })) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); @@ -204,7 +209,7 @@ public void execute( 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); + return new Result(outputAttributes, collectedPages, profiles.getDriverProfiles(), profiles.getPlannerProfiles(), execInfo); }))) { try (Releasable ignored = exchangeSource.addEmptySink()) { // run compute on the coordinator @@ -279,7 +284,7 @@ public void execute( EsqlExecutionInfo.Cluster.Status.PARTIAL ).setFailures(List.of(new ShardSearchFailure(e))).build() ); - dataNodesListener.onResponse(List.of()); + dataNodesListener.onResponse(ComputeListener.CollectedProfiles.EMPTY); } else { dataNodesListener.onFailure(e); } @@ -339,7 +344,12 @@ 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++) { 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 ba3d09db051be..3a61d3edcc6d3 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,6 @@ 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.exchange.ExchangeService; import org.elasticsearch.compute.operator.exchange.ExchangeSink; import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; @@ -270,7 +269,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(ComputeListener.CollectedProfiles.EMPTY); return; } var computeContext = new ComputeContext( 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..94f677ead32db 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 @@ -56,7 +56,7 @@ public void shutdownTransportService() { terminate(threadPool); } - private List randomProfiles() { + private ComputeListener.CollectedProfiles randomProfiles() { int numProfiles = randomIntBetween(0, 2); List profiles = new ArrayList<>(numProfiles); for (int i = 0; i < numProfiles; i++) { @@ -75,20 +75,22 @@ private List randomProfiles() { ) ); } - return profiles; + // NOCOMMIT - add random planner profiles? + return new ComputeListener.CollectedProfiles(profiles, List.of()); } 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).getDriverProfiles(), empty()); + assertThat(results.actionGet(10, TimeUnit.SECONDS).getPlannerProfiles(), empty()); } public void testCollectComputeResults() { - PlainActionFuture> future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); List allProfiles = new ArrayList<>(); AtomicInteger onFailure = new AtomicInteger(); try (var computeListener = new ComputeListener(threadPool, onFailure::incrementAndGet, future)) { @@ -104,7 +106,7 @@ public void testCollectComputeResults() { } else { var profiles = randomProfiles(); allProfiles.addAll(profiles); - ActionListener> subListener = computeListener.acquireCompute(); + ActionListener subListener = computeListener.acquireCompute(); threadPool.schedule( ActionRunnable.wrap(subListener, l -> l.onResponse(profiles)), TimeValue.timeValueNanos(between(0, 100)), From bd1567cee3144a0bd0e1b1ee24a25955187d8697 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Fri, 14 Mar 2025 15:52:11 -0400 Subject: [PATCH 05/20] Everything Compiles! --- .../xpack/esql/action/EsqlQueryResponse.java | 14 ++++-- .../xpack/esql/planner/PlannerProfile.java | 4 +- .../esql/plugin/ClusterComputeHandler.java | 6 +-- .../xpack/esql/plugin/CollectedProfiles.java | 49 +++++++++++++++++++ .../xpack/esql/plugin/ComputeListener.java | 36 -------------- .../xpack/esql/plugin/ComputeResponse.java | 12 ++--- .../xpack/esql/plugin/ComputeService.java | 10 ++-- .../esql/plugin/DataNodeComputeHandler.java | 10 ++-- .../esql/plugin/DataNodeComputeResponse.java | 15 +++--- .../esql/plugin/DataNodeRequestSender.java | 7 ++- .../xpack/esql/session/EsqlCCSUtils.java | 2 +- .../elasticsearch/xpack/esql/CsvTests.java | 5 +- .../action/EsqlQueryResponseProfileTests.java | 11 +++-- .../esql/action/EsqlQueryResponseTests.java | 4 +- .../esql/plugin/ComputeListenerTests.java | 30 ++++++------ .../plugin/DataNodeRequestSenderTests.java | 14 +++--- 16 files changed, 130 insertions(+), 99 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java 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 a852798c568b8..14196a893ac95 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 @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.core.esql.action.EsqlResponse; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.planner.PlannerProfile; +import org.elasticsearch.xpack.esql.plugin.CollectedProfiles; import java.io.IOException; import java.util.Collections; @@ -349,19 +350,24 @@ public EsqlResponse responseInternal() { public static class Profile implements Writeable, ChunkedToXContentObject { private final List drivers; - private final PlannerProfile plannerProfile; + private final List plannerProfile; - public Profile(List drivers, PlannerProfile plannerProfile) { + public Profile(List drivers, List plannerProfile) { this.drivers = drivers; this.plannerProfile = plannerProfile; } + public Profile(CollectedProfiles profiles) { + this.drivers = profiles.getDriverProfiles(); + this.plannerProfile = profiles.getPlannerProfiles(); + } + public Profile(StreamInput in) throws IOException { this.drivers = in.readCollectionAsImmutableList(DriverProfile::readFrom); if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.plannerProfile = new PlannerProfile(in); + this.plannerProfile = in.readCollectionAsImmutableList(PlannerProfile::readFrom); } else { - this.plannerProfile = PlannerProfile.EMPTY; + this.plannerProfile = List.of(); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index fd2ddeed63b4c..a66a4d41350f9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -26,12 +26,12 @@ public class PlannerProfile implements Writeable, ChunkedToXContentObject { private final boolean isLocalPlanning; - public PlannerProfile() { + public static PlannerProfile readFrom(StreamInput in) throws IOException { // NOCOMMIT throw new UnsupportedOperationException(); } - public PlannerProfile(StreamInput in) throws IOException { + public PlannerProfile() { // NOCOMMIT throw new UnsupportedOperationException(); } 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 9f0978b1164a6..946719add1b04 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 @@ -74,7 +74,7 @@ void startComputeOnRemoteCluster( RemoteCluster cluster, Runnable cancelQueryOnFailure, EsqlExecutionInfo executionInfo, - ActionListener listener + ActionListener listener ) { var queryPragmas = configuration.pragmas(); listener = ActionListener.runBefore(listener, exchangeSource.addEmptySink()::close); @@ -86,10 +86,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(ComputeListener.CollectedProfiles.EMPTY); + l.onResponse(CollectedProfiles.EMPTY); } else if (configuration.allowPartialResults()) { EsqlCCSUtils.markClusterWithFinalStateAndNoShards(executionInfo, clusterAlias, EsqlExecutionInfo.Cluster.Status.PARTIAL, e); - l.onResponse(ComputeListener.CollectedProfiles.EMPTY); + l.onResponse(CollectedProfiles.EMPTY); } else { l.onFailure(e); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java new file mode 100644 index 0000000000000..38e1e13db91b3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java @@ -0,0 +1,49 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plugin; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.compute.operator.DriverProfile; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; + +import java.io.IOException; +import java.util.List; + +/** + * Holds the collection of driver profiles and query planning profiles from the individual nodes processing the query. + */ +public class CollectedProfiles implements Writeable { + public static final CollectedProfiles EMPTY = new CollectedProfiles(List.of(), List.of()); + + private List driverProfiles; + private List plannerProfiles; + + public CollectedProfiles(List driverProfiles, List plannerProfiles) { + this.driverProfiles = driverProfiles; + this.plannerProfiles = plannerProfiles; + } + + public CollectedProfiles(StreamInput in) throws IOException { + + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + + } + + public List getDriverProfiles() { + return driverProfiles; + } + + public List getPlannerProfiles() { + return plannerProfiles; + } +} 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 03db158b7ae13..b3f023aa819d9 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 @@ -9,9 +9,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.RefCountingListener; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.ResponseHeadersCollector; @@ -19,7 +16,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.esql.planner.PlannerProfile; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -38,38 +34,6 @@ final class ComputeListener implements Releasable { private final ResponseHeadersCollector responseHeaders; private final Runnable runOnFailure; - /** - * Holds the collection of driver profiles and query planning profiles from the individual nodes processing the query. - */ - public static class CollectedProfiles implements Writeable { - public static final CollectedProfiles EMPTY = new CollectedProfiles(List.of(), List.of()); - - private List driverProfiles; - private List plannerProfiles; - - CollectedProfiles(List driverProfiles, List plannerProfiles) { - this.driverProfiles = driverProfiles; - this.plannerProfiles = plannerProfiles; - } - - CollectedProfiles(StreamInput in) throws IOException { - - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - - } - - public List getDriverProfiles() { - return driverProfiles; - } - - public List getPlannerProfiles() { - return plannerProfiles; - } - } - ComputeListener(ThreadPool threadPool, Runnable runOnFailure, ActionListener delegate) { this.runOnFailure = runOnFailure; this.responseHeaders = new ResponseHeadersCollector(threadPool.getThreadContext()); 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 744bfae62890c..5a05ed5348120 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 @@ -21,7 +21,7 @@ * The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest} */ final class ComputeResponse extends TransportResponse { - private final ComputeListener.CollectedProfiles profiles; + private final CollectedProfiles profiles; // for use with ClusterComputeRequests (cross-cluster searches) private final TimeValue took; // overall took time for a specific cluster in a cross-cluster search @@ -30,12 +30,12 @@ final class ComputeResponse extends TransportResponse { public final int skippedShards; public final int failedShards; - ComputeResponse(ComputeListener.CollectedProfiles profiles) { + ComputeResponse(CollectedProfiles profiles) { this(profiles, null, null, null, null, null); } ComputeResponse( - ComputeListener.CollectedProfiles profiles, + CollectedProfiles profiles, TimeValue took, Integer totalShards, Integer successfulShards, @@ -55,9 +55,9 @@ final class ComputeResponse extends TransportResponse { if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { if (in.readBoolean()) { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - profiles = new ComputeListener.CollectedProfiles(in); + profiles = new CollectedProfiles(in); } else { - profiles = new ComputeListener.CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + profiles = new CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); } } else { profiles = null; @@ -103,7 +103,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - public ComputeListener.CollectedProfiles getProfiles() { + public CollectedProfiles getProfiles() { return profiles; } 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 594dea0d49f8e..e3867a455350f 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 @@ -51,6 +51,7 @@ import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.EsPhysicalOperationProviders; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.session.Configuration; import org.elasticsearch.xpack.esql.session.Result; @@ -284,7 +285,7 @@ public void execute( EsqlExecutionInfo.Cluster.Status.PARTIAL ).setFailures(List.of(new ShardSearchFailure(e))).build() ); - dataNodesListener.onResponse(ComputeListener.CollectedProfiles.EMPTY); + dataNodesListener.onResponse(CollectedProfiles.EMPTY); } else { dataNodesListener.onFailure(e); } @@ -348,7 +349,7 @@ void runCompute( CancellableTask task, ComputeContext context, PhysicalPlan plan, - ActionListener listener + ActionListener listener ) { listener = ActionListener.runBefore(listener, () -> Releasables.close(context.searchContexts())); List contexts = new ArrayList<>(context.searchContexts().size()); @@ -368,6 +369,7 @@ public SourceProvider createSourceProvider() { ); } final List drivers; + final PlannerProfile localPlannerProfile = new PlannerProfile(); try { LocalExecutionPlanner planner = new LocalExecutionPlanner( context.sessionId(), @@ -406,9 +408,9 @@ public SourceProvider createSourceProvider() { } ActionListener listenerCollectingStatus = listener.map(ignored -> { if (context.configuration().profile()) { - return drivers.stream().map(Driver::profile).toList(); + return new CollectedProfiles(drivers.stream().map(Driver::profile).toList(), List.of(localPlannerProfile)); } else { - return List.of(); + return CollectedProfiles.EMPTY; } }); 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 3a61d3edcc6d3..de8dd0e6ae033 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 @@ -237,11 +237,11 @@ 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(ComputeListener.CollectedProfiles result) { + public void onResponse(CollectedProfiles result) { try { onBatchCompleted(endBatchIndex); } finally { @@ -255,7 +255,7 @@ public void onFailure(Exception e) { for (ShardId shardId : shardIds) { addShardLevelFailure(shardId, e); } - onResponse(ComputeListener.CollectedProfiles.EMPTY); + onResponse(CollectedProfiles.EMPTY); } else { // TODO: add these to fatal failures so we can continue processing other shards. try { @@ -269,7 +269,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(ComputeListener.CollectedProfiles.EMPTY); + batchListener.onResponse(CollectedProfiles.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 bfaf0b92f887e..892bccb8ddefc 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 @@ -13,7 +13,6 @@ import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.io.IOException; import java.util.List; @@ -24,10 +23,10 @@ * The compute result of {@link DataNodeRequest} */ final class DataNodeComputeResponse extends TransportResponse { - private final ComputeListener.CollectedProfiles profiles; + private final CollectedProfiles profiles; private final Map shardLevelFailures; - DataNodeComputeResponse(ComputeListener.CollectedProfiles profiles, Map shardLevelFailures) { + DataNodeComputeResponse(CollectedProfiles profiles, Map shardLevelFailures) { this.profiles = profiles; this.shardLevelFailures = shardLevelFailures; } @@ -35,16 +34,16 @@ final class DataNodeComputeResponse extends TransportResponse { DataNodeComputeResponse(StreamInput in) throws IOException { if (DataNodeComputeHandler.supportShardLevelRetryFailure(in.getTransportVersion())) { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.profiles = new ComputeListener.CollectedProfiles(in); + this.profiles = new CollectedProfiles(in); } else { - this.profiles = new ComputeListener.CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + this.profiles = new CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); } this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); } else { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.profiles = new ComputeListener.CollectedProfiles(in); + this.profiles = new CollectedProfiles(in); } else { - this.profiles = new ComputeListener.CollectedProfiles( + this.profiles = new CollectedProfiles( Objects.requireNonNullElse(new ComputeResponse(in).getProfiles().getDriverProfiles(), List.of()), List.of() ); @@ -70,7 +69,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - ComputeListener.CollectedProfiles profiles() { + CollectedProfiles profiles() { return profiles; } 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 ea96c1577b7e6..fb6dc0e747a44 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 @@ -19,7 +19,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.compute.operator.FailureCollector; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; @@ -155,9 +154,9 @@ private void reportFailures(ComputeListener computeListener) { } 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(ComputeListener.CollectedProfiles profiles) { + void onAfter(CollectedProfiles profiles) { nodePermits.get(request.node).release(); trySendingRequestsForPendingShards(targetShards, computeListener); listener.onResponse(profiles); @@ -185,7 +184,7 @@ public void onFailure(Exception e, boolean receivedData) { trackShardLevelFailure(shardId, receivedData, e); pendingShardIds.add(shardId); } - onAfter(ComputeListener.CollectedProfiles.EMPTY); + onAfter(CollectedProfiles.EMPTY); } }); } 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 c07ba002524ef..b0cc386df30b7 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 @@ -78,7 +78,7 @@ public void onFailure(Exception e) { if (returnSuccessWithEmptyResult(executionInfo, e)) { updateExecutionInfoToReturnEmptyResult(executionInfo, e); listener.onResponse( - new Result(Analyzer.NO_FIELDS, Collections.emptyList(), Collections.emptyList(), PlannerProfile.EMPTY, executionInfo) + new Result(Analyzer.NO_FIELDS, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), executionInfo) ); } else { listener.onFailure(e); 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 11cfd7343a6da..27f7ae11d3360 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 @@ -692,6 +692,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, List.of(), List.of(), null)) + ); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java index a0357c2393971..baa085ab4456c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.OperatorStatus; import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.esql.plugin.CollectedProfiles; import java.util.List; @@ -31,7 +32,10 @@ protected EsqlQueryResponse.Profile createTestInstance() { @Override protected EsqlQueryResponse.Profile mutateInstance(EsqlQueryResponse.Profile instance) { - return new EsqlQueryResponse.Profile(randomValueOtherThan(instance.drivers(), this::randomDriverProfiles)); + return new EsqlQueryResponse.Profile( + randomValueOtherThan(instance.drivers(), () -> randomDriverProfiles().getDriverProfiles()), + List.of() + ); } @Override @@ -39,8 +43,9 @@ protected NamedWriteableRegistry getNamedWriteableRegistry() { return new NamedWriteableRegistry(List.of(AbstractPageMappingOperator.Status.ENTRY)); } - private List randomDriverProfiles() { - return randomList(10, this::randomDriverProfile); + private CollectedProfiles randomDriverProfiles() { + // NOCOMMIT - test planner profiles too + return new CollectedProfiles(randomList(10, this::randomDriverProfile), List.of()); } private DriverProfile randomDriverProfile() { 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..d484809b120b1 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 @@ -730,7 +730,9 @@ public void testProfileXContent() { List.of(new OperatorStatus("asdf", new AbstractPageMappingOperator.Status(10021, 10, 111, 222))), DriverSleeps.empty() ) - ) + ), + // NOCOMMIT - test planner profiles here + List.of() ), false, false, 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 94f677ead32db..302f1b9aad49d 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 @@ -56,7 +56,7 @@ public void shutdownTransportService() { terminate(threadPool); } - private ComputeListener.CollectedProfiles randomProfiles() { + private CollectedProfiles randomProfiles() { int numProfiles = randomIntBetween(0, 2); List profiles = new ArrayList<>(numProfiles); for (int i = 0; i < numProfiles; i++) { @@ -76,11 +76,11 @@ private ComputeListener.CollectedProfiles randomProfiles() { ); } // NOCOMMIT - add random planner profiles? - return new ComputeListener.CollectedProfiles(profiles, List.of()); + return new CollectedProfiles(profiles, List.of()); } public void testEmpty() { - PlainActionFuture results = new PlainActionFuture<>(); + PlainActionFuture results = new PlainActionFuture<>(); try (var ignored = new ComputeListener(threadPool, () -> {}, results)) { assertFalse(results.isDone()); } @@ -90,7 +90,7 @@ public void testEmpty() { } public void testCollectComputeResults() { - PlainActionFuture future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); List allProfiles = new ArrayList<>(); AtomicInteger onFailure = new AtomicInteger(); try (var computeListener = new ComputeListener(threadPool, onFailure::incrementAndGet, future)) { @@ -105,8 +105,8 @@ public void testCollectComputeResults() { ); } else { var profiles = randomProfiles(); - allProfiles.addAll(profiles); - ActionListener subListener = computeListener.acquireCompute(); + allProfiles.addAll(profiles.getDriverProfiles()); + ActionListener subListener = computeListener.acquireCompute(); threadPool.schedule( ActionRunnable.wrap(subListener, l -> l.onResponse(profiles)), TimeValue.timeValueNanos(between(0, 100)), @@ -115,9 +115,10 @@ public void testCollectComputeResults() { } } } - List profiles = future.actionGet(10, TimeUnit.SECONDS); + CollectedProfiles profiles = future.actionGet(10, TimeUnit.SECONDS); assertThat( - profiles.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), + // TODO: Test planner profiles here? + profiles.getDriverProfiles().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)); @@ -131,11 +132,11 @@ 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())), TimeValue.timeValueNanos(between(0, 100)), @@ -164,11 +165,12 @@ public void testCancelOnFailure() { public void testCollectWarnings() throws Exception { 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(CollectedProfiles result) { assertThat( - result.stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), + // TODO: test planner profiles here? + result.getDriverProfiles().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() @@ -204,7 +206,7 @@ public void onFailure(Exception e) { ); } else { var resp = randomProfiles(); - allProfiles.addAll(resp); + allProfiles.addAll(resp.getDriverProfiles()); 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 dd02d6fc3b189..6d3a31185b95a 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 @@ -112,7 +112,7 @@ public void testOnePass() { Queue sent = ConcurrentCollections.newQueue(); var future = sendRequests(targetShards, randomBoolean(), (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(CollectedProfiles.EMPTY, Map.of()))); }); safeGet(future); assertThat(sent.size(), equalTo(2)); @@ -131,7 +131,7 @@ public void testMissingShards() { var targetShards = List.of(targetShard(shard1, node1), targetShard(shard3), targetShard(shard4, node2, node3)); var future = sendRequests(targetShards, true, (node, shardIds, aliasFilters, listener) -> { assertThat(shard3, not(in(shardIds))); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(List.of(), Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(CollectedProfiles.EMPTY, Map.of()))); }); ComputeResponse resp = safeGet(future); assertThat(resp.totalShards, equalTo(3)); @@ -158,7 +158,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(CollectedProfiles.EMPTY, failures))); }); try { future.actionGet(1, TimeUnit.MINUTES); @@ -187,7 +187,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(CollectedProfiles.EMPTY, failures))); }); var error = expectThrows(Exception.class, future::actionGet); assertNotNull(ExceptionsHelper.unwrap(error, IOException.class)); @@ -212,7 +212,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(CollectedProfiles.EMPTY, Map.of()))); } }); Exception exception = expectThrows(Exception.class, future::actionGet); @@ -232,7 +232,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(CollectedProfiles.EMPTY, Map.of()))); } }); ComputeResponse resp = safeGet(future); @@ -253,7 +253,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(CollectedProfiles.EMPTY, Map.of()))); } })); assertThat(response.totalShards, equalTo(1)); From 873f2d73cf7ed80ffc4171911554e0187d305561 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 14 Mar 2025 20:05:27 +0000 Subject: [PATCH 06/20] [CI] Auto commit changes from spotless --- .../elasticsearch/xpack/esql/plugin/ComputeService.java | 7 +------ .../org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java | 1 - 2 files changed, 1 insertion(+), 7 deletions(-) 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 e3867a455350f..7e8d1053aff9f 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 @@ -345,12 +345,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++) { 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 b0cc386df30b7..445ac85c22d8e 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 @@ -31,7 +31,6 @@ import org.elasticsearch.xpack.esql.analysis.TableInfo; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.Collections; import java.util.HashMap; From 9a09236a5f336073937226563b95bbac2f315fff Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Fri, 14 Mar 2025 16:30:55 -0400 Subject: [PATCH 07/20] Profile serialization --- .../xpack/esql/action/EsqlQueryResponse.java | 3 +++ .../xpack/esql/planner/PlannerProfile.java | 14 ++++++-------- .../xpack/esql/plugin/ComputeService.java | 2 +- .../xpack/esql/session/EsqlSession.java | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) 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 14196a893ac95..1e75450563250 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 @@ -375,6 +375,9 @@ public Profile(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(drivers); + if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { + out.writeCollection(plannerProfile); + } } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index a66a4d41350f9..966ea86e34c14 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -22,24 +22,22 @@ */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { - public static final PlannerProfile EMPTY = new PlannerProfile(); + public static final PlannerProfile EMPTY = new PlannerProfile(false); private final boolean isLocalPlanning; public static PlannerProfile readFrom(StreamInput in) throws IOException { - // NOCOMMIT - throw new UnsupportedOperationException(); + boolean isLocalPlanning = in.readBoolean(); + return new PlannerProfile(isLocalPlanning); } - public PlannerProfile() { - // NOCOMMIT - throw new UnsupportedOperationException(); + public PlannerProfile(boolean isLocalPlanning) { + this.isLocalPlanning = isLocalPlanning; } @Override public void writeTo(StreamOutput out) throws IOException { - // NOCOMMIT - throw new UnsupportedOperationException(); + out.writeBoolean(isLocalPlanning); } @Override 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 e3867a455350f..d96ca854b1faa 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 @@ -369,7 +369,7 @@ public SourceProvider createSourceProvider() { ); } final List drivers; - final PlannerProfile localPlannerProfile = new PlannerProfile(); + final PlannerProfile localPlannerProfile = new PlannerProfile(true); try { LocalExecutionPlanner planner = new LocalExecutionPlanner( context.sessionId(), 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 30eeb87135f06..03422b977b244 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 @@ -148,7 +148,7 @@ public EsqlSession( this.logicalPlanOptimizer = logicalPlanOptimizer; this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planTelemetry = planTelemetry; - this.plannerProfile = new PlannerProfile(); + this.plannerProfile = new PlannerProfile(false); this.indicesExpressionGrouper = indicesExpressionGrouper; this.preMapper = new PreMapper(services); } From f28aa9ea61a7c54f0bc3dda45241ee1fbf781887 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 17 Mar 2025 10:29:25 -0400 Subject: [PATCH 08/20] Just use Profie instead of creating a new class --- .../xpack/esql/action/EsqlQueryResponse.java | 16 ++++-- .../esql/plugin/ClusterComputeHandler.java | 7 +-- .../xpack/esql/plugin/CollectedProfiles.java | 49 ------------------- .../xpack/esql/plugin/ComputeListener.java | 8 +-- .../xpack/esql/plugin/ComputeResponse.java | 13 ++--- .../xpack/esql/plugin/ComputeService.java | 9 ++-- .../esql/plugin/DataNodeComputeHandler.java | 11 +++-- .../esql/plugin/DataNodeComputeResponse.java | 15 +++--- .../esql/plugin/DataNodeRequestSender.java | 7 +-- .../action/EsqlQueryResponseProfileTests.java | 8 +-- .../esql/plugin/ComputeListenerTests.java | 21 ++++---- .../plugin/DataNodeRequestSenderTests.java | 15 +++--- 12 files changed, 74 insertions(+), 105 deletions(-) delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java 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 1e75450563250..54bcf6e8c37a0 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 @@ -28,7 +28,6 @@ import org.elasticsearch.xpack.core.esql.action.EsqlResponse; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.planner.PlannerProfile; -import org.elasticsearch.xpack.esql.plugin.CollectedProfiles; import java.io.IOException; import java.util.Collections; @@ -352,12 +351,15 @@ public static class Profile implements Writeable, ChunkedToXContentObject { private final List drivers; private final List plannerProfile; + public static final Profile EMPTY = new Profile(List.of(), List.of()); + public Profile(List drivers, List plannerProfile) { this.drivers = drivers; this.plannerProfile = plannerProfile; } - public Profile(CollectedProfiles profiles) { + // NOCOMMIT - this should be removed + public Profile(Profile profiles) { this.drivers = profiles.getDriverProfiles(); this.plannerProfile = profiles.getPlannerProfiles(); } @@ -369,9 +371,15 @@ public Profile(StreamInput in) throws IOException { } else { this.plannerProfile = List.of(); } + } + public List getDriverProfiles() { + return drivers; } + public List getPlannerProfiles() { + return plannerProfile; + } @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(drivers); @@ -389,12 +397,12 @@ public boolean equals(Object o) { return false; } Profile profile = (Profile) o; - return Objects.equals(drivers, profile.drivers); + return Objects.equals(drivers, profile.drivers) && Objects.equals(plannerProfile, profile.plannerProfile); } @Override public int hashCode() { - return Objects.hash(drivers); + return Objects.hash(drivers, plannerProfile); } @Override 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 946719add1b04..b12ae348a649f 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 @@ -26,6 +26,7 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.session.Configuration; @@ -74,7 +75,7 @@ void startComputeOnRemoteCluster( RemoteCluster cluster, Runnable cancelQueryOnFailure, EsqlExecutionInfo executionInfo, - ActionListener listener + ActionListener listener ) { var queryPragmas = configuration.pragmas(); listener = ActionListener.runBefore(listener, exchangeSource.addEmptySink()::close); @@ -86,10 +87,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(CollectedProfiles.EMPTY); + l.onResponse(EsqlQueryResponse.Profile.EMPTY); } else if (configuration.allowPartialResults()) { EsqlCCSUtils.markClusterWithFinalStateAndNoShards(executionInfo, clusterAlias, EsqlExecutionInfo.Cluster.Status.PARTIAL, e); - l.onResponse(CollectedProfiles.EMPTY); + l.onResponse(EsqlQueryResponse.Profile.EMPTY); } else { l.onFailure(e); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java deleted file mode 100644 index 38e1e13db91b3..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/CollectedProfiles.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.plugin; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.compute.operator.DriverProfile; -import org.elasticsearch.xpack.esql.planner.PlannerProfile; - -import java.io.IOException; -import java.util.List; - -/** - * Holds the collection of driver profiles and query planning profiles from the individual nodes processing the query. - */ -public class CollectedProfiles implements Writeable { - public static final CollectedProfiles EMPTY = new CollectedProfiles(List.of(), List.of()); - - private List driverProfiles; - private List plannerProfiles; - - public CollectedProfiles(List driverProfiles, List plannerProfiles) { - this.driverProfiles = driverProfiles; - this.plannerProfiles = plannerProfiles; - } - - public CollectedProfiles(StreamInput in) throws IOException { - - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - - } - - public List getDriverProfiles() { - return driverProfiles; - } - - public List getPlannerProfiles() { - return plannerProfiles; - } -} 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 b3f023aa819d9..23f7f09db4723 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 @@ -14,6 +14,7 @@ import org.elasticsearch.compute.operator.ResponseHeadersCollector; import org.elasticsearch.core.Releasable; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.ArrayList; @@ -34,7 +35,7 @@ final class ComputeListener implements Releasable { 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<>()); @@ -42,7 +43,7 @@ final class ComputeListener implements Releasable { // 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(new CollectedProfiles(collectedProfiles.stream().toList(), collectedPlannerProfiles.stream().toList())); + delegate.onResponse(new EsqlQueryResponse.Profile(collectedProfiles.stream().toList(), collectedPlannerProfiles.stream().toList())); })); } @@ -63,11 +64,12 @@ 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 -> { responseHeaders.collect(); if (profiles != null) { + // TODO: move profile merging onto profile object if (profiles.getDriverProfiles().isEmpty() == false) { collectedProfiles.addAll(profiles.getDriverProfiles()); } 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 5a05ed5348120..90ea04b73e01b 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 @@ -13,6 +13,7 @@ import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.TimeValue; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import java.io.IOException; import java.util.List; @@ -21,7 +22,7 @@ * The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest} */ final class ComputeResponse extends TransportResponse { - private final CollectedProfiles profiles; + private final EsqlQueryResponse.Profile profiles; // for use with ClusterComputeRequests (cross-cluster searches) private final TimeValue took; // overall took time for a specific cluster in a cross-cluster search @@ -30,12 +31,12 @@ final class ComputeResponse extends TransportResponse { public final int skippedShards; public final int failedShards; - ComputeResponse(CollectedProfiles profiles) { + ComputeResponse(EsqlQueryResponse.Profile profiles) { this(profiles, null, null, null, null, null); } ComputeResponse( - CollectedProfiles profiles, + EsqlQueryResponse.Profile profiles, TimeValue took, Integer totalShards, Integer successfulShards, @@ -55,9 +56,9 @@ final class ComputeResponse extends TransportResponse { if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) { if (in.readBoolean()) { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - profiles = new CollectedProfiles(in); + profiles = new EsqlQueryResponse.Profile(in); } else { - profiles = new CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + profiles = new EsqlQueryResponse.Profile(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); } } else { profiles = null; @@ -103,7 +104,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - public CollectedProfiles getProfiles() { + public EsqlQueryResponse.Profile getProfiles() { return profiles; } 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 c880b8885fb49..2a9e9303fd9a5 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 @@ -41,6 +41,7 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.action.EsqlQueryAction; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.enrich.EnrichLookupService; @@ -285,7 +286,7 @@ public void execute( EsqlExecutionInfo.Cluster.Status.PARTIAL ).setFailures(List.of(new ShardSearchFailure(e))).build() ); - dataNodesListener.onResponse(CollectedProfiles.EMPTY); + dataNodesListener.onResponse(EsqlQueryResponse.Profile.EMPTY); } else { dataNodesListener.onFailure(e); } @@ -345,7 +346,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++) { @@ -403,9 +404,9 @@ public SourceProvider createSourceProvider() { } ActionListener listenerCollectingStatus = listener.map(ignored -> { if (context.configuration().profile()) { - return new CollectedProfiles(drivers.stream().map(Driver::profile).toList(), List.of(localPlannerProfile)); + return new EsqlQueryResponse.Profile(drivers.stream().map(Driver::profile).toList(), List.of(localPlannerProfile)); } else { - return CollectedProfiles.EMPTY; + return EsqlQueryResponse.Profile.EMPTY; } }); 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 de8dd0e6ae033..5289008e7603a 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 @@ -38,6 +38,7 @@ import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; @@ -237,11 +238,11 @@ 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(CollectedProfiles result) { + public void onResponse(EsqlQueryResponse.Profile result) { try { onBatchCompleted(endBatchIndex); } finally { @@ -255,7 +256,7 @@ public void onFailure(Exception e) { for (ShardId shardId : shardIds) { addShardLevelFailure(shardId, e); } - onResponse(CollectedProfiles.EMPTY); + onResponse(EsqlQueryResponse.Profile.EMPTY); } else { // TODO: add these to fatal failures so we can continue processing other shards. try { @@ -269,7 +270,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(CollectedProfiles.EMPTY); + batchListener.onResponse(EsqlQueryResponse.Profile.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 892bccb8ddefc..c0e6e4d1eb361 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 @@ -13,6 +13,7 @@ import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import java.io.IOException; import java.util.List; @@ -23,10 +24,10 @@ * The compute result of {@link DataNodeRequest} */ final class DataNodeComputeResponse extends TransportResponse { - private final CollectedProfiles profiles; + private final EsqlQueryResponse.Profile profiles; private final Map shardLevelFailures; - DataNodeComputeResponse(CollectedProfiles profiles, Map shardLevelFailures) { + DataNodeComputeResponse(EsqlQueryResponse.Profile profiles, Map shardLevelFailures) { this.profiles = profiles; this.shardLevelFailures = shardLevelFailures; } @@ -34,16 +35,16 @@ final class DataNodeComputeResponse extends TransportResponse { DataNodeComputeResponse(StreamInput in) throws IOException { if (DataNodeComputeHandler.supportShardLevelRetryFailure(in.getTransportVersion())) { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.profiles = new CollectedProfiles(in); + this.profiles = new EsqlQueryResponse.Profile(in); } else { - this.profiles = new CollectedProfiles(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); + this.profiles = new EsqlQueryResponse.Profile(in.readCollectionAsImmutableList(DriverProfile::readFrom), List.of()); } this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException); } else { if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { - this.profiles = new CollectedProfiles(in); + this.profiles = new EsqlQueryResponse.Profile(in); } else { - this.profiles = new CollectedProfiles( + this.profiles = new EsqlQueryResponse.Profile( Objects.requireNonNullElse(new ComputeResponse(in).getProfiles().getDriverProfiles(), List.of()), List.of() ); @@ -69,7 +70,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - CollectedProfiles profiles() { + EsqlQueryResponse.Profile profiles() { return profiles; } 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 fb6dc0e747a44..9bb2c5a1148cd 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 @@ -31,6 +31,7 @@ import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.action.EsqlSearchShardsAction; import java.util.ArrayList; @@ -154,9 +155,9 @@ private void reportFailures(ComputeListener computeListener) { } 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(CollectedProfiles profiles) { + void onAfter(EsqlQueryResponse.Profile profiles) { nodePermits.get(request.node).release(); trySendingRequestsForPendingShards(targetShards, computeListener); listener.onResponse(profiles); @@ -184,7 +185,7 @@ public void onFailure(Exception e, boolean receivedData) { trackShardLevelFailure(shardId, receivedData, e); pendingShardIds.add(shardId); } - onAfter(CollectedProfiles.EMPTY); + onAfter(EsqlQueryResponse.Profile.EMPTY); } }); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java index baa085ab4456c..6e9bf76a3d9b9 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseProfileTests.java @@ -15,7 +15,6 @@ import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.OperatorStatus; import org.elasticsearch.test.AbstractWireSerializingTestCase; -import org.elasticsearch.xpack.esql.plugin.CollectedProfiles; import java.util.List; @@ -27,7 +26,7 @@ protected Writeable.Reader instanceReader() { @Override protected EsqlQueryResponse.Profile createTestInstance() { - return new EsqlQueryResponse.Profile(randomDriverProfiles()); + return randomDriverProfiles(); } @Override @@ -43,9 +42,10 @@ protected NamedWriteableRegistry getNamedWriteableRegistry() { return new NamedWriteableRegistry(List.of(AbstractPageMappingOperator.Status.ENTRY)); } - private CollectedProfiles randomDriverProfiles() { + // TODO: this method is duplicated in ComputeListenerTests + private EsqlQueryResponse.Profile randomDriverProfiles() { // NOCOMMIT - test planner profiles too - return new CollectedProfiles(randomList(10, this::randomDriverProfile), List.of()); + return new EsqlQueryResponse.Profile(randomList(10, this::randomDriverProfile), List.of()); } private DriverProfile randomDriverProfile() { 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 302f1b9aad49d..091f649c2f613 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 @@ -20,6 +20,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.junit.After; import org.junit.Before; @@ -56,7 +57,7 @@ public void shutdownTransportService() { terminate(threadPool); } - private CollectedProfiles randomProfiles() { + private EsqlQueryResponse.Profile randomProfiles() { int numProfiles = randomIntBetween(0, 2); List profiles = new ArrayList<>(numProfiles); for (int i = 0; i < numProfiles; i++) { @@ -76,11 +77,11 @@ private CollectedProfiles randomProfiles() { ); } // NOCOMMIT - add random planner profiles? - return new CollectedProfiles(profiles, List.of()); + return new EsqlQueryResponse.Profile(profiles, List.of()); } public void testEmpty() { - PlainActionFuture results = new PlainActionFuture<>(); + PlainActionFuture results = new PlainActionFuture<>(); try (var ignored = new ComputeListener(threadPool, () -> {}, results)) { assertFalse(results.isDone()); } @@ -90,7 +91,7 @@ public void testEmpty() { } public void testCollectComputeResults() { - PlainActionFuture future = new PlainActionFuture<>(); + PlainActionFuture future = new PlainActionFuture<>(); List allProfiles = new ArrayList<>(); AtomicInteger onFailure = new AtomicInteger(); try (var computeListener = new ComputeListener(threadPool, onFailure::incrementAndGet, future)) { @@ -106,7 +107,7 @@ public void testCollectComputeResults() { } else { var profiles = randomProfiles(); allProfiles.addAll(profiles.getDriverProfiles()); - ActionListener subListener = computeListener.acquireCompute(); + ActionListener subListener = computeListener.acquireCompute(); threadPool.schedule( ActionRunnable.wrap(subListener, l -> l.onResponse(profiles)), TimeValue.timeValueNanos(between(0, 100)), @@ -115,7 +116,7 @@ public void testCollectComputeResults() { } } } - CollectedProfiles profiles = future.actionGet(10, TimeUnit.SECONDS); + EsqlQueryResponse.Profile profiles = future.actionGet(10, TimeUnit.SECONDS); assertThat( // TODO: Test planner profiles here? profiles.getDriverProfiles().stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), @@ -132,11 +133,11 @@ 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())), TimeValue.timeValueNanos(between(0, 100)), @@ -165,9 +166,9 @@ public void testCancelOnFailure() { public void testCollectWarnings() throws Exception { List allProfiles = new ArrayList<>(); Map> allWarnings = new HashMap<>(); - ActionListener rootListener = new ActionListener<>() { + ActionListener rootListener = new ActionListener<>() { @Override - public void onResponse(CollectedProfiles result) { + public void onResponse(EsqlQueryResponse.Profile result) { assertThat( // TODO: test planner profiles here? result.getDriverProfiles().stream().collect(Collectors.toMap(p -> p, p -> 1, Integer::sum)), 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 6d3a31185b95a..03eca76c9d4f6 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 @@ -32,6 +32,7 @@ import org.elasticsearch.threadpool.FixedExecutorBuilder; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.junit.After; import org.junit.Before; @@ -112,7 +113,7 @@ public void testOnePass() { Queue sent = ConcurrentCollections.newQueue(); var future = sendRequests(targetShards, randomBoolean(), (node, shardIds, aliasFilters, listener) -> { sent.add(new NodeRequest(node, shardIds, aliasFilters)); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(CollectedProfiles.EMPTY, Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of()))); }); safeGet(future); assertThat(sent.size(), equalTo(2)); @@ -131,7 +132,7 @@ public void testMissingShards() { var targetShards = List.of(targetShard(shard1, node1), targetShard(shard3), targetShard(shard4, node2, node3)); var future = sendRequests(targetShards, true, (node, shardIds, aliasFilters, listener) -> { assertThat(shard3, not(in(shardIds))); - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(CollectedProfiles.EMPTY, Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of()))); }); ComputeResponse resp = safeGet(future); assertThat(resp.totalShards, equalTo(3)); @@ -158,7 +159,7 @@ public void testRetryThenSuccess() { if (node.equals(node4) && shardIds.contains(shard2)) { failures.put(shard2, new IOException("test")); } - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(CollectedProfiles.EMPTY, failures))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, failures))); }); try { future.actionGet(1, TimeUnit.MINUTES); @@ -187,7 +188,7 @@ public void testRetryButFail() { if (shardIds.contains(shard5)) { failures.put(shard5, new IOException("test failure for shard5")); } - runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(CollectedProfiles.EMPTY, failures))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, failures))); }); var error = expectThrows(Exception.class, future::actionGet); assertNotNull(ExceptionsHelper.unwrap(error, IOException.class)); @@ -212,7 +213,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(CollectedProfiles.EMPTY, Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of()))); } }); Exception exception = expectThrows(Exception.class, future::actionGet); @@ -232,7 +233,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(CollectedProfiles.EMPTY, Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of()))); } }); ComputeResponse resp = safeGet(future); @@ -253,7 +254,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(CollectedProfiles.EMPTY, Map.of()))); + runWithDelay(() -> listener.onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of()))); } })); assertThat(response.totalShards, equalTo(1)); From 704d51a87495f0783c1dd1cfae39cda364cc8b4d Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 17 Mar 2025 11:51:32 -0400 Subject: [PATCH 09/20] Use profile instead of explicit lists --- .../xpack/esql/action/EsqlQueryResponse.java | 16 ++++++++++------ .../xpack/esql/plugin/ComputeService.java | 5 ++--- .../esql/plugin/TransportEsqlQueryAction.java | 2 +- .../xpack/esql/session/EsqlCCSUtils.java | 3 ++- .../xpack/esql/session/EsqlSession.java | 13 ++++++------- .../elasticsearch/xpack/esql/session/Result.java | 11 ++++------- .../org/elasticsearch/xpack/esql/CsvTests.java | 3 ++- 7 files changed, 27 insertions(+), 26 deletions(-) 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 54bcf6e8c37a0..6f2496e935dca 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 @@ -30,6 +30,7 @@ import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -353,17 +354,16 @@ public static class Profile implements Writeable, ChunkedToXContentObject { public static final Profile EMPTY = new Profile(List.of(), List.of()); + public Profile() { + this.drivers = new ArrayList<>(); + this.plannerProfile = new ArrayList<>(); + } + public Profile(List drivers, List plannerProfile) { this.drivers = drivers; this.plannerProfile = plannerProfile; } - // NOCOMMIT - this should be removed - public Profile(Profile profiles) { - this.drivers = profiles.getDriverProfiles(); - this.plannerProfile = profiles.getPlannerProfiles(); - } - public Profile(StreamInput in) throws IOException { this.drivers = in.readCollectionAsImmutableList(DriverProfile::readFrom); if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_PLANNER_PROFILE)) { @@ -373,6 +373,10 @@ public Profile(StreamInput in) throws IOException { } } + public void merge(Profile other) { + this.drivers.addAll(other.drivers); + this.plannerProfile.addAll(other.plannerProfile); + } public List getDriverProfiles() { return drivers; } 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 2a9e9303fd9a5..0267e320a427a 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 @@ -176,8 +176,7 @@ public void execute( return new Result( physicalPlan.output(), collectedPages, - profiles.getDriverProfiles(), - profiles.getPlannerProfiles(), + profiles, execInfo ); })) @@ -211,7 +210,7 @@ public void execute( 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.getDriverProfiles(), profiles.getPlannerProfiles(), execInfo); + return new Result(outputAttributes, collectedPages, profiles, execInfo); }))) { try (Releasable ignored = exchangeSource.addEmptySink()) { // run compute on the coordinator 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 41d4b76c35fee..828a4c454ee22 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 @@ -316,7 +316,7 @@ 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(), result.plannerProfiles()) + ? result.profile() : null; threadPool.getThreadContext().addResponseHeader(AsyncExecutionId.ASYNC_EXECUTION_IS_RUNNING_HEADER, "?0"); if (task instanceof EsqlQueryTask asyncTask && request.keepOnCompletion()) { 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 445ac85c22d8e..765f2c91013ea 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 @@ -27,6 +27,7 @@ import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo.Cluster; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.analysis.TableInfo; import org.elasticsearch.xpack.esql.index.IndexResolution; @@ -77,7 +78,7 @@ public void onFailure(Exception e) { if (returnSuccessWithEmptyResult(executionInfo, e)) { updateExecutionInfoToReturnEmptyResult(executionInfo, e); listener.onResponse( - new Result(Analyzer.NO_FIELDS, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), executionInfo) + new Result(Analyzer.NO_FIELDS, Collections.emptyList(), EsqlQueryResponse.Profile.EMPTY, executionInfo) ); } else { listener.onFailure(e); 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 03422b977b244..74be10dccae7d 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 @@ -17,7 +17,6 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexMode; @@ -28,6 +27,7 @@ import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; import org.elasticsearch.xpack.esql.analysis.EnrichResolution; @@ -244,7 +244,7 @@ private void executeSubPlans( // TODO: merge into one method if (subplans.size() > 0) { // code-path to execute subplans - executeSubPlan(new ArrayList<>(), physicalPlan, iterator, executionInfo, runner, listener); + executeSubPlan(new EsqlQueryResponse.Profile(), physicalPlan, iterator, executionInfo, runner, listener); } else { // execute main plan runner.run(physicalPlan, listener); @@ -252,7 +252,7 @@ private void executeSubPlans( } private void executeSubPlan( - List profileAccumulator, + EsqlQueryResponse.Profile profileAccumulator, PhysicalPlan plan, Iterator subPlanIterator, EsqlExecutionInfo executionInfo, @@ -265,7 +265,7 @@ private void executeSubPlan( runner.run(tuple.physical, listener.delegateFailureAndWrap((next, result) -> { try { - profileAccumulator.addAll(result.profiles()); + profileAccumulator.merge(result.profile()); LocalRelation resultWrapper = resultToPlan(tuple.logical, result); // replace the original logical plan with the backing result @@ -302,10 +302,9 @@ private void executeSubPlan( if (subPlanIterator.hasNext() == false) { // In the production path, this is runner.run calls ComputeService.execute runner.run(newPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { - profileAccumulator.addAll(finalResult.profiles()); - plannerProfiles.addAll(finalResult.plannerProfiles()); + profileAccumulator.merge(finalResult.profile()); finalListener.onResponse( - new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, plannerProfiles, executionInfo) + new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, executionInfo) ); })); } else { 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 70acb963e2e0f..c809a42a2feea 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,12 +9,11 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; -import org.elasticsearch.compute.operator.DriverProfile; import org.elasticsearch.core.Nullable; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.List; @@ -25,18 +24,16 @@ * 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 + * @param profile {@link org.elasticsearch.xpack.esql.action.EsqlQueryResponse.Profile}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 plannerProfiles {@link PlannerProfile} for this query. Like the driver profile, this is only - * included if the user asks for it. * @param executionInfo Metadata about the execution of this query. Used for cross cluster queries. */ public record Result( List schema, List pages, - List profiles, - List plannerProfiles, + EsqlQueryResponse.Profile profile, @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 27f7ae11d3360..5cabf06d31f9f 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 @@ -50,6 +50,7 @@ import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo; import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; import org.elasticsearch.xpack.esql.analysis.EnrichResolution; @@ -694,7 +695,7 @@ 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(), List.of(), null)) + listener.map(ignore -> new Result(physicalPlan.output(), collectedPages, EsqlQueryResponse.Profile.EMPTY, null)) ); } } From c7b782dfced3d8e8932758317a98759bf40580ca Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 17 Mar 2025 12:21:30 -0400 Subject: [PATCH 10/20] fix tests --- .../xpack/esql/plugin/DataNodeRequestSender.java | 2 +- .../xpack/esql/plugin/DataNodeRequestSenderTests.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) 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 5e49b9e928b49..0726fcc620276 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 @@ -278,7 +278,7 @@ public void onSkip() { if (rootTask.isCancelled()) { onFailure(new TaskCancelledException("null"), true); } else { - onResponse(new DataNodeComputeResponse(List.of(), Map.of())); + onResponse(new DataNodeComputeResponse(EsqlQueryResponse.Profile.EMPTY, Map.of())); } } }); 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 1d82669107bec..03a0aca56ddec 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 @@ -325,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(EsqlQueryResponse.Profile.EMPTY, Map.of())); }); })); assertThat(sent.size(), equalTo(5)); @@ -348,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(EsqlQueryResponse.Profile.EMPTY, Map.of())); } else { listener.onSkip(); } @@ -370,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(EsqlQueryResponse.Profile.EMPTY, Map.of())); } else if (Objects.equals(node.getId(), node2.getId()) && shardIds.equals(List.of(shard1))) { listener.onSkip(); } @@ -395,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(EsqlQueryResponse.Profile.EMPTY, Map.of()))); })); assertThat(sent, equalTo(List.of("node-1", "node-2", "node-3", "node-4"))); } @@ -408,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(EsqlQueryResponse.Profile.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))))); From 73ddcb76eabced77720234fc76da9cef8e2b4de1 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 17 Mar 2025 17:19:15 +0000 Subject: [PATCH 11/20] [CI] Auto commit changes from spotless --- .../elasticsearch/xpack/esql/action/EsqlQueryResponse.java | 2 ++ .../elasticsearch/xpack/esql/plugin/ComputeListener.java | 4 +++- .../elasticsearch/xpack/esql/plugin/ComputeService.java | 7 +------ .../xpack/esql/plugin/TransportEsqlQueryAction.java | 4 +--- .../org/elasticsearch/xpack/esql/session/EsqlSession.java | 4 +--- 5 files changed, 8 insertions(+), 13 deletions(-) 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 bdc952369824b..2d621a4d56cd4 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 @@ -375,6 +375,7 @@ public void merge(Profile other) { this.drivers.addAll(other.drivers); this.plannerProfile.addAll(other.plannerProfile); } + public List getDriverProfiles() { return drivers; } @@ -382,6 +383,7 @@ public List getDriverProfiles() { public List getPlannerProfiles() { return plannerProfile; } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(drivers); 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 23f7f09db4723..a5c3aa01cd603 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 @@ -43,7 +43,9 @@ final class ComputeListener implements Releasable { // 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(new EsqlQueryResponse.Profile(collectedProfiles.stream().toList(), collectedPlannerProfiles.stream().toList())); + delegate.onResponse( + new EsqlQueryResponse.Profile(collectedProfiles.stream().toList(), collectedPlannerProfiles.stream().toList()) + ); })); } 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 9727e4248fbaa..84bdd2fa67b8c 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 @@ -173,12 +173,7 @@ public void execute( try ( var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result( - physicalPlan.output(), - collectedPages, - profiles, - execInfo - ); + return new Result(physicalPlan.output(), collectedPages, profiles, execInfo); })) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); 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 828a4c454ee22..4bf347ccb4cd9 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,9 +315,7 @@ 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() - ? result.profile() - : null; + EsqlQueryResponse.Profile profile = configuration.profile() ? result.profile() : null; threadPool.getThreadContext().addResponseHeader(AsyncExecutionId.ASYNC_EXECUTION_IS_RUNNING_HEADER, "?0"); if (task instanceof EsqlQueryTask asyncTask && request.keepOnCompletion()) { String asyncExecutionId = asyncTask.getExecutionId().getEncoded(); 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 74be10dccae7d..e84134d2a460c 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 @@ -303,9 +303,7 @@ private void executeSubPlan( // In the production path, this is runner.run calls ComputeService.execute runner.run(newPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { profileAccumulator.merge(finalResult.profile()); - finalListener.onResponse( - new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, executionInfo) - ); + finalListener.onResponse(new Result(finalResult.schema(), finalResult.pages(), profileAccumulator, executionInfo)); })); } else { // continue executing the subplans From 62da773e6bc7c75ada028a2ba54fb4f54e538330 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 17 Mar 2025 17:10:13 -0400 Subject: [PATCH 12/20] plumb planner profile through to the rule runner --- .../xpack/esql/analysis/Analyzer.java | 5 ++-- .../xpack/esql/execution/PlanExecutor.java | 2 +- .../optimizer/LocalLogicalPlanOptimizer.java | 5 ++-- .../optimizer/LocalPhysicalPlanOptimizer.java | 5 ++-- .../esql/optimizer/LogicalPlanOptimizer.java | 5 ++-- .../esql/optimizer/PhysicalPlanOptimizer.java | 5 ++-- .../xpack/esql/planner/PlannerUtils.java | 23 +++++++++++++++---- .../xpack/esql/plugin/ComputeService.java | 15 ++++++------ .../esql/rule/ParameterizedRuleExecutor.java | 3 ++- .../xpack/esql/session/EsqlSession.java | 11 +++++---- .../elasticsearch/xpack/esql/CsvTests.java | 14 ++++++++--- .../esql/analysis/AnalyzerTestUtils.java | 8 +++---- .../xpack/esql/analysis/AnalyzerTests.java | 6 ++--- .../xpack/esql/analysis/ParsingTests.java | 2 +- .../function/CheckLicenseTests.java | 2 +- .../LocalLogicalPlanOptimizerTests.java | 9 ++++---- .../LocalPhysicalPlanOptimizerTests.java | 2 +- .../optimizer/LogicalPlanOptimizerTests.java | 14 +++++------ .../optimizer/PhysicalPlanOptimizerTests.java | 7 +++--- .../esql/optimizer/TestPlannerOptimizer.java | 3 ++- .../logical/PropagateInlineEvalsTests.java | 2 +- .../xpack/esql/planner/FilterTests.java | 2 +- .../esql/planner/QueryTranslatorTests.java | 2 +- .../esql/plugin/ClusterRequestTests.java | 2 +- .../DataNodeRequestSerializationTests.java | 2 +- 25 files changed, 94 insertions(+), 62 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java index 93e9ede80b420..20977067e62b9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java @@ -95,6 +95,7 @@ import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.rule.ParameterizedRule; import org.elasticsearch.xpack.esql.rule.ParameterizedRuleExecutor; import org.elasticsearch.xpack.esql.rule.Rule; @@ -189,8 +190,8 @@ public class Analyzer extends ParameterizedRuleExecutor { - public LocalLogicalPlanOptimizer(LocalLogicalOptimizerContext localLogicalOptimizerContext) { - super(localLogicalOptimizerContext); + public LocalLogicalPlanOptimizer(LocalLogicalOptimizerContext localLogicalOptimizerContext, PlannerProfile profile) { + super(localLogicalOptimizerContext, profile); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java index a865f784137ad..4b7e4739880b9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java @@ -19,6 +19,7 @@ import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.SpatialDocValuesExtraction; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.SpatialShapeBoundsExtraction; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.rule.ParameterizedRuleExecutor; import org.elasticsearch.xpack.esql.rule.Rule; @@ -36,8 +37,8 @@ public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor searchContexts, Configuration configuration, FoldContext foldCtx, - PhysicalPlan plan + PhysicalPlan plan, + PlannerProfile profile ) { - return localPlan(configuration, foldCtx, plan, SearchContextStats.from(searchContexts)); + return localPlan(configuration, foldCtx, plan, SearchContextStats.from(searchContexts), profile); } - public static PhysicalPlan localPlan(Configuration configuration, FoldContext foldCtx, PhysicalPlan plan, SearchStats searchStats) { - final var logicalOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats)); - var physicalOptimizer = new LocalPhysicalPlanOptimizer(new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats)); + public static PhysicalPlan localPlan( + Configuration configuration, + FoldContext foldCtx, + PhysicalPlan plan, + SearchStats searchStats, + PlannerProfile profile + ) { + final var logicalOptimizer = new LocalLogicalPlanOptimizer( + new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats), + profile + ); + var physicalOptimizer = new LocalPhysicalPlanOptimizer( + new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats), + profile + ); return localPlan(plan, logicalOptimizer, physicalOptimizer); } 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 9727e4248fbaa..f9beed32a6335 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 @@ -173,12 +173,7 @@ public void execute( try ( var computeListener = new ComputeListener(transportService.getThreadPool(), cancelQueryOnFailure, listener.map(profiles -> { updateExecutionInfoAfterCoordinatorOnlyQuery(execInfo); - return new Result( - physicalPlan.output(), - collectedPages, - profiles, - execInfo - ); + return new Result(physicalPlan.output(), collectedPages, profiles, execInfo); })) ) { runCompute(rootTask, computeContext, coordinatorPlan, computeListener.acquireCompute()); @@ -385,7 +380,13 @@ public SourceProvider createSourceProvider() { LOGGER.debug("Received physical plan:\n{}", plan); - plan = PlannerUtils.localPlan(context.searchExecutionContexts(), context.configuration(), context.foldCtx(), plan); + plan = PlannerUtils.localPlan( + context.searchExecutionContexts(), + context.configuration(), + context.foldCtx(), + plan, + localPlannerProfile + ); // the planner will also set the driver parallelism in LocalExecutionPlanner.LocalExecutionPlan (used down below) // it's doing this in the planning of EsQueryExec (the source of the data) // see also EsPhysicalOperationProviders.sourcePhysicalOperation diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java index fedef03799093..923d1a84dad13 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.rule; import org.elasticsearch.xpack.esql.core.tree.Node; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.function.Function; @@ -15,7 +16,7 @@ public abstract class ParameterizedRuleExecutor, private final Context context; - protected ParameterizedRuleExecutor(Context context) { + protected ParameterizedRuleExecutor(Context context, PlannerProfile profile) { this.context = context; } 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 74be10dccae7d..a79e421773472 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 @@ -51,6 +51,7 @@ import org.elasticsearch.xpack.esql.index.EsIndex; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.index.MappingException; +import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; @@ -130,7 +131,7 @@ public EsqlSession( EnrichPolicyResolver enrichPolicyResolver, PreAnalyzer preAnalyzer, EsqlFunctionRegistry functionRegistry, - LogicalPlanOptimizer logicalPlanOptimizer, + LogicalOptimizerContext logicalOptimizerContext, Mapper mapper, Verifier verifier, PlanTelemetry planTelemetry, @@ -145,10 +146,10 @@ public EsqlSession( this.verifier = verifier; this.functionRegistry = functionRegistry; this.mapper = mapper; - this.logicalPlanOptimizer = logicalPlanOptimizer; - this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); - this.planTelemetry = planTelemetry; this.plannerProfile = new PlannerProfile(false); + this.logicalPlanOptimizer = new LogicalPlanOptimizer(logicalOptimizerContext, this.plannerProfile); + this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration), this.plannerProfile); + this.planTelemetry = planTelemetry; this.indicesExpressionGrouper = indicesExpressionGrouper; this.preMapper = new PreMapper(services); } @@ -345,7 +346,7 @@ public void analyzedPlan( Function analyzeAction = (l) -> { Analyzer analyzer = new Analyzer( new AnalyzerContext(configuration, functionRegistry, l.indices, l.lookupIndices, l.enrichResolution), - verifier + verifier, plannerProfile ); LogicalPlan plan = analyzer.analyze(parsed); plan.setAnalyzed(); 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 c4b5c449f3608..a9196b13c394b 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 @@ -83,6 +83,7 @@ import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.planner.TestPhysicalOperationProviders; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; @@ -466,7 +467,11 @@ private static EnrichPolicy loadEnrichPolicyMapping(String policyFileName) { private LogicalPlan analyzedPlan(LogicalPlan parsed, CsvTestsDataLoader.MultiIndexTestDataset datasets) { var indexResolution = loadIndexResolution(datasets); var enrichPolicies = loadEnrichPolicies(); - var analyzer = new Analyzer(new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies), TEST_VERIFIER); + var analyzer = new Analyzer( + new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies), + TEST_VERIFIER, + new PlannerProfile(false) + ); LogicalPlan plan = analyzer.analyze(parsed); plan.setAnalyzed(); LOGGER.debug("Analyzed plan:\n{}", plan); @@ -536,7 +541,7 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { null, null, functionRegistry, - new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, foldCtx)), + new LogicalOptimizerContext(configuration, foldCtx), mapper, TEST_VERIFIER, new PlanTelemetry(functionRegistry), @@ -670,7 +675,10 @@ void executeSubPlan( drivers.addAll(coordinatorNodeExecutionPlan.createDrivers(getTestName())); if (dataNodePlan != null) { var searchStats = new DisabledSearchStats(); - var logicalTestOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats)); + var logicalTestOptimizer = new LocalLogicalPlanOptimizer( + new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats), + new PlannerProfile(true) + ); var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java index d4e786a9d9bb0..63f5c2acff792 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -59,7 +59,7 @@ public static Analyzer analyzer(IndexResolution indexResolution, Verifier verifi defaultLookupResolution(), defaultEnrichResolution() ), - verifier + verifier, ); } @@ -72,14 +72,14 @@ public static Analyzer analyzer(IndexResolution indexResolution, Map resolved = new ArrayList<>(); @@ -1777,7 +1777,7 @@ public void testEnrichFieldsIncludeMatchField() { ) ); AnalyzerContext context = new AnalyzerContext(configuration(query), new EsqlFunctionRegistry(), testIndex, enrichResolution); - Analyzer analyzer = new Analyzer(context, TEST_VERIFIER); + Analyzer analyzer = new Analyzer(context, TEST_VERIFIER, ); LogicalPlan plan = analyze(query, analyzer); var limit = as(plan, Limit.class); assertThat(Expressions.names(limit.output()), contains("language_name", "language_code")); @@ -2174,7 +2174,7 @@ public void testLookupJoinUnknownIndex() { Map.of("foobar", missingLookupIndex), defaultEnrichResolution() ), - TEST_VERIFIER + TEST_VERIFIER, ); String query = "FROM test | LOOKUP JOIN foobar ON last_name"; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java index e458fb009d5c3..76bc0c975bc15 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java @@ -46,7 +46,7 @@ public class ParsingTests extends ESTestCase { private final IndexResolution defaultIndex = loadIndexResolution("mapping-basic.json"); private final Analyzer defaultAnalyzer = new Analyzer( new AnalyzerContext(TEST_CFG, new EsqlFunctionRegistry(), defaultIndex, emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); public void testCaseFunctionInvalidInputs() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java index cf2de30e44456..e550ee1b54206 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java @@ -91,7 +91,7 @@ public EsqlFunctionRegistry snapshotRegistry() { private static Analyzer analyzer(EsqlFunctionRegistry registry, License.OperationMode operationMode) { return new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, registry, analyzerDefaultMapping(), defaultEnrichResolution()), - new Verifier(new Metrics(new EsqlFunctionRegistry()), getLicenseState(operationMode)) + new Verifier(new Metrics(new EsqlFunctionRegistry()), getLicenseState(operationMode)), ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java index 98f3d1d2d8d8e..44cb1064dad6a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java @@ -50,6 +50,7 @@ import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.stats.SearchStats; import org.hamcrest.Matchers; import org.junit.BeforeClass; @@ -99,7 +100,7 @@ public static void init() { analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); } @@ -403,13 +404,13 @@ public void testSparseDocument() throws Exception { var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); var analyzed = analyzer.analyze(parser.createStatement(query)); var optimized = logicalOptimizer.optimize(analyzed); var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); - var plan = new LocalLogicalPlanOptimizer(localContext).localOptimize(optimized); + var plan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true)).localOptimize(optimized); var project = as(plan, Project.class); assertThat(project.projections(), hasSize(10)); @@ -560,7 +561,7 @@ private LogicalPlan plan(String query) { private LogicalPlan localPlan(LogicalPlan plan, SearchStats searchStats) { var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); // System.out.println(plan); - var localPlan = new LocalLogicalPlanOptimizer(localContext).localOptimize(plan); + var localPlan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true)).localOptimize(plan); // System.out.println(localPlan); return localPlan; } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java index 42afeb7fadbfb..aad13fe29dfbb 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java @@ -178,7 +178,7 @@ private Analyzer makeAnalyzer(String mappingFileName, EnrichResolution enrichRes return new Analyzer( new AnalyzerContext(config, new EsqlFunctionRegistry(), getIndexResult, enrichResolution), - new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)) + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index fb3a2651b5060..e3b66292ca06e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -247,7 +247,7 @@ public static void init() { defaultLookupResolution(), enrichResolution ), - TEST_VERIFIER + TEST_VERIFIER, ); // Some tests use data from the airports index, so we load it here, and use it in the plan_airports() function. @@ -256,7 +256,7 @@ public static void init() { IndexResolution getIndexResultAirports = IndexResolution.valid(airports); analyzerAirports = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); // Some tests need additional types, so we load that index here and use it in the plan_types() function. @@ -265,7 +265,7 @@ public static void init() { IndexResolution getIndexResultTypes = IndexResolution.valid(types); analyzerTypes = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultTypes, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); // Some tests use mappings from mapping-extra.json to be able to test more types so we load it here @@ -274,14 +274,14 @@ public static void init() { IndexResolution getIndexResultExtra = IndexResolution.valid(extra); analyzerExtra = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultExtra, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); metricMapping = loadMapping("k8s-mappings.json"); var metricsIndex = IndexResolution.valid(new EsIndex("k8s", metricMapping, Map.of("k8s", IndexMode.TIME_SERIES))); metricsAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), metricsIndex, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); var multiIndexMapping = loadMapping("mapping-basic.json"); @@ -296,7 +296,7 @@ public static void init() { ); multiIndexAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), multiIndex, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); } @@ -5266,7 +5266,7 @@ public void testEmptyMappingIndex() { IndexResolution getIndexResultAirports = IndexResolution.valid(empty); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); var plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test"))); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 891070516f586..7d2e82b520d3a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -130,6 +130,7 @@ import org.elasticsearch.xpack.esql.plan.physical.UnaryExec; import org.elasticsearch.xpack.esql.planner.EsPhysicalOperationProviders; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; import org.elasticsearch.xpack.esql.plugin.QueryPragmas; @@ -356,7 +357,7 @@ TestDataSource makeTestDataSource( IndexResolution getIndexResult = IndexResolution.valid(index); Analyzer analyzer = new Analyzer( new AnalyzerContext(config, functionRegistry, getIndexResult, lookupResolution, enrichResolution), - TEST_VERIFIER + TEST_VERIFIER, ); return new TestDataSource(mapping, index, analyzer, stats); } @@ -7651,7 +7652,7 @@ private LocalExecutionPlanner.LocalExecutionPlan physicalOperationsFromPhysicalP // The TopN needs an estimated row size for the planner to work var plans = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode(EstimatesRowSize.estimateRowSize(0, plan), config); plan = useDataNodePlan ? plans.v2() : plans.v1(); - plan = PlannerUtils.localPlan(List.of(), config, FoldContext.small(), plan); + plan = PlannerUtils.localPlan(List.of(), config, FoldContext.small(), plan, new PlannerProfile(true)); ExchangeSinkHandler exchangeSinkHandler = new ExchangeSinkHandler(null, 10, () -> 10); LocalExecutionPlanner planner = new LocalExecutionPlanner( "test", @@ -7920,7 +7921,7 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { // individually hence why here the plan is kept as is var l = p.transformUp(FragmentExec.class, fragment -> { - var localPlan = PlannerUtils.localPlan(config, FoldContext.small(), fragment, searchStats); + var localPlan = PlannerUtils.localPlan(config, FoldContext.small(), fragment, searchStats, new PlannerProfile(true)); return EstimatesRowSize.estimateRowSize(fragment.estimatedRowSize(), localPlan); }); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java index e6a7d110f8c09..385ecf1b1c2a1 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java @@ -13,6 +13,7 @@ import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; import org.elasticsearch.xpack.esql.session.Configuration; @@ -63,7 +64,7 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { // individually hence why here the plan is kept as is var logicalTestOptimizer = new LocalLogicalPlanOptimizer( - new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats) + new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats), new PlannerProfile(true) ); var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( new LocalPhysicalOptimizerContext(config, FoldContext.small(), searchStats), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java index 8f2063146cabd..de43d5ebca313 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java @@ -65,7 +65,7 @@ public static void init() { defaultLookupResolution(), new EnrichResolution() ), - TEST_VERIFIER + TEST_VERIFIER, ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java index 90f25db232ec7..bd79660a8f707 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java @@ -84,7 +84,7 @@ public static void init() { analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java index f9732272dbd74..9278e72c7073a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java @@ -47,7 +47,7 @@ private static Analyzer makeAnalyzer(String mappingFileName) { return new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, new EnrichResolution()), - new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)) + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java index e58824290c49e..18d84535d3c2c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java @@ -192,7 +192,7 @@ static LogicalPlan parse(String query) { var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java index fac3495697da8..314323df77afd 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java @@ -293,7 +293,7 @@ static LogicalPlan parse(String query) { var logicalOptimizer = new LogicalPlanOptimizer(new LogicalOptimizerContext(TEST_CFG, FoldContext.small())); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), - TEST_VERIFIER + TEST_VERIFIER, ); return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); } From 41ac4d46da8e7a6e8b528cbe77feb678c90a3934 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Tue, 18 Mar 2025 11:35:12 -0400 Subject: [PATCH 13/20] push the profile one layer deeper into the runner --- .../xpack/esql/rule/ParameterizedRuleExecutor.java | 1 + .../org/elasticsearch/xpack/esql/rule/RuleExecutor.java | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java index 923d1a84dad13..ee4f5a67af020 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutor.java @@ -17,6 +17,7 @@ public abstract class ParameterizedRuleExecutor, private final Context context; protected ParameterizedRuleExecutor(Context context, PlannerProfile profile) { + super(profile); this.context = context; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java index 7df5a029d724e..4016d6d696085 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java @@ -11,6 +11,7 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.core.tree.NodeUtils; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -21,6 +22,7 @@ public abstract class RuleExecutor> { private final Logger log = LogManager.getLogger(getClass()); + private final PlannerProfile profile; public static class Limiter { public static final Limiter DEFAULT = new Limiter(100); @@ -211,6 +213,10 @@ protected final ExecutionInfo executeWithInfo(TreeType plan) { return new ExecutionInfo(plan, currentPlan, transformations); } + protected RuleExecutor(PlannerProfile profile) { + this.profile = profile; + } + protected Function transform(Rule rule) { return rule::apply; } From 9bc955066c2e0197c17e587a92f2fcf8d43ac295 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Tue, 18 Mar 2025 13:25:48 -0400 Subject: [PATCH 14/20] add new parameters everywhere --- .../xpack/esql/execution/PlanExecutor.java | 12 ++++++------ .../xpack/esql/planner/PlannerProfile.java | 10 +++++++--- .../xpack/esql/plugin/ComputeService.java | 3 ++- .../xpack/esql/plugin/EsqlPlugin.java | 8 +++++++- .../xpack/esql/session/EsqlSession.java | 3 ++- .../elasticsearch/xpack/esql/CsvTests.java | 9 ++++----- .../esql/analysis/AnalyzerTestUtils.java | 9 +++++---- .../xpack/esql/analysis/AnalyzerTests.java | 7 ++++--- .../xpack/esql/analysis/ParsingTests.java | 3 ++- .../function/CheckLicenseTests.java | 3 ++- .../LocalLogicalPlanOptimizerTests.java | 12 ++++++------ .../LocalPhysicalPlanOptimizerTests.java | 5 +++-- .../optimizer/LogicalPlanOptimizerTests.java | 19 ++++++++++--------- .../optimizer/PhysicalPlanOptimizerTests.java | 10 +++++----- .../TestLocalPhysicalPlanOptimizer.java | 3 ++- .../optimizer/TestPhysicalPlanOptimizer.java | 3 ++- .../esql/optimizer/TestPlannerOptimizer.java | 11 ++++++++--- .../logical/PropagateInlineEvalsTests.java | 3 ++- .../xpack/esql/planner/FilterTests.java | 8 ++++++-- .../esql/planner/QueryTranslatorTests.java | 2 +- .../esql/plugin/ClusterRequestTests.java | 5 +++-- .../DataNodeRequestSerializationTests.java | 9 +++++++-- .../telemetry/PlanExecutorMetricsTests.java | 2 +- 23 files changed, 97 insertions(+), 62 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java index 4a3153722bc53..64d5c10a906ad 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java @@ -19,7 +19,6 @@ import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolver; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.Configuration; @@ -42,8 +41,9 @@ public class PlanExecutor { private final Metrics metrics; private final Verifier verifier; private final PlanTelemetryManager planTelemetryManager; + private final String nodeName; - public PlanExecutor(IndexResolver indexResolver, MeterRegistry meterRegistry, XPackLicenseState licenseState) { + public PlanExecutor(IndexResolver indexResolver, MeterRegistry meterRegistry, XPackLicenseState licenseState, String nodeName) { this.indexResolver = indexResolver; this.preAnalyzer = new PreAnalyzer(); this.functionRegistry = new EsqlFunctionRegistry(); @@ -51,6 +51,7 @@ public PlanExecutor(IndexResolver indexResolver, MeterRegistry meterRegistry, XP this.metrics = new Metrics(functionRegistry); this.verifier = new Verifier(metrics, licenseState); this.planTelemetryManager = new PlanTelemetryManager(meterRegistry); + this.nodeName = nodeName; } public void esql( @@ -63,11 +64,11 @@ public void esql( IndicesExpressionGrouper indicesExpressionGrouper, EsqlSession.PlanRunner planRunner, TransportActionServices services, - ActionListener listener - ) { + ActionListener listener) { final PlanTelemetry planTelemetry = new PlanTelemetry(functionRegistry); final var session = new EsqlSession( sessionId, + nodeName, cfg, indexResolver, enrichPolicyResolver, @@ -78,8 +79,7 @@ public void esql( verifier, planTelemetry, indicesExpressionGrouper, - services - ); + services); QueryMetric clientId = QueryMetric.fromString("rest"); metrics.total(clientId); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 966ea86e34c14..0296726d97595 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -22,22 +22,26 @@ */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { - public static final PlannerProfile EMPTY = new PlannerProfile(false); + public static final PlannerProfile EMPTY = new PlannerProfile(false, ""); private final boolean isLocalPlanning; + private final String nodeName; public static PlannerProfile readFrom(StreamInput in) throws IOException { boolean isLocalPlanning = in.readBoolean(); - return new PlannerProfile(isLocalPlanning); + String nodeName = in.readString(); + return new PlannerProfile(isLocalPlanning, nodeName); } - public PlannerProfile(boolean isLocalPlanning) { + public PlannerProfile(boolean isLocalPlanning, String nodeName) { this.isLocalPlanning = isLocalPlanning; + this.nodeName = nodeName; } @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(isLocalPlanning); + out.writeString(nodeName); } @Override 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 f9beed32a6335..03488f3832c62 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 @@ -27,6 +27,7 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; +import org.elasticsearch.node.Node; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SourceProvider; @@ -360,7 +361,7 @@ public SourceProvider createSourceProvider() { ); } final List drivers; - final PlannerProfile localPlannerProfile = new PlannerProfile(true); + final PlannerProfile localPlannerProfile = new PlannerProfile(true, Node.NODE_NAME_SETTING.get(clusterService.getSettings())); try { LocalExecutionPlanner planner = new LocalExecutionPlanner( context.sessionId(), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java index c864c4b99d6d8..48b80b43f16ca 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java @@ -39,6 +39,7 @@ import org.elasticsearch.compute.operator.topn.TopNOperatorStatus; import org.elasticsearch.features.NodeFeature; import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.node.Node; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestController; @@ -121,7 +122,12 @@ public Collection createComponents(PluginServices services) { var blockFactoryProvider = blockFactoryProvider(circuitBreaker, bigArrays, maxPrimitiveArrayBlockSize); setupSharedSecrets(); return List.of( - new PlanExecutor(new IndexResolver(services.client()), services.telemetryProvider().getMeterRegistry(), getLicenseState()), + new PlanExecutor( + new IndexResolver(services.client()), + services.telemetryProvider().getMeterRegistry(), + getLicenseState(), + Node.NODE_NAME_SETTING.get(settings) + ), new ExchangeService( services.clusterService().getSettings(), services.threadPool(), 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 774b565bde229..5ba865c63da41 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 @@ -126,6 +126,7 @@ public interface PlanRunner { public EsqlSession( String sessionId, + String nodeName, Configuration configuration, IndexResolver indexResolver, EnrichPolicyResolver enrichPolicyResolver, @@ -146,7 +147,7 @@ public EsqlSession( this.verifier = verifier; this.functionRegistry = functionRegistry; this.mapper = mapper; - this.plannerProfile = new PlannerProfile(false); + this.plannerProfile = new PlannerProfile(false, nodeName); this.logicalPlanOptimizer = new LogicalPlanOptimizer(logicalOptimizerContext, this.plannerProfile); this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration), this.plannerProfile); this.planTelemetry = planTelemetry; 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 a9196b13c394b..e36554b77e45f 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 @@ -70,7 +70,6 @@ import org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer; import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.optimizer.TestLocalPhysicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.logical.Enrich; @@ -470,7 +469,7 @@ private LogicalPlan analyzedPlan(LogicalPlan parsed, CsvTestsDataLoader.MultiInd var analyzer = new Analyzer( new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies), TEST_VERIFIER, - new PlannerProfile(false) + new PlannerProfile(false, "") ); LogicalPlan plan = analyzer.analyze(parsed); plan.setAnalyzed(); @@ -536,6 +535,7 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { FoldContext foldCtx = FoldContext.small(); EsqlSession session = new EsqlSession( getTestName(), + "nodeName", configuration, null, null, @@ -546,8 +546,7 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { TEST_VERIFIER, new PlanTelemetry(functionRegistry), null, - EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES - ); + EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES); TestPhysicalOperationProviders physicalOperationProviders = testOperationProviders(foldCtx, testDatasets); PlainActionFuture listener = new PlainActionFuture<>(); @@ -677,7 +676,7 @@ void executeSubPlan( var searchStats = new DisabledSearchStats(); var logicalTestOptimizer = new LocalLogicalPlanOptimizer( new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats), - new PlannerProfile(true) + new PlannerProfile(true, "") ); var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java index 63f5c2acff792..a78830293e562 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -18,6 +18,7 @@ import org.elasticsearch.xpack.esql.parser.QueryParams; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.session.Configuration; import java.util.ArrayList; @@ -59,7 +60,7 @@ public static Analyzer analyzer(IndexResolution indexResolution, Verifier verifi defaultLookupResolution(), defaultEnrichResolution() ), - verifier, + verifier, new PlannerProfile(false, "") ); } @@ -72,14 +73,14 @@ public static Analyzer analyzer(IndexResolution indexResolution, Map resolved = new ArrayList<>(); @@ -1777,7 +1778,7 @@ public void testEnrichFieldsIncludeMatchField() { ) ); AnalyzerContext context = new AnalyzerContext(configuration(query), new EsqlFunctionRegistry(), testIndex, enrichResolution); - Analyzer analyzer = new Analyzer(context, TEST_VERIFIER, ); + Analyzer analyzer = new Analyzer(context, TEST_VERIFIER, new PlannerProfile(false, "")); LogicalPlan plan = analyze(query, analyzer); var limit = as(plan, Limit.class); assertThat(Expressions.names(limit.output()), contains("language_name", "language_code")); @@ -2174,7 +2175,7 @@ public void testLookupJoinUnknownIndex() { Map.of("foobar", missingLookupIndex), defaultEnrichResolution() ), - TEST_VERIFIER, + TEST_VERIFIER,new PlannerProfile(false, "") ); String query = "FROM test | LOOKUP JOIN foobar ON last_name"; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java index 76bc0c975bc15..759466234526f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.xpack.esql.parser.ParsingException; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; import java.io.IOException; @@ -46,7 +47,7 @@ public class ParsingTests extends ESTestCase { private final IndexResolution defaultIndex = loadIndexResolution("mapping-basic.json"); private final Analyzer defaultAnalyzer = new Analyzer( new AnalyzerContext(TEST_CFG, new EsqlFunctionRegistry(), defaultIndex, emptyPolicyResolution()), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); public void testCaseFunctionInvalidInputs() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java index e550ee1b54206..bc7473e111d6f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.logical.Limit; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.telemetry.Metrics; import java.util.List; @@ -91,7 +92,7 @@ public EsqlFunctionRegistry snapshotRegistry() { private static Analyzer analyzer(EsqlFunctionRegistry registry, License.OperationMode operationMode) { return new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, registry, analyzerDefaultMapping(), defaultEnrichResolution()), - new Verifier(new Metrics(new EsqlFunctionRegistry()), getLicenseState(operationMode)), + new Verifier(new Metrics(new EsqlFunctionRegistry()), getLicenseState(operationMode)), new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java index 44cb1064dad6a..70979154a7719 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java @@ -96,11 +96,11 @@ public static void init() { mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); IndexResolution getIndexResult = IndexResolution.valid(test); - logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); + logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); } @@ -400,17 +400,17 @@ public void testSparseDocument() throws Exception { EsIndex index = new EsIndex("large", large, Map.of("large", IndexMode.STANDARD)); IndexResolution getIndexResult = IndexResolution.valid(index); - var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); + var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), - TEST_VERIFIER, + TEST_VERIFIER,new PlannerProfile(false, "") ); var analyzed = analyzer.analyze(parser.createStatement(query)); var optimized = logicalOptimizer.optimize(analyzed); var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); - var plan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true)).localOptimize(optimized); + var plan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true, "")).localOptimize(optimized); var project = as(plan, Project.class); assertThat(project.projections(), hasSize(10)); @@ -561,7 +561,7 @@ private LogicalPlan plan(String query) { private LogicalPlan localPlan(LogicalPlan plan, SearchStats searchStats) { var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); // System.out.println(plan); - var localPlan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true)).localOptimize(plan); + var localPlan = new LocalLogicalPlanOptimizer(localContext, new PlannerProfile(true, "")).localOptimize(plan); // System.out.println(localPlan); return localPlan; } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java index aad13fe29dfbb..19866e90998ff 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java @@ -70,6 +70,7 @@ import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; import org.elasticsearch.xpack.esql.plan.physical.TopNExec; import org.elasticsearch.xpack.esql.planner.FilterTests; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.plugin.QueryPragmas; import org.elasticsearch.xpack.esql.querydsl.query.SingleValueQuery; import org.elasticsearch.xpack.esql.rule.Rule; @@ -178,7 +179,7 @@ private Analyzer makeAnalyzer(String mappingFileName, EnrichResolution enrichRes return new Analyzer( new AnalyzerContext(config, new EsqlFunctionRegistry(), getIndexResult, enrichResolution), - new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), new PlannerProfile(false, "") ); } @@ -414,7 +415,7 @@ public void testMultiCountAllWithFilter() { @SuppressWarnings("unchecked") public void testSingleCountWithStatsFilter() { // an optimizer that filters out the ExtractAggregateCommonFilter rule - var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()) { + var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")) { @Override protected List> batches() { var oldBatches = super.batches(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index e3b66292ca06e..19794141e6a5e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -129,6 +129,7 @@ import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.junit.BeforeClass; import java.util.ArrayList; @@ -218,7 +219,7 @@ public static class SubstitutionOnlyOptimizer extends LogicalPlanOptimizer { public static SubstitutionOnlyOptimizer INSTANCE = new SubstitutionOnlyOptimizer(unboundLogicalOptimizerContext()); SubstitutionOnlyOptimizer(LogicalOptimizerContext optimizerContext) { - super(optimizerContext); + super(optimizerContext, new PlannerProfile(false, "")); } @Override @@ -231,7 +232,7 @@ protected List> batches() { public static void init() { parser = new EsqlParser(); logicalOptimizerCtx = unboundLogicalOptimizerContext(); - logicalOptimizer = new LogicalPlanOptimizer(logicalOptimizerCtx); + logicalOptimizer = new LogicalPlanOptimizer(logicalOptimizerCtx, new PlannerProfile(false, "")); enrichResolution = new EnrichResolution(); AnalyzerTestUtils.loadEnrichPolicyResolution(enrichResolution, "languages_idx", "id", "languages_idx", "mapping-languages.json"); @@ -247,7 +248,7 @@ public static void init() { defaultLookupResolution(), enrichResolution ), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); // Some tests use data from the airports index, so we load it here, and use it in the plan_airports() function. @@ -256,7 +257,7 @@ public static void init() { IndexResolution getIndexResultAirports = IndexResolution.valid(airports); analyzerAirports = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); // Some tests need additional types, so we load that index here and use it in the plan_types() function. @@ -265,7 +266,7 @@ public static void init() { IndexResolution getIndexResultTypes = IndexResolution.valid(types); analyzerTypes = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultTypes, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); // Some tests use mappings from mapping-extra.json to be able to test more types so we load it here @@ -274,14 +275,14 @@ public static void init() { IndexResolution getIndexResultExtra = IndexResolution.valid(extra); analyzerExtra = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultExtra, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); metricMapping = loadMapping("k8s-mappings.json"); var metricsIndex = IndexResolution.valid(new EsIndex("k8s", metricMapping, Map.of("k8s", IndexMode.TIME_SERIES))); metricsAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), metricsIndex, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); var multiIndexMapping = loadMapping("mapping-basic.json"); @@ -296,7 +297,7 @@ public static void init() { ); multiIndexAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), multiIndex, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); } @@ -5266,7 +5267,7 @@ public void testEmptyMappingIndex() { IndexResolution getIndexResultAirports = IndexResolution.valid(empty); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); var plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test"))); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 7d2e82b520d3a..13c3a4905d6d4 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -252,8 +252,8 @@ public PhysicalPlanOptimizerTests(String name, Configuration config) { @Before public void init() { parser = new EsqlParser(); - logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); - physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config)); + logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); + physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config), new PlannerProfile(false, "")); EsqlFunctionRegistry functionRegistry = new EsqlFunctionRegistry(); mapper = new Mapper(); var enrichResolution = setupEnrichResolution(); @@ -357,7 +357,7 @@ TestDataSource makeTestDataSource( IndexResolution getIndexResult = IndexResolution.valid(index); Analyzer analyzer = new Analyzer( new AnalyzerContext(config, functionRegistry, getIndexResult, lookupResolution, enrichResolution), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); return new TestDataSource(mapping, index, analyzer, stats); } @@ -7652,7 +7652,7 @@ private LocalExecutionPlanner.LocalExecutionPlan physicalOperationsFromPhysicalP // The TopN needs an estimated row size for the planner to work var plans = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode(EstimatesRowSize.estimateRowSize(0, plan), config); plan = useDataNodePlan ? plans.v2() : plans.v1(); - plan = PlannerUtils.localPlan(List.of(), config, FoldContext.small(), plan, new PlannerProfile(true)); + plan = PlannerUtils.localPlan(List.of(), config, FoldContext.small(), plan, new PlannerProfile(true, "")); ExchangeSinkHandler exchangeSinkHandler = new ExchangeSinkHandler(null, 10, () -> 10); LocalExecutionPlanner planner = new LocalExecutionPlanner( "test", @@ -7921,7 +7921,7 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { // individually hence why here the plan is kept as is var l = p.transformUp(FragmentExec.class, fragment -> { - var localPlan = PlannerUtils.localPlan(config, FoldContext.small(), fragment, searchStats, new PlannerProfile(true)); + var localPlan = PlannerUtils.localPlan(config, FoldContext.small(), fragment, searchStats, new PlannerProfile(true, "")); return EstimatesRowSize.estimateRowSize(fragment.estimatedRowSize(), localPlan); }); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java index 0d1e85171bac3..4fc8b3d6c05ff 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.optimizer; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.util.List; @@ -20,7 +21,7 @@ public TestLocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context) { } public TestLocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context, boolean esRules) { - super(context); + super(context, new PlannerProfile(true, "")); this.esRules = esRules; } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java index e26779e075b68..df3cacd4806f4 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.optimizer; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.rule.RuleExecutor; public class TestPhysicalPlanOptimizer extends PhysicalPlanOptimizer { @@ -15,7 +16,7 @@ public class TestPhysicalPlanOptimizer extends PhysicalPlanOptimizer { private static final Iterable> rules = initializeRules(false); public TestPhysicalPlanOptimizer(PhysicalOptimizerContext context) { - super(context); + super(context, new PlannerProfile(false, "")); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java index 385ecf1b1c2a1..c566102a5a198 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPlannerOptimizer.java @@ -28,7 +28,11 @@ public class TestPlannerOptimizer { private final Configuration config; public TestPlannerOptimizer(Configuration config, Analyzer analyzer) { - this(config, analyzer, new LogicalPlanOptimizer(new LogicalOptimizerContext(config, FoldContext.small()))); + this( + config, + analyzer, + new LogicalPlanOptimizer(new LogicalOptimizerContext(config, FoldContext.small()), new PlannerProfile(false, "")) + ); } public TestPlannerOptimizer(Configuration config, Analyzer analyzer, LogicalPlanOptimizer logicalOptimizer) { @@ -37,7 +41,7 @@ public TestPlannerOptimizer(Configuration config, Analyzer analyzer, LogicalPlan this.logicalOptimizer = logicalOptimizer; parser = new EsqlParser(); - physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config)); + physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config), new PlannerProfile(false, "")); mapper = new Mapper(); } @@ -64,7 +68,8 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { // individually hence why here the plan is kept as is var logicalTestOptimizer = new LocalLogicalPlanOptimizer( - new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats), new PlannerProfile(true) + new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats), + new PlannerProfile(true, "") ); var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( new LocalPhysicalOptimizerContext(config, FoldContext.small(), searchStats), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java index de43d5ebca313..93919f75d1ec6 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.xpack.esql.plan.logical.join.InlineJoin; import org.elasticsearch.xpack.esql.plan.logical.join.StubRelation; import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.junit.BeforeClass; import java.util.List; @@ -65,7 +66,7 @@ public static void init() { defaultLookupResolution(), new EnrichResolution() ), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java index bd79660a8f707..283fdc60f544c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java @@ -78,13 +78,17 @@ public static void init() { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); IndexResolution getIndexResult = IndexResolution.valid(test); - logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); - physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(EsqlTestUtils.TEST_CFG)); + logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); + physicalPlanOptimizer = new PhysicalPlanOptimizer( + new PhysicalOptimizerContext(EsqlTestUtils.TEST_CFG), + new PlannerProfile(false, "") + ); mapper = new Mapper(); analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), TEST_VERIFIER, + new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java index 9278e72c7073a..44c02d8a3dab1 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java @@ -47,7 +47,7 @@ private static Analyzer makeAnalyzer(String mappingFileName) { return new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, new EnrichResolution()), - new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java index 18d84535d3c2c..749ca51457716 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import java.io.IOException; import java.util.ArrayList; @@ -189,10 +190,10 @@ static LogicalPlan parse(String query) { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); IndexResolution getIndexResult = IndexResolution.valid(test); - var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); + var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), - TEST_VERIFIER, + TEST_VERIFIER, new PlannerProfile(false, "") ); return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java index 314323df77afd..a68fd1ba9700e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerProfile; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; import java.io.IOException; @@ -290,16 +291,20 @@ static LogicalPlan parse(String query) { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); IndexResolution getIndexResult = IndexResolution.valid(test); - var logicalOptimizer = new LogicalPlanOptimizer(new LogicalOptimizerContext(TEST_CFG, FoldContext.small())); + var logicalOptimizer = new LogicalPlanOptimizer( + new LogicalOptimizerContext(TEST_CFG, FoldContext.small()), + new PlannerProfile(false, "") + ); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), TEST_VERIFIER, + new PlannerProfile(false, "") ); return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); } static PhysicalPlan mapAndMaybeOptimize(LogicalPlan logicalPlan) { - var physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(TEST_CFG)); + var physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(TEST_CFG), new PlannerProfile(false, "")); var mapper = new Mapper(); var physical = mapper.map(logicalPlan); if (randomBoolean()) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java index 6c3995302767f..e490ef68b41e7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java @@ -105,7 +105,7 @@ public void testFailedMetric() { return null; }).when(esqlClient).execute(eq(EsqlResolveFieldsAction.TYPE), any(), any()); - var planExecutor = new PlanExecutor(indexResolver, MeterRegistry.NOOP, new XPackLicenseState(() -> 0L)); + var planExecutor = new PlanExecutor(indexResolver, MeterRegistry.NOOP, new XPackLicenseState(() -> 0L), ""); var enrichResolver = mockEnrichResolver(); var request = new EsqlQueryRequest(); From 0530d409a612be976bd9cadf61634ff92c9da5c0 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 18 Mar 2025 17:32:59 +0000 Subject: [PATCH 15/20] [CI] Auto commit changes from spotless --- .../xpack/esql/execution/PlanExecutor.java | 6 ++++-- .../xpack/esql/session/EsqlSession.java | 3 ++- .../elasticsearch/xpack/esql/CsvTests.java | 3 ++- .../esql/analysis/AnalyzerTestUtils.java | 12 +++++++---- .../xpack/esql/analysis/AnalyzerTests.java | 3 ++- .../xpack/esql/analysis/ParsingTests.java | 3 ++- .../function/CheckLicenseTests.java | 3 ++- .../LocalLogicalPlanOptimizerTests.java | 6 ++++-- .../LocalPhysicalPlanOptimizerTests.java | 3 ++- .../optimizer/LogicalPlanOptimizerTests.java | 21 ++++++++++++------- .../optimizer/PhysicalPlanOptimizerTests.java | 3 ++- .../logical/PropagateInlineEvalsTests.java | 3 ++- .../esql/planner/QueryTranslatorTests.java | 3 ++- .../esql/plugin/ClusterRequestTests.java | 3 ++- 14 files changed, 50 insertions(+), 25 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java index 64d5c10a906ad..02c80296ba381 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java @@ -64,7 +64,8 @@ public void esql( IndicesExpressionGrouper indicesExpressionGrouper, EsqlSession.PlanRunner planRunner, TransportActionServices services, - ActionListener listener) { + ActionListener listener + ) { final PlanTelemetry planTelemetry = new PlanTelemetry(functionRegistry); final var session = new EsqlSession( sessionId, @@ -79,7 +80,8 @@ public void esql( verifier, planTelemetry, indicesExpressionGrouper, - services); + services + ); QueryMetric clientId = QueryMetric.fromString("rest"); metrics.total(clientId); 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 5ba865c63da41..6e095d1a1d7a3 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 @@ -345,7 +345,8 @@ public void analyzedPlan( Function analyzeAction = (l) -> { Analyzer analyzer = new Analyzer( new AnalyzerContext(configuration, functionRegistry, l.indices, l.lookupIndices, l.enrichResolution), - verifier, plannerProfile + verifier, + plannerProfile ); LogicalPlan plan = analyzer.analyze(parsed); plan.setAnalyzed(); 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 e36554b77e45f..5e56d6ccc54f1 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 @@ -546,7 +546,8 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { TEST_VERIFIER, new PlanTelemetry(functionRegistry), null, - EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES); + EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES + ); TestPhysicalOperationProviders physicalOperationProviders = testOperationProviders(foldCtx, testDatasets); PlainActionFuture listener = new PlainActionFuture<>(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java index a78830293e562..2ca606260a93f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -60,7 +60,8 @@ public static Analyzer analyzer(IndexResolution indexResolution, Verifier verifi defaultLookupResolution(), defaultEnrichResolution() ), - verifier, new PlannerProfile(false, "") + verifier, + new PlannerProfile(false, "") ); } @@ -73,14 +74,16 @@ public static Analyzer analyzer(IndexResolution indexResolution, Map 0L)), new PlannerProfile(false, "") + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), + new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index 19794141e6a5e..833e901d0ad82 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -248,7 +248,8 @@ public static void init() { defaultLookupResolution(), enrichResolution ), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); // Some tests use data from the airports index, so we load it here, and use it in the plan_airports() function. @@ -257,7 +258,8 @@ TEST_VERIFIER, new PlannerProfile(false, "") IndexResolution getIndexResultAirports = IndexResolution.valid(airports); analyzerAirports = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); // Some tests need additional types, so we load that index here and use it in the plan_types() function. @@ -266,7 +268,8 @@ TEST_VERIFIER, new PlannerProfile(false, "") IndexResolution getIndexResultTypes = IndexResolution.valid(types); analyzerTypes = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultTypes, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); // Some tests use mappings from mapping-extra.json to be able to test more types so we load it here @@ -275,14 +278,16 @@ TEST_VERIFIER, new PlannerProfile(false, "") IndexResolution getIndexResultExtra = IndexResolution.valid(extra); analyzerExtra = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultExtra, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); metricMapping = loadMapping("k8s-mappings.json"); var metricsIndex = IndexResolution.valid(new EsIndex("k8s", metricMapping, Map.of("k8s", IndexMode.TIME_SERIES))); metricsAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), metricsIndex, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); var multiIndexMapping = loadMapping("mapping-basic.json"); @@ -297,7 +302,8 @@ TEST_VERIFIER, new PlannerProfile(false, "") ); multiIndexAnalyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), multiIndex, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); } @@ -5267,7 +5273,8 @@ public void testEmptyMappingIndex() { IndexResolution getIndexResultAirports = IndexResolution.valid(empty); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResultAirports, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); var plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test"))); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 13c3a4905d6d4..71e622f3171dc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -357,7 +357,8 @@ TestDataSource makeTestDataSource( IndexResolution getIndexResult = IndexResolution.valid(index); Analyzer analyzer = new Analyzer( new AnalyzerContext(config, functionRegistry, getIndexResult, lookupResolution, enrichResolution), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); return new TestDataSource(mapping, index, analyzer, stats); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java index 93919f75d1ec6..59af39c633c4e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java @@ -66,7 +66,8 @@ public static void init() { defaultLookupResolution(), new EnrichResolution() ), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java index 44c02d8a3dab1..4f5d2ea93abfb 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java @@ -47,7 +47,8 @@ private static Analyzer makeAnalyzer(String mappingFileName) { return new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, new EnrichResolution()), - new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), new PlannerProfile(false, "") + new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)), + new PlannerProfile(false, "") ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java index 749ca51457716..0b53d837cdad2 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java @@ -193,7 +193,8 @@ static LogicalPlan parse(String query) { var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext(), new PlannerProfile(false, "")); var analyzer = new Analyzer( new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), - TEST_VERIFIER, new PlannerProfile(false, "") + TEST_VERIFIER, + new PlannerProfile(false, "") ); return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); } From aa6b085675ff3755555897d23fbf3ebb155e2b02 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Thu, 20 Mar 2025 13:12:00 -0400 Subject: [PATCH 16/20] rule layout draft --- .../xpack/esql/planner/PlannerProfile.java | 38 ++++++++++++++++--- 1 file changed, 33 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 0296726d97595..2e8c9c100ecab 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -12,6 +12,8 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ChunkedToXContentObject; import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.util.Iterator; @@ -22,22 +24,48 @@ */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { + public record RuleProfile(String batchName, String ruleName, long durationNanos, int runs, int runsWithChanges) implements Writeable, ToXContentObject { + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(batchName); + out.writeString(ruleName); + out.writeLong(durationNanos); + out.writeInt(runs); + out.writeInt(runsWithChanges); + } + + public RuleProfile readFrom(StreamInput in) throws IOException { + String batchName = in.readString(); + String ruleName = in.readString(); + long durationNanos = in.readLong(); + int runs = in.readInt(); + int runsWithChanges = in.readInt(); + return new RuleProfile(batchName, ruleName, durationNanos, runs, runsWithChanges); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + // NOCOMMIT + return null; + } + } + public static final PlannerProfile EMPTY = new PlannerProfile(false, ""); private final boolean isLocalPlanning; private final String nodeName; + public PlannerProfile(boolean isLocalPlanning, String nodeName) { + this.isLocalPlanning = isLocalPlanning; + this.nodeName = nodeName; + } + public static PlannerProfile readFrom(StreamInput in) throws IOException { boolean isLocalPlanning = in.readBoolean(); String nodeName = in.readString(); return new PlannerProfile(isLocalPlanning, nodeName); } - public PlannerProfile(boolean isLocalPlanning, String nodeName) { - this.isLocalPlanning = isLocalPlanning; - this.nodeName = nodeName; - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(isLocalPlanning); From 6658b960f70f92cfb4f67d03bf8f2db3bfafe55a Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Thu, 27 Mar 2025 15:15:09 +0000 Subject: [PATCH 17/20] [CI] Auto commit changes from spotless --- .../elasticsearch/xpack/esql/execution/PlanExecutor.java | 8 +++++++- .../elasticsearch/xpack/esql/planner/PlannerProfile.java | 5 ++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java index 6bc2d9a90df9d..a164247065724 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java @@ -45,7 +45,13 @@ public class PlanExecutor { private final String nodeName; private final EsqlQueryLog queryLog; - public PlanExecutor(IndexResolver indexResolver, MeterRegistry meterRegistry, XPackLicenseState licenseState, EsqlQueryLog queryLog, String nodeName) { + public PlanExecutor( + IndexResolver indexResolver, + MeterRegistry meterRegistry, + XPackLicenseState licenseState, + EsqlQueryLog queryLog, + String nodeName + ) { this.indexResolver = indexResolver; this.preAnalyzer = new PreAnalyzer(); this.functionRegistry = new EsqlFunctionRegistry(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 2e8c9c100ecab..1a2aa1e49e4d7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -24,7 +24,10 @@ */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { - public record RuleProfile(String batchName, String ruleName, long durationNanos, int runs, int runsWithChanges) implements Writeable, ToXContentObject { + public record RuleProfile(String batchName, String ruleName, long durationNanos, int runs, int runsWithChanges) + implements + Writeable, + ToXContentObject { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(batchName); From b9c07ee7dfd201efeabb29824e805756247976f3 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Thu, 27 Mar 2025 13:51:50 -0400 Subject: [PATCH 18/20] fix some merge errors --- .../xpack/esql/plugin/TransportEsqlQueryAction.java | 1 - .../xpack/esql/planner/PlanConcurrencyCalculatorTests.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) 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 df3b1fddee02b..1e48db703b2ab 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 @@ -316,7 +316,6 @@ 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(); List columns = result.schema().stream().map(c -> { List originalTypes; if (c.originalTypes() == null) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java index da36b42d1241b..7911f9c44af35 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java @@ -239,10 +239,10 @@ private void assertConcurrency(String query, Integer concurrencyPragmaValue, Int Analyzer analyzer = analyzer(analyzerDefaultMapping(), TEST_VERIFIER, configuration); LogicalPlan logicalPlan = AnalyzerTestUtils.analyze(query, analyzer); - logicalPlan = new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, FoldContext.small())).optimize(logicalPlan); + logicalPlan = new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, FoldContext.small()), PlannerProfile.EMPTY).optimize(logicalPlan); PhysicalPlan physicalPlan = new Mapper().map(logicalPlan); - physicalPlan = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)).optimize(physicalPlan); + physicalPlan = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration), PlannerProfile.EMPTY).optimize(physicalPlan); PhysicalPlan dataNodePlan = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode(physicalPlan, configuration).v2(); From 229c11db86c36ed0dc14b3c78b57613ae975eb27 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 31 Mar 2025 08:37:20 -0400 Subject: [PATCH 19/20] lay out data structures for profile --- .../xpack/esql/planner/PlannerProfile.java | 60 ++++++++++++++----- .../xpack/esql/rule/RuleExecutor.java | 2 + 2 files changed, 47 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 2e8c9c100ecab..516608943e953 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -16,7 +16,12 @@ import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; /** * Stores profiling information about the query plan. This can be the top level planning on the coordinating node, or the local @@ -24,30 +29,46 @@ */ public class PlannerProfile implements Writeable, ChunkedToXContentObject { - public record RuleProfile(String batchName, String ruleName, long durationNanos, int runs, int runsWithChanges) implements Writeable, ToXContentObject { + public static final class RuleProfile implements Writeable, ChunkedToXContentObject { + private final List runDurations; + private int runsWithChanges; + + public RuleProfile() { + this.runDurations = new ArrayList<>(); + this.runsWithChanges = 0; + } + + public RuleProfile(List durationNanos, int runsWithChanges) { + this.runDurations = durationNanos; + this.runsWithChanges = runsWithChanges; + } + @Override public void writeTo(StreamOutput out) throws IOException { - out.writeString(batchName); - out.writeString(ruleName); - out.writeLong(durationNanos); - out.writeInt(runs); + out.writeCollection(runDurations, StreamOutput::writeLong); out.writeInt(runsWithChanges); } public RuleProfile readFrom(StreamInput in) throws IOException { - String batchName = in.readString(); - String ruleName = in.readString(); - long durationNanos = in.readLong(); - int runs = in.readInt(); + List runDurations = in.readCollectionAsImmutableList(StreamInput::readLong); int runsWithChanges = in.readInt(); - return new RuleProfile(batchName, ruleName, durationNanos, runs, runsWithChanges); + return new RuleProfile(runDurations, runsWithChanges); + } + + public void addRun(long runDuration, boolean hasChanges) { + runDurations.add(runDuration); + if (hasChanges) { + runsWithChanges++; + } } @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + public Iterator toXContentChunked(ToXContent.Params params) { // NOCOMMIT return null; } + + // NOCOMMIT equals and hashcode, once the fields are stable } public static final PlannerProfile EMPTY = new PlannerProfile(false, ""); @@ -55,21 +76,30 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws private final boolean isLocalPlanning; private final String nodeName; + private Map> ruleProfiles; + public PlannerProfile(boolean isLocalPlanning, String nodeName) { this.isLocalPlanning = isLocalPlanning; this.nodeName = nodeName; + ruleProfiles = new HashMap<>(); } public static PlannerProfile readFrom(StreamInput in) throws IOException { - boolean isLocalPlanning = in.readBoolean(); - String nodeName = in.readString(); - return new PlannerProfile(isLocalPlanning, nodeName); + // NOCOMMIT - need to read back the profiles + return new PlannerProfile(in.readBoolean(), in.readString()); + } + + public void recordRuleProfile(String batchName, String ruleName, long runDuration, boolean hasChanges) { + ruleProfiles.computeIfAbsent(batchName, k -> new HashMap<>()) + .computeIfAbsent(ruleName, k -> new RuleProfile()) + .addRun(runDuration, hasChanges); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(isLocalPlanning); out.writeString(nodeName); + // NOCOMMIT - need to write out the profile map } @Override @@ -77,5 +107,5 @@ public Iterator toXContentChunked(ToXContent.Params params // NOCOMMIT throw new UnsupportedOperationException(); } - + // NOCOMMIT equals and hashcode, once the fields are stable } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java index 4016d6d696085..bf69f108d3a5b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java @@ -167,6 +167,7 @@ protected final ExecutionInfo executeWithInfo(TreeType plan) { batchRuns++; for (Rule rule : batch.rules) { + long ruleStart = System.nanoTime(); if (log.isTraceEnabled()) { log.trace("About to apply rule {}", rule); } @@ -184,6 +185,7 @@ protected final ExecutionInfo executeWithInfo(TreeType plan) { log.trace("Rule {} applied w/o changes", rule); } } + profile.recordRuleProfile(batch.name(), rule.name(), ruleStart - System.nanoTime(), hasChanged); } batchDuration = System.currentTimeMillis() - batchStart; } while (hasChanged && batch.limit.reached(batchRuns) == false); From 5dd319d1716c25122ebf7cd724d908aa92a39692 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 31 Mar 2025 15:53:43 +0000 Subject: [PATCH 20/20] [CI] Auto commit changes from spotless --- .../org/elasticsearch/xpack/esql/planner/PlannerProfile.java | 3 --- .../xpack/esql/planner/PlanConcurrencyCalculatorTests.java | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java index 516608943e953..141b6cddd71fc 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerProfile.java @@ -12,8 +12,6 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ChunkedToXContentObject; import org.elasticsearch.xcontent.ToXContent; -import org.elasticsearch.xcontent.ToXContentObject; -import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.util.ArrayList; @@ -21,7 +19,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; /** * Stores profiling information about the query plan. This can be the top level planning on the coordinating node, or the local diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java index 7911f9c44af35..e26f58de070b2 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java @@ -239,7 +239,8 @@ private void assertConcurrency(String query, Integer concurrencyPragmaValue, Int Analyzer analyzer = analyzer(analyzerDefaultMapping(), TEST_VERIFIER, configuration); LogicalPlan logicalPlan = AnalyzerTestUtils.analyze(query, analyzer); - logicalPlan = new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, FoldContext.small()), PlannerProfile.EMPTY).optimize(logicalPlan); + logicalPlan = new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, FoldContext.small()), PlannerProfile.EMPTY) + .optimize(logicalPlan); PhysicalPlan physicalPlan = new Mapper().map(logicalPlan); physicalPlan = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration), PlannerProfile.EMPTY).optimize(physicalPlan);