From 70de334c97716e1b034a3a0682f60a3b85c3cd66 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 17:11:24 +0200 Subject: [PATCH 01/17] Make ESQL RuleExecutor asynchronous. --- .../xpack/esql/rule/ParameterizedRule.java | 57 +++++- .../esql/rule/ParameterizedRuleExecutor.java | 11 +- .../elasticsearch/xpack/esql/rule/Rule.java | 87 ++++++--- .../xpack/esql/rule/RuleExecutor.java | 165 +++++++++++++----- 4 files changed, 245 insertions(+), 75 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java index ba771d503da08..e0b4ce611c8f7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java @@ -7,13 +7,62 @@ package org.elasticsearch.xpack.esql.rule; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.core.tree.Node; -public abstract class ParameterizedRule, P> extends Rule { +public interface ParameterizedRule, P> extends Rule { - public abstract T apply(T t, P p); + public abstract void apply(T t, P p, ActionListener listener); - public T apply(T t) { - throw new RuleExecutionException("Cannot call parameterized rule without parameter"); + /** + * Abstract base class for asynchronous parameterized rules that use ActionListener callbacks. + * This follows the same pattern as Rule.Async but adds parameter support. + */ + abstract class Async, P> extends Rule.Async implements ParameterizedRule { + + protected Async() { + super(); + } + + protected Async(String name) { + super(name); + } + + public abstract void apply(T t, P p, ActionListener listener); + + @Override + public final void apply(T t, ActionListener listener) { + listener.onFailure(new RuleExecutionException("Cannot call parameterized rule without parameter")); + } + } + + /** + * Abstract base class for synchronous parameterized rules that return results directly. + * The RuleExecutor will wrap these in async callbacks when executing. + */ + abstract class Sync, P> extends Async { + + protected Sync() { + this(null); + } + + protected Sync(String name) { + super(name); + } + + @Override + public final void apply(T t, P p, ActionListener listener) { + try { + T result = apply(t, p); + listener.onResponse(result); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Synchronous apply method to be implemented by subclasses. + */ + public abstract T apply(T t, P p); } } 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..d6a4df39579e2 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 @@ -7,10 +7,9 @@ package org.elasticsearch.xpack.esql.rule; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.core.tree.Node; -import java.util.function.Function; - public abstract class ParameterizedRuleExecutor, Context> extends RuleExecutor { private final Context context; @@ -25,7 +24,11 @@ protected Context context() { @Override @SuppressWarnings({ "rawtypes", "unchecked" }) - protected Function transform(Rule rule) { - return (rule instanceof ParameterizedRule pr) ? t -> (TreeType) pr.apply(t, context) : t -> rule.apply(t); + protected void applyRule(Rule rule, TreeType plan, ActionListener listener) { + if (rule instanceof ParameterizedRule pr) { + pr.apply(plan, context, listener); + } else { + rule.apply(plan, listener); + } } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/Rule.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/Rule.java index d8b16d350e3ea..751a1ca12f1c2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/Rule.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/Rule.java @@ -6,6 +6,7 @@ */ package org.elasticsearch.xpack.esql.rule; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; import org.elasticsearch.xpack.esql.core.tree.Node; @@ -19,33 +20,79 @@ * Rules could could be built as lambdas but most * rules are much larger, so we keep them as full-blown subclasses. */ -public abstract class Rule> { +public interface Rule> { - protected Logger log = LogManager.getLogger(getClass()); + Class typeToken(); - private final String name; - private final Class typeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + String name(); - protected Rule() { - this(null); - } + void apply(T t, ActionListener listener); - protected Rule(String name) { - this.name = (name == null ? ReflectionUtils.ruleLikeNaming(getClass()) : name); - } + /** + * Abstract base class for asynchronous rules that use ActionListener callbacks. + * This is the current implementation pattern for rules. + */ + abstract class Async> implements Rule { - public Class typeToken() { - return typeToken; - } + protected Logger log = LogManager.getLogger(getClass()); - public String name() { - return name; - } + private final String name; + private final Class typeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + + protected Async() { + this(null); + } + + protected Async(String name) { + this.name = (name == null ? ReflectionUtils.ruleLikeNaming(getClass()) : name); + } + + @Override + public Class typeToken() { + return typeToken; + } - @Override - public String toString() { - return name(); + @Override + public String name() { + return name; + } + + @Override + public String toString() { + return name(); + } + + @Override + public abstract void apply(T t, ActionListener listener); } - public abstract T apply(T t); + /** + * Abstract base class for synchronous rules that return results directly. + * The RuleExecutor will wrap these in async callbacks when executing. + */ + abstract class Sync> extends Async { + + protected Sync() { + this(null); + } + + protected Sync(String name) { + super(name); + } + + @Override + public final void apply(T t, ActionListener listener) { + try { + T result = apply(t); + listener.onResponse(result); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Synchronous apply method to be implemented by subclasses. + */ + public abstract T apply(T t); + } } 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..09b130d679a9c 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 @@ -8,6 +8,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.core.tree.NodeUtils; @@ -16,7 +18,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.function.Function; +import java.util.concurrent.atomic.AtomicReference; public abstract class RuleExecutor> { @@ -86,10 +88,10 @@ public class Transformation { private final String name; private Boolean lazyHasChanged; - Transformation(String name, TreeType plan, Function transform) { + Transformation(String name, TreeType before, TreeType after) { this.name = name; - this.before = plan; - this.after = transform.apply(before); + this.before = before; + this.after = after; } public boolean hasChanged() { @@ -137,60 +139,91 @@ public Map, List> transformations() { } protected final TreeType execute(TreeType plan) { - return executeWithInfo(plan).after; + // TODO: remove when all implementinc classes are using the async version. + PlainActionFuture executionInfoFuture = new PlainActionFuture<>(); + executeWithInfo(plan, executionInfoFuture); + return executionInfoFuture.actionGet().after; + } + + protected final void execute(TreeType plan, ActionListener listener) { + executeWithInfo(plan, ActionListener.wrap(executionInfo -> listener.onResponse(executionInfo.after()), listener::onFailure)); } - protected final ExecutionInfo executeWithInfo(TreeType plan) { - TreeType currentPlan = plan; + protected final void executeWithInfo(TreeType plan, ActionListener listener) { + AtomicReference currentPlan = new AtomicReference<>(plan); long totalDuration = 0; - Map, List> transformations = new LinkedHashMap<>(); + if (batches == null) { batches = batches(); } - for (Batch batch : batches) { - int batchRuns = 0; - List tfs = new ArrayList<>(); - transformations.put(batch, tfs); + executeBatches(plan, currentPlan, transformations, batches.iterator(), totalDuration, listener); + } - boolean hasChanged = false; - long batchStart = System.currentTimeMillis(); - long batchDuration = 0; + private void executeBatches( + TreeType originalPlan, + AtomicReference currentPlan, + Map, List> transformations, + java.util.Iterator> batchIterator, + long totalDuration, + ActionListener listener + ) { + if (!batchIterator.hasNext()) { + TreeType finalPlan = currentPlan.get(); + if (false == finalPlan.equals(originalPlan) && log.isDebugEnabled()) { + log.debug( + "Tree transformation took {}\n{}", + TimeValue.timeValueMillis(totalDuration), + NodeUtils.diffString(originalPlan, finalPlan) + ); + } + listener.onResponse(new ExecutionInfo(originalPlan, finalPlan, transformations)); + return; + } - // run each batch until no change occurs or the limit is reached - do { - hasChanged = false; - batchRuns++; + Batch batch = batchIterator.next(); + List tfs = new ArrayList<>(); + transformations.put(batch, tfs); - for (Rule rule : batch.rules) { - if (log.isTraceEnabled()) { - log.trace("About to apply rule {}", rule); - } - Transformation tf = new Transformation(rule.name(), currentPlan, transform(rule)); - tfs.add(tf); - currentPlan = tf.after; - - if (tf.hasChanged()) { - hasChanged = true; - if (log.isTraceEnabled()) { - log.trace("Rule {} applied\n{}", rule, NodeUtils.diffString(tf.before, tf.after)); - } - } else { - if (log.isTraceEnabled()) { - log.trace("Rule {} applied w/o changes", rule); - } + long batchStart = System.currentTimeMillis(); + executeBatch(batch, currentPlan, tfs, 0, batchStart, ActionListener.wrap(batchDuration -> { + long newTotalDuration = totalDuration + batchDuration; + executeBatches(originalPlan, currentPlan, transformations, batchIterator, newTotalDuration, listener); + }, listener::onFailure)); + } + + private void executeBatch( + Batch batch, + AtomicReference currentPlan, + List tfs, + int batchRuns, + long batchStart, + ActionListener listener + ) { + int currentBatchRuns = batchRuns + 1; + + executeRules(currentPlan, tfs, batch.rules, 0, false, ActionListener.wrap(batchHasChanged -> { + long batchDuration = System.currentTimeMillis() - batchStart; + + if (batchHasChanged) { + try { + if (batch.limit.reached(currentBatchRuns) == false) { + // Continue with another batch iteration - reset the batch start time + executeBatch(batch, currentPlan, tfs, currentBatchRuns, System.currentTimeMillis(), listener); + return; } + } catch (RuleExecutionException e) { + listener.onFailure(e); + return; } - batchDuration = System.currentTimeMillis() - batchStart; - } while (hasChanged && batch.limit.reached(batchRuns) == false); - - totalDuration += batchDuration; + } + // Batch is complete if (log.isTraceEnabled()) { - TreeType before = plan; - TreeType after = plan; + TreeType before = currentPlan.get(); + TreeType after = currentPlan.get(); if (tfs.isEmpty() == false) { before = tfs.get(0).before; after = tfs.get(tfs.size() - 1).after; @@ -202,16 +235,54 @@ protected final ExecutionInfo executeWithInfo(TreeType plan) { NodeUtils.diffString(before, after) ); } + listener.onResponse(batchDuration); + }, listener::onFailure)); + } + + private void executeRules( + AtomicReference currentPlan, + List tfs, + Rule[] rules, + int ruleIndex, + boolean hasChanged, + ActionListener listener + ) { + if (ruleIndex >= rules.length) { + listener.onResponse(hasChanged); + return; } - if (false == currentPlan.equals(plan) && log.isDebugEnabled()) { - log.debug("Tree transformation took {}\n{}", TimeValue.timeValueMillis(totalDuration), NodeUtils.diffString(plan, currentPlan)); + Rule rule = rules[ruleIndex]; + if (log.isTraceEnabled()) { + log.trace("About to apply rule {}", rule); } - return new ExecutionInfo(plan, currentPlan, transformations); + TreeType planBeforeRule = currentPlan.get(); + applyRule(rule, planBeforeRule, listener.delegateFailureAndWrap((l, transformedPlan) -> { + Transformation tf = new Transformation(rule.name(), planBeforeRule, transformedPlan); + tfs.add(tf); + currentPlan.set(tf.after()); + + boolean ruleHasChanged = tf.hasChanged(); + if (ruleHasChanged) { + if (log.isTraceEnabled()) { + log.trace("Rule {} applied\n{}", rule, NodeUtils.diffString(tf.before(), tf.after())); + } + } else { + if (log.isTraceEnabled()) { + log.trace("Rule {} applied w/o changes", rule); + } + } + + executeRules(currentPlan, tfs, rules, ruleIndex + 1, hasChanged || ruleHasChanged, l); + })); } - protected Function transform(Rule rule) { - return rule::apply; + protected void applyRule(Rule rule, TreeType plan, ActionListener listener) { + try { + rule.apply(plan, listener); + } catch (Exception e) { + listener.onFailure(e); + } } } From e2b58376a3bfe66256507fd9f85fbb4f6d054dfa Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 17:12:23 +0200 Subject: [PATCH 02/17] Modify analyzer and optimizer rules to be sync rules. --- .../xpack/esql/analysis/Analyzer.java | 40 ++--- .../xpack/esql/analysis/AnalyzerRules.java | 58 +++---- .../optimizer/PhysicalOptimizerRules.java | 70 +++++---- .../BooleanFunctionEqualsElimination.java | 2 +- .../rules/logical/BooleanSimplification.java | 2 +- .../logical/CombineBinaryComparisons.java | 2 +- .../rules/logical/CombineDisjunctions.java | 2 +- .../optimizer/rules/logical/CombineEvals.java | 2 +- .../rules/logical/CombineProjections.java | 2 +- .../rules/logical/ConstantFolding.java | 2 +- .../logical/ExtractAggregateCommonFilter.java | 2 +- .../optimizer/rules/logical/FoldNull.java | 2 +- .../rules/logical/LiteralsOnTheRight.java | 2 +- .../rules/logical/OptimizerRules.java | 148 +++++++++--------- .../rules/logical/PartiallyFoldCase.java | 2 +- .../rules/logical/PropagateEmptyRelation.java | 2 +- .../rules/logical/PropagateEquals.java | 2 +- .../rules/logical/PropagateEvalFoldables.java | 2 +- .../rules/logical/PropagateInlineEvals.java | 2 +- .../rules/logical/PropagateNullable.java | 2 +- .../rules/logical/PropgateUnmappedFields.java | 2 +- .../optimizer/rules/logical/PruneColumns.java | 2 +- .../rules/logical/PruneEmptyPlans.java | 2 +- .../optimizer/rules/logical/PruneFilters.java | 2 +- .../rules/logical/PruneLiteralsInOrderBy.java | 2 +- .../rules/logical/PruneRedundantOrderBy.java | 2 +- .../logical/PruneRedundantSortClauses.java | 2 +- .../rules/logical/PruneUnusedIndexMode.java | 2 +- .../logical/PushDownAndCombineFilters.java | 2 +- .../logical/PushDownAndCombineLimits.java | 2 +- .../logical/PushDownAndCombineOrderBy.java | 2 +- .../logical/PushDownAndCombineSample.java | 2 +- .../rules/logical/PushDownEnrich.java | 2 +- .../optimizer/rules/logical/PushDownEval.java | 2 +- .../rules/logical/PushDownInferencePlan.java | 2 +- .../logical/PushDownJoinPastProject.java | 2 +- .../rules/logical/PushDownRegexExtract.java | 2 +- .../rules/logical/RemoveStatsOverride.java | 2 +- ...ReplaceAggregateAggExpressionWithEval.java | 2 +- ...laceAggregateNestedExpressionWithEval.java | 2 +- .../ReplaceAliasingEvalWithProject.java | 2 +- .../logical/ReplaceLimitAndSortAsTopN.java | 2 +- .../rules/logical/ReplaceLookupWithJoin.java | 2 +- .../ReplaceOrderByExpressionWithEval.java | 2 +- .../rules/logical/ReplaceRegexMatch.java | 6 +- .../logical/ReplaceRowAsLocalRelation.java | 2 +- .../ReplaceStatsFilteredAggWithEval.java | 2 +- ...laceStringCasingWithInsensitiveEquals.java | 2 +- ...StringCasingWithInsensitiveRegexMatch.java | 2 +- .../ReplaceTrivialTypeConversions.java | 2 +- .../rules/logical/SetAsOptimized.java | 2 +- .../SimplifyComparisonsArithmetics.java | 2 +- .../logical/SkipQueryOnEmptyMappings.java | 2 +- .../rules/logical/SkipQueryOnLimitZero.java | 2 +- .../logical/SplitInWithFoldableValue.java | 2 +- .../logical/SubstituteFilteredExpression.java | 5 +- .../SubstituteSurrogateAggregations.java | 2 +- .../SubstituteSurrogateExpressions.java | 2 +- .../logical/SubstituteSurrogatePlans.java | 2 +- .../logical/TranslateTimeSeriesAggregate.java | 2 +- .../rules/logical/local/InferIsNotNull.java | 2 +- .../local/InferNonNullAggConstraint.java | 4 +- .../PruneLeftJoinOnNullMatchingField.java | 2 +- .../local/ReplaceFieldWithConstantOrNull.java | 2 +- .../local/ReplaceTopNWithLimitAndSort.java | 2 +- .../rules/physical/ProjectAwayColumns.java | 2 +- .../local/EnableSpatialDistancePushdown.java | 2 +- .../physical/local/InsertFieldExtraction.java | 4 +- .../local/ParallelizeTimeSeriesSource.java | 2 +- .../physical/local/PushFiltersToSource.java | 2 +- .../physical/local/PushLimitToSource.java | 2 +- .../physical/local/PushSampleToSource.java | 2 +- .../physical/local/PushStatsToSource.java | 4 +- .../physical/local/PushTopNToSource.java | 2 +- .../local/ReplaceSourceAttributes.java | 2 +- .../local/SpatialDocValuesExtraction.java | 2 +- .../local/SpatialShapeBoundsExtraction.java | 4 +- .../optimizer/LogicalPlanOptimizerTests.java | 2 +- .../esql/optimizer/OptimizerRulesTests.java | 2 +- 79 files changed, 240 insertions(+), 241 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 1b48ffd22b491..d34d9a5b37735 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 @@ -212,7 +212,7 @@ protected List> batches() { return RULES; } - private static class ResolveTable extends ParameterizedAnalyzerRule { + private static class ResolveTable extends ParameterizedAnalyzerRule.Sync { @Override protected LogicalPlan rule(UnresolvedRelation plan, AnalyzerContext context) { @@ -312,7 +312,7 @@ private static void mappingAsAttributes(List list, Source source, Str } } - private static class ResolveEnrich extends ParameterizedAnalyzerRule { + private static class ResolveEnrich extends ParameterizedAnalyzerRule.Sync { @Override protected LogicalPlan rule(Enrich plan, AnalyzerContext context) { @@ -398,7 +398,7 @@ private static NamedExpression createEnrichFieldExpression( } } - private static class ResolveInference extends ParameterizedAnalyzerRule, AnalyzerContext> { + private static class ResolveInference extends ParameterizedAnalyzerRule.Sync, AnalyzerContext> { @Override protected LogicalPlan rule(InferencePlan plan, AnalyzerContext context) { assert plan.inferenceId().resolved() && plan.inferenceId().foldable(); @@ -426,7 +426,7 @@ protected LogicalPlan rule(InferencePlan plan, AnalyzerContext context) { } } - private static class ResolveLookupTables extends ParameterizedAnalyzerRule { + private static class ResolveLookupTables extends ParameterizedAnalyzerRule.Sync { @Override protected LogicalPlan rule(Lookup lookup, AnalyzerContext context) { @@ -474,7 +474,7 @@ private LocalRelation tableMapAsRelation(Source source, Map mapT } } - public static class ResolveRefs extends ParameterizedAnalyzerRule { + public static class ResolveRefs extends ParameterizedAnalyzerRule.Sync { @Override protected LogicalPlan rule(LogicalPlan plan, AnalyzerContext context) { if (plan.childrenResolved() == false) { @@ -1286,7 +1286,7 @@ private static Attribute handleSpecialFields(UnresolvedAttribute u, Attribute na return named.withLocation(u.source()); } - private static class ResolveFunctions extends ParameterizedAnalyzerRule { + private static class ResolveFunctions extends ParameterizedAnalyzerRule.Sync { @Override protected LogicalPlan rule(LogicalPlan plan, AnalyzerContext context) { @@ -1319,7 +1319,7 @@ public static org.elasticsearch.xpack.esql.core.expression.function.Function res } } - private static class AddImplicitLimit extends ParameterizedRule { + private static class AddImplicitLimit extends ParameterizedRule.Sync { @Override public LogicalPlan apply(LogicalPlan logicalPlan, AnalyzerContext context) { List limits = logicalPlan.collectFirstChildren(Limit.class::isInstance); @@ -1338,7 +1338,7 @@ public LogicalPlan apply(LogicalPlan logicalPlan, AnalyzerContext context) { } } - private static class AddImplicitForkLimit extends ParameterizedRule { + private static class AddImplicitForkLimit extends ParameterizedRule.Sync { private final AddImplicitLimit addImplicitLimit = new AddImplicitLimit(); @Override @@ -1386,7 +1386,7 @@ private BitSet gatherPreAnalysisMetrics(LogicalPlan plan, BitSet b) { * * Coalesce(Int, Long) will NOT be converted to Coalesce(Long, Long) or Coalesce(Int, Int). */ - private static class ImplicitCasting extends ParameterizedRule { + private static class ImplicitCasting extends ParameterizedRule.Sync { @Override public LogicalPlan apply(LogicalPlan plan, AnalyzerContext context) { // do implicit casting for function arguments @@ -1634,15 +1634,16 @@ private static Expression processVectorFunction(org.elasticsearch.xpack.esql.cor } /** - * The EsqlIndexResolver will create InvalidMappedField instances for fields that are ambiguous (i.e. have multiple mappings). - * During {@link ResolveRefs} we do not convert these to UnresolvedAttribute instances, as we want to first determine if they can - * instead be handled by conversion functions within the query. This rule looks for matching conversion functions and converts - * those fields into MultiTypeEsField, which encapsulates the knowledge of how to convert these into a single type. - * This knowledge will be used later in generating the FieldExtractExec with built-in type conversion. - * Any fields which could not be resolved by conversion functions will be converted to UnresolvedAttribute instances in a later rule - * (See {@link UnionTypesCleanup} below). + * {@link ResolveUnionTypes} creates new, synthetic attributes for union types: + * If there was no {@code AbstractConvertFunction} that resolved multi-type fields in the {@link ResolveUnionTypes} rule, + * then there could still be some {@code FieldAttribute}s that contain unresolved {@link MultiTypeEsField}s. + * These need to be converted back to actual {@code UnresolvedAttribute} in order for validation to generate appropriate failures. + *

+ * Finally, if {@code client_ip} is present in 2 indices, once with type {@code ip} and once with type {@code keyword}, + * using {@code EVAL x = to_ip(client_ip)} will create a single attribute @{code $$client_ip$converted_to$ip}. + * This should not spill into the query output, so we drop such attributes at the end. */ - private static class ResolveUnionTypes extends Rule { + private static class ResolveUnionTypes extends Rule.Sync { record TypeResolutionKey(String fieldName, DataType fieldType) {} @@ -1652,6 +1653,7 @@ record TypeResolutionKey(String fieldName, DataType fieldType) {} public LogicalPlan apply(LogicalPlan plan) { unionFieldAttributes = new ArrayList<>(); return plan.transformUp(LogicalPlan.class, p -> p.childrenResolved() == false ? p : doRule(p)); + } private LogicalPlan doRule(LogicalPlan plan) { @@ -1850,7 +1852,7 @@ private static Expression typeSpecificConvert(ConvertFunction convert, Source so * using {@code EVAL x = to_ip(client_ip)} will create a single attribute @{code $$client_ip$converted_to$ip}. * This should not spill into the query output, so we drop such attributes at the end. */ - private static class UnionTypesCleanup extends Rule { + private static class UnionTypesCleanup extends Rule.Sync { public LogicalPlan apply(LogicalPlan plan) { LogicalPlan planWithCheckedUnionTypes = plan.transformUp( LogicalPlan.class, @@ -1898,7 +1900,7 @@ private static LogicalPlan planWithoutSyntheticAttributes(LogicalPlan plan) { /** * Cast the union typed fields in EsRelation to date_nanos if they are mixed date and date_nanos types. */ - private static class DateMillisToNanosInEsRelation extends Rule { + private static class DateMillisToNanosInEsRelation extends Rule.Sync { private final boolean isSnapshot; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java index 4889583d1b8a9..21f1bca8d0841 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java @@ -22,51 +22,37 @@ public final class AnalyzerRules { - public abstract static class AnalyzerRule extends Rule { - - // transformUp (post-order) - that is first children and then the node - // but with a twist; only if the tree is not resolved or analyzed - @Override - public final LogicalPlan apply(LogicalPlan plan) { - return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t)); - } + public interface AnalyzerRule extends Rule { + abstract class Sync extends Rule.Sync { + // transformUp (post-order) - that is first children and then the node + // but with a twist; only if the tree is not resolved or analyzed + @Override + public final LogicalPlan apply(LogicalPlan plan) { + return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t)); + } - protected abstract LogicalPlan rule(SubPlan plan); + protected abstract LogicalPlan rule(SubPlan plan); - protected boolean skipResolved() { - return true; + protected boolean skipResolved() { + return true; + } } } - public abstract static class ParameterizedAnalyzerRule extends ParameterizedRule< - SubPlan, - LogicalPlan, - P> { - - // transformUp (post-order) - that is first children and then the node - // but with a twist; only if the tree is not resolved or analyzed - public final LogicalPlan apply(LogicalPlan plan, P context) { - return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t, context)); - } - - protected abstract LogicalPlan rule(SubPlan plan, P context); - - protected boolean skipResolved() { - return true; - } - } + public interface ParameterizedAnalyzerRule extends ParameterizedRule { + abstract class Sync extends ParameterizedRule.Sync { + // transformUp (post-order) - that is first children and then the node + // but with a twist; only if the tree is not resolved or analyzed + public final LogicalPlan apply(LogicalPlan plan, P context) { + return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t, context)); + } - public abstract static class BaseAnalyzerRule extends AnalyzerRule { + protected abstract LogicalPlan rule(SubPlan plan, P context); - @Override - protected LogicalPlan rule(LogicalPlan plan) { - if (plan.childrenResolved() == false) { - return plan; + protected boolean skipResolved() { + return true; } - return doRule(plan); } - - protected abstract LogicalPlan doRule(LogicalPlan plan); } public static List maybeResolveAgainstList( diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java index ee192c2420da8..9e6c5e4cc0d97 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules; import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.rule.ParameterizedRule; @@ -14,50 +15,53 @@ public class PhysicalOptimizerRules { - public abstract static class ParameterizedOptimizerRule extends ParameterizedRule< - SubPlan, - PhysicalPlan, - P> { + public interface ParameterizedOptimizerRule extends ParameterizedRule { + abstract class Sync extends ParameterizedRule.Sync + implements + ParameterizedOptimizerRule { - private final TransformDirection direction; + private final TransformDirection direction; - public ParameterizedOptimizerRule() { - this(TransformDirection.DOWN); - } + public Sync() { + this(OptimizerRules.TransformDirection.DOWN); + } - protected ParameterizedOptimizerRule(TransformDirection direction) { - this.direction = direction; - } + protected Sync(TransformDirection direction) { + this.direction = direction; + } - @Override - public final PhysicalPlan apply(PhysicalPlan plan, P context) { - return direction == TransformDirection.DOWN - ? plan.transformDown(typeToken(), t -> rule(t, context)) - : plan.transformUp(typeToken(), t -> rule(t, context)); - } + @Override + public final PhysicalPlan apply(PhysicalPlan plan, P context) { + return direction == OptimizerRules.TransformDirection.DOWN + ? plan.transformDown(typeToken(), t -> rule(t, context)) + : plan.transformUp(typeToken(), t -> rule(t, context)); + } - protected abstract PhysicalPlan rule(SubPlan plan, P context); + protected abstract PhysicalPlan rule(SubPlan plan, P context); + } } - public abstract static class OptimizerRule extends Rule { + public interface OptimizerRule extends Rule { + abstract class Sync extends Rule.Sync implements OptimizerRule { - private final TransformDirection direction; + private final TransformDirection direction; - public OptimizerRule() { - this(TransformDirection.DOWN); - } + public Sync() { + this(OptimizerRules.TransformDirection.DOWN); + } - protected OptimizerRule(TransformDirection direction) { - this.direction = direction; - } + protected Sync(TransformDirection direction) { + this.direction = direction; + } - @Override - public final PhysicalPlan apply(PhysicalPlan plan) { - return direction == TransformDirection.DOWN - ? plan.transformDown(typeToken(), this::rule) - : plan.transformUp(typeToken(), this::rule); - } + @Override + public final PhysicalPlan apply(PhysicalPlan plan) { + return direction == OptimizerRules.TransformDirection.DOWN + ? plan.transformDown(typeToken(), this::rule) + : plan.transformUp(typeToken(), this::rule); + } - protected abstract PhysicalPlan rule(SubPlan plan); + protected abstract PhysicalPlan rule(SubPlan plan); + } } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java index d3825361f7de6..8a21303c65bb4 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java @@ -22,7 +22,7 @@ * This rule must always be placed after {@link LiteralsOnTheRight} * since it looks at TRUE/FALSE literals' existence on the right hand-side of the {@link Equals}/{@link NotEquals} expressions. */ -public final class BooleanFunctionEqualsElimination extends OptimizerRules.OptimizerExpressionRule { +public final class BooleanFunctionEqualsElimination extends OptimizerRules.OptimizerExpressionRule.Sync { public BooleanFunctionEqualsElimination() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java index 3be1889ab3c4d..65783c2034fd7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java @@ -29,7 +29,7 @@ import static org.elasticsearch.xpack.esql.expression.predicate.Predicates.splitOr; import static org.elasticsearch.xpack.esql.expression.predicate.Predicates.subtract; -public final class BooleanSimplification extends OptimizerRules.OptimizerExpressionRule { +public final class BooleanSimplification extends OptimizerRules.OptimizerExpressionRule.Sync { public BooleanSimplification() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java index e2b5df3a82783..c4921e79f8171 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java @@ -26,7 +26,7 @@ import java.util.ArrayList; import java.util.List; -public final class CombineBinaryComparisons extends OptimizerRules.OptimizerExpressionRule { +public final class CombineBinaryComparisons extends OptimizerRules.OptimizerExpressionRule.Sync { public CombineBinaryComparisons() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java index b570d5c3bc5f4..a581dba538fc9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java @@ -44,7 +44,7 @@ * This rule does NOT check for type compatibility as that phase has been * already be verified in the analyzer. */ -public final class CombineDisjunctions extends OptimizerRules.OptimizerExpressionRule { +public final class CombineDisjunctions extends OptimizerRules.OptimizerExpressionRule.Sync { public CombineDisjunctions() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java index 7afe33d8daecc..ad07a2b0ce769 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java @@ -15,7 +15,7 @@ * Combine multiple Evals into one in order to reduce the number of nodes in a plan. * TODO: eliminate unnecessary fields inside the eval as well */ -public final class CombineEvals extends OptimizerRules.OptimizerRule { +public final class CombineEvals extends OptimizerRules.OptimizerRule.Sync { public CombineEvals() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java index 0a3f0cca0d9e0..328dec640e6d5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java @@ -29,7 +29,7 @@ import java.util.List; import java.util.Set; -public final class CombineProjections extends OptimizerRules.OptimizerRule { +public final class CombineProjections extends OptimizerRules.OptimizerRule.Sync { // don't drop groupings from a local plan, as the layout has already been agreed upon private final boolean local; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java index 27eec8de59020..57ea58bb1be09 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java @@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.core.expression.Literal; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -public final class ConstantFolding extends OptimizerRules.OptimizerExpressionRule { +public final class ConstantFolding extends OptimizerRules.OptimizerExpressionRule.Sync { public ConstantFolding() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ExtractAggregateCommonFilter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ExtractAggregateCommonFilter.java index d98e0200d266d..96e4171388f0e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ExtractAggregateCommonFilter.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ExtractAggregateCommonFilter.java @@ -30,7 +30,7 @@ * ... | WHERE b > 0 | STATS MIN(a), MIN(c) | ... * */ -public final class ExtractAggregateCommonFilter extends OptimizerRules.OptimizerRule { +public final class ExtractAggregateCommonFilter extends OptimizerRules.OptimizerRule.Sync { public ExtractAggregateCommonFilter() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java index d60cb130f37b7..f26ab365cf2b9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java @@ -17,7 +17,7 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -public class FoldNull extends OptimizerRules.OptimizerExpressionRule { +public final class FoldNull extends OptimizerRules.OptimizerExpressionRule.Sync { public FoldNull() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java index 6504e6042c33a..938f9a22c89d8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java @@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.core.expression.predicate.BinaryOperator; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -public final class LiteralsOnTheRight extends OptimizerRules.OptimizerExpressionRule> { +public final class LiteralsOnTheRight extends OptimizerRules.OptimizerExpressionRule.Sync> { public LiteralsOnTheRight() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java index a32bf3a720088..659c8dbed0e61 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java @@ -19,94 +19,96 @@ public final class OptimizerRules { - public abstract static class OptimizerRule extends Rule { - - private final TransformDirection direction; - - public OptimizerRule() { - this(TransformDirection.DOWN); - } - - protected OptimizerRule(TransformDirection direction) { - this.direction = direction; - } - - @Override - public final LogicalPlan apply(LogicalPlan plan) { - return direction == TransformDirection.DOWN - ? plan.transformDown(typeToken(), this::rule) - : plan.transformUp(typeToken(), this::rule); - } - - protected abstract LogicalPlan rule(SubPlan plan); + public enum TransformDirection { + UP, + DOWN } - public abstract static class OptimizerExpressionRule extends ParameterizedRule< - LogicalPlan, - LogicalPlan, - LogicalOptimizerContext> { + public interface OptimizerRule extends Rule { + abstract class Sync extends Rule.Sync implements OptimizerRule { - private final TransformDirection direction; - // overriding type token which returns the correct class but does an uncheck cast to LogicalPlan due to its generic bound - // a proper solution is to wrap the Expression rule into a Plan rule but that would affect the rule declaration - // so instead this is hacked here - private final Class expressionTypeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + private final TransformDirection direction; - public OptimizerExpressionRule(TransformDirection direction) { - this.direction = direction; - } + public Sync() { + this(TransformDirection.DOWN); + } - @Override - public final LogicalPlan apply(LogicalPlan plan, LogicalOptimizerContext ctx) { - return direction == TransformDirection.DOWN - ? plan.transformExpressionsDown(this::shouldVisit, expressionTypeToken, e -> rule(e, ctx)) - : plan.transformExpressionsUp(this::shouldVisit, expressionTypeToken, e -> rule(e, ctx)); - } + protected Sync(TransformDirection direction) { + this.direction = direction; + } - protected abstract Expression rule(E e, LogicalOptimizerContext ctx); - - /** - * Defines if a node should be visited or not. - * Allows to skip nodes that are not applicable for the rule even if they contain expressions. - * By default that skips FROM, LIMIT, PROJECT, KEEP and DROP but this list could be extended or replaced in subclasses. - */ - protected boolean shouldVisit(Node node) { - return switch (node) { - case EsRelation relation -> false; - case Project project -> false;// this covers project, keep and drop - case Limit limit -> false; - default -> true; - }; - } + @Override + public final LogicalPlan apply(LogicalPlan plan) { + return direction == TransformDirection.DOWN + ? plan.transformDown(typeToken(), this::rule) + : plan.transformUp(typeToken(), this::rule); + } - public Class expressionToken() { - return expressionTypeToken; + protected abstract LogicalPlan rule(SubPlan plan); } } - public enum TransformDirection { - UP, - DOWN + public interface OptimizerExpressionRule extends ParameterizedRule { + + abstract class Sync extends ParameterizedRule.Sync { + private final TransformDirection direction; + // overriding type token which returns the correct class but does an uncheck cast to LogicalPlan due to its generic bound + // a proper solution is to wrap the Expression rule into a Plan rule but that would affect the rule declaration + // so instead this is hacked here + private final Class expressionTypeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + + public Sync(TransformDirection direction) { + this.direction = direction; + } + + @Override + public final LogicalPlan apply(LogicalPlan plan, LogicalOptimizerContext ctx) { + return direction == TransformDirection.DOWN + ? plan.transformExpressionsDown(this::shouldVisit, expressionTypeToken, e -> rule(e, ctx)) + : plan.transformExpressionsUp(this::shouldVisit, expressionTypeToken, e -> rule(e, ctx)); + } + + protected abstract Expression rule(E e, LogicalOptimizerContext ctx); + + /** + * Defines if a node should be visited or not. + * Allows to skip nodes that are not applicable for the rule even if they contain expressions. + * By default that skips FROM, LIMIT, PROJECT, KEEP and DROP but this list could be extended or replaced in subclasses. + */ + protected boolean shouldVisit(Node node) { + return switch (node) { + case EsRelation relation -> false; + case Project project -> false;// this covers project, keep and drop + case Limit limit -> false; + default -> true; + }; + } + + public Class expressionToken() { + return expressionTypeToken; + } + } } - public abstract static class ParameterizedOptimizerRule extends ParameterizedRule< - SubPlan, - LogicalPlan, - P> { + public interface ParameterizedOptimizerRule extends ParameterizedRule { + abstract class Sync extends ParameterizedRule.Sync + implements + ParameterizedOptimizerRule { - private final TransformDirection direction; + private final TransformDirection direction; - protected ParameterizedOptimizerRule(TransformDirection direction) { - this.direction = direction; - } + protected Sync(TransformDirection direction) { + this.direction = direction; + } - @Override - public final LogicalPlan apply(LogicalPlan plan, P context) { - return direction == TransformDirection.DOWN - ? plan.transformDown(typeToken(), t -> rule(t, context)) - : plan.transformUp(typeToken(), t -> rule(t, context)); - } + @Override + public final LogicalPlan apply(LogicalPlan plan, P context) { + return direction == TransformDirection.DOWN + ? plan.transformDown(typeToken(), t -> rule(t, context)) + : plan.transformUp(typeToken(), t -> rule(t, context)); + } - protected abstract LogicalPlan rule(SubPlan plan, P context); + protected abstract LogicalPlan rule(SubPlan plan, P context); + } } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java index 0111c7cdd806a..dc964a7f4bc2a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java @@ -23,7 +23,7 @@ * EVAL c=foo * } */ -public final class PartiallyFoldCase extends OptimizerRules.OptimizerExpressionRule { +public final class PartiallyFoldCase extends OptimizerRules.OptimizerExpressionRule.Sync { public PartiallyFoldCase() { super(DOWN); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java index b6f185c856693..fd34b93d238ee 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java @@ -28,7 +28,7 @@ import java.util.List; @SuppressWarnings("removal") -public class PropagateEmptyRelation extends OptimizerRules.ParameterizedOptimizerRule { +public class PropagateEmptyRelation extends OptimizerRules.ParameterizedOptimizerRule.Sync { public PropagateEmptyRelation() { super(OptimizerRules.TransformDirection.DOWN); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java index 8289db1531ddc..70c31edc8a679 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java @@ -36,7 +36,7 @@ * When encountering a different Equals, non-containing {@link Range} or {@link BinaryComparison}, the conjunction becomes false. * When encountering a containing {@link Range}, {@link BinaryComparison} or {@link NotEquals}, these get eliminated by the equality. */ -public final class PropagateEquals extends OptimizerRules.OptimizerExpressionRule { +public final class PropagateEquals extends OptimizerRules.OptimizerExpressionRule.Sync { public PropagateEquals() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java index f8c08705ce05a..2aa0d6a97d7e5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java @@ -21,7 +21,7 @@ * Replace any reference attribute with its source, if it does not affect the result. * This avoids ulterior look-ups between attributes and its source across nodes. */ -public final class PropagateEvalFoldables extends ParameterizedRule { +public final class PropagateEvalFoldables extends ParameterizedRule.Sync { @Override public LogicalPlan apply(LogicalPlan plan, LogicalOptimizerContext ctx) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvals.java index 1d921a5037b6f..ab33615e698bd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvals.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvals.java @@ -27,7 +27,7 @@ * As the grouping key is used to perform the join, the evaluation required for creating it has to be copied to the left side * as well. */ -public class PropagateInlineEvals extends OptimizerRules.OptimizerRule { +public final class PropagateInlineEvals extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(InlineJoin plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java index 1c9ce3541f81b..75f9247cfde6d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java @@ -27,7 +27,7 @@ // a IS NULL AND a IS NOT NULL -> FALSE // a IS NULL AND a > 10 -> a IS NULL and FALSE // can be extended to handle null conditions where available -public class PropagateNullable extends OptimizerRules.OptimizerExpressionRule { +public final class PropagateNullable extends OptimizerRules.OptimizerExpressionRule.Sync { public PropagateNullable() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropgateUnmappedFields.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropgateUnmappedFields.java index 6163a50a42ea4..40a9d350f675c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropgateUnmappedFields.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropgateUnmappedFields.java @@ -21,7 +21,7 @@ * Merges unmapped fields into the output of the ES relation. This marking is necessary for the block loaders to force loading from _source * if the field is unmapped. */ -public class PropgateUnmappedFields extends Rule { +public class PropgateUnmappedFields extends Rule.Sync { @Override public LogicalPlan apply(LogicalPlan logicalPlan) { if (logicalPlan instanceof EsRelation) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java index 58e20030adb40..ccafa7160e1e9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java @@ -33,7 +33,7 @@ /** * Remove unused columns created in the plan, in fields inside eval or aggregations inside stats. */ -public final class PruneColumns extends Rule { +public final class PruneColumns extends Rule.Sync { @Override public LogicalPlan apply(LogicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java index afd2b4e05493b..36c97ddaa7d94 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java @@ -12,7 +12,7 @@ import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; -public final class PruneEmptyPlans extends OptimizerRules.OptimizerRule { +public final class PruneEmptyPlans extends OptimizerRules.OptimizerRule.Sync { public static LogicalPlan skipPlan(UnaryPlan plan) { return new LocalRelation(plan.source(), plan.output(), LocalSupplier.EMPTY); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java index 88c3b46549d42..051fea08f300a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java @@ -20,7 +20,7 @@ import static org.elasticsearch.xpack.esql.core.expression.Literal.FALSE; import static org.elasticsearch.xpack.esql.core.expression.Literal.TRUE; -public final class PruneFilters extends OptimizerRules.OptimizerRule { +public final class PruneFilters extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Filter filter) { Expression condition = filter.condition().transformUp(BinaryLogic.class, PruneFilters::foldBinaryLogic); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java index 1b9cabeb1fae5..1c58bb5920a6b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java @@ -14,7 +14,7 @@ import java.util.ArrayList; import java.util.List; -public final class PruneLiteralsInOrderBy extends OptimizerRules.OptimizerRule { +public final class PruneLiteralsInOrderBy extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(OrderBy ob) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantOrderBy.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantOrderBy.java index 2495f72864d1c..5d25089a511f2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantOrderBy.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantOrderBy.java @@ -39,7 +39,7 @@ *

* This rule finds and prunes redundant SORTs, attempting to make the plan executable. */ -public class PruneRedundantOrderBy extends OptimizerRules.OptimizerRule { +public final class PruneRedundantOrderBy extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(LogicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java index a571aa5de7d0b..137d9d7fbe482 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java @@ -14,7 +14,7 @@ import java.util.ArrayList; -public final class PruneRedundantSortClauses extends OptimizerRules.OptimizerRule { +public final class PruneRedundantSortClauses extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(OrderBy plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneUnusedIndexMode.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneUnusedIndexMode.java index 7b3e248e54c72..fb422c94dd288 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneUnusedIndexMode.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneUnusedIndexMode.java @@ -17,7 +17,7 @@ /** * Uses the standard index mode if the special index mode is not required in the query. */ -public final class PruneUnusedIndexMode extends OptimizerRules.OptimizerRule { +public final class PruneUnusedIndexMode extends OptimizerRules.OptimizerRule.Sync { public PruneUnusedIndexMode() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java index 6b4411fafbbfe..bbad8de0e655c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java @@ -34,7 +34,7 @@ import java.util.function.Function; import java.util.function.Predicate; -public final class PushDownAndCombineFilters extends OptimizerRules.OptimizerRule { +public final class PushDownAndCombineFilters extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Filter filter) { LogicalPlan plan = filter; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java index 7bad50abd46de..5eb07a2044b53 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java @@ -24,7 +24,7 @@ import java.util.ArrayList; import java.util.List; -public final class PushDownAndCombineLimits extends OptimizerRules.ParameterizedOptimizerRule { +public final class PushDownAndCombineLimits extends OptimizerRules.ParameterizedOptimizerRule.Sync { public PushDownAndCombineLimits() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java index c0c78b05e1ce3..5e1db13412848 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java @@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.plan.logical.OrderBy; import org.elasticsearch.xpack.esql.plan.logical.Project; -public final class PushDownAndCombineOrderBy extends OptimizerRules.OptimizerRule { +public final class PushDownAndCombineOrderBy extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(OrderBy orderBy) { LogicalPlan child = orderBy.child(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineSample.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineSample.java index 3c2f2a17650e5..5bb3c60477965 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineSample.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineSample.java @@ -50,7 +50,7 @@ * * */ -public class PushDownAndCombineSample extends OptimizerRules.ParameterizedOptimizerRule { +public class PushDownAndCombineSample extends OptimizerRules.ParameterizedOptimizerRule.Sync { public PushDownAndCombineSample() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java index 4e370078cf45c..fcba10b319efe 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -public final class PushDownEnrich extends OptimizerRules.OptimizerRule { +public final class PushDownEnrich extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Enrich en) { return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(en); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java index bd8aac47c47c3..5d7cc22e0cb7e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.Eval; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -public final class PushDownEval extends OptimizerRules.OptimizerRule { +public final class PushDownEval extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Eval eval) { return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(eval); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownInferencePlan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownInferencePlan.java index 20e21adb89142..b648123f32116 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownInferencePlan.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownInferencePlan.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.inference.InferencePlan; -public final class PushDownInferencePlan extends OptimizerRules.OptimizerRule> { +public final class PushDownInferencePlan extends OptimizerRules.OptimizerRule.Sync> { @Override protected LogicalPlan rule(InferencePlan p) { return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(p); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java index 004f421c49ad0..e05ec80c45c66 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java @@ -38,7 +38,7 @@ * {@code ... | EVAL $$field = field | LOOKUP JOIN lu_idx ON key | RENAME $$field AS otherfield}. * Leaving {@code EVAL $$field = field} in place of the original projection, rather than a Project, avoids infinite loops. */ -public final class PushDownJoinPastProject extends OptimizerRules.OptimizerRule { +public final class PushDownJoinPastProject extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Join join) { if (join instanceof InlineJoin) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java index e58ed1315c034..6f28e6622f6a1 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.RegexExtract; -public final class PushDownRegexExtract extends OptimizerRules.OptimizerRule { +public final class PushDownRegexExtract extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(RegexExtract re) { return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(re); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java index 97659f2a77813..0782362797d31 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java @@ -35,7 +35,7 @@ * becomes * {@code STATS max($x + 1) BY $x = a + b} */ -public final class RemoveStatsOverride extends OptimizerRules.OptimizerRule { +public final class RemoveStatsOverride extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Aggregate aggregate) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateAggExpressionWithEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateAggExpressionWithEval.java index 075d8676abde4..709a2a0775cb9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateAggExpressionWithEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateAggExpressionWithEval.java @@ -43,7 +43,7 @@ * becomes * stats a = min(x), c = count(*) by g | eval b = a, d = c | keep a, b, c, d, g */ -public final class ReplaceAggregateAggExpressionWithEval extends OptimizerRules.OptimizerRule { +public final class ReplaceAggregateAggExpressionWithEval extends OptimizerRules.OptimizerRule.Sync { public ReplaceAggregateAggExpressionWithEval() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateNestedExpressionWithEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateNestedExpressionWithEval.java index dd7ee26aa84bd..5821edf3d0954 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateNestedExpressionWithEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAggregateNestedExpressionWithEval.java @@ -33,7 +33,7 @@ * becomes * {@code EVAL `a + 1` = a + 1, `x % 2` = x % 2 | INLINESTATS SUM(`a+1`_ref) BY `x % 2`_ref} */ -public final class ReplaceAggregateNestedExpressionWithEval extends OptimizerRules.OptimizerRule { +public final class ReplaceAggregateNestedExpressionWithEval extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Aggregate aggregate) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java index bd506862108ee..c00513f84441d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java @@ -32,7 +32,7 @@ * eval x = a + 1, z = a + 1 + 1, w = a + 1 + 1 * project x, z, z as y, w */ -public final class ReplaceAliasingEvalWithProject extends Rule { +public final class ReplaceAliasingEvalWithProject extends Rule.Sync { @Override public LogicalPlan apply(LogicalPlan logicalPlan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java index 7d44fa1fda5a2..d15f629e82d41 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java @@ -12,7 +12,7 @@ import org.elasticsearch.xpack.esql.plan.logical.OrderBy; import org.elasticsearch.xpack.esql.plan.logical.TopN; -public final class ReplaceLimitAndSortAsTopN extends OptimizerRules.OptimizerRule { +public final class ReplaceLimitAndSortAsTopN extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Limit plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java index 09ed113de1622..f81197dce2b22 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java @@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.plan.logical.Lookup; import org.elasticsearch.xpack.esql.plan.logical.join.Join; -public final class ReplaceLookupWithJoin extends OptimizerRules.OptimizerRule { +public final class ReplaceLookupWithJoin extends OptimizerRules.OptimizerRule.Sync { public ReplaceLookupWithJoin() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java index 3ea469781ae78..f24a2d07fae6f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java @@ -20,7 +20,7 @@ import static org.elasticsearch.xpack.esql.core.expression.Attribute.rawTemporaryName; -public final class ReplaceOrderByExpressionWithEval extends OptimizerRules.OptimizerRule { +public final class ReplaceOrderByExpressionWithEval extends OptimizerRules.OptimizerRule.Sync { private static int counter = 0; @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java index 76bc31510ec3e..a6ad5a45fd581 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java @@ -16,14 +16,14 @@ import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.parser.ParsingException; -public final class ReplaceRegexMatch extends OptimizerRules.OptimizerExpressionRule> { +public final class ReplaceRegexMatch extends OptimizerRules.OptimizerExpressionRule.Sync> { public ReplaceRegexMatch() { super(OptimizerRules.TransformDirection.DOWN); } @Override - public Expression rule(RegexMatch regexMatch, LogicalOptimizerContext ctx) { + public Expression rule(RegexMatch regexMatch, LogicalOptimizerContext ctx) { Expression e = regexMatch; StringPattern pattern = regexMatch.pattern(); boolean matchesAll; @@ -49,7 +49,7 @@ public Expression rule(RegexMatch regexMatch, LogicalOptimizerContext ctx) { return e; } - protected Expression regexToEquals(RegexMatch regexMatch, Literal literal) { + protected Expression regexToEquals(RegexMatch regexMatch, Literal literal) { return new Equals(regexMatch.source(), regexMatch.field(), literal); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java index 9e7b6ce80422d..1d4e211ab3b93 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java @@ -18,7 +18,7 @@ import java.util.ArrayList; import java.util.List; -public final class ReplaceRowAsLocalRelation extends OptimizerRules.ParameterizedOptimizerRule { +public final class ReplaceRowAsLocalRelation extends OptimizerRules.ParameterizedOptimizerRule.Sync { public ReplaceRowAsLocalRelation() { super(OptimizerRules.TransformDirection.DOWN); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java index a7e56a5f25fc8..0f7cec3f543a6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java @@ -36,7 +36,7 @@ * ... | STATS x = someAgg(y) {BY z} > | EVAL x = NULL | KEEP x{, z} | ... * */ -public class ReplaceStatsFilteredAggWithEval extends OptimizerRules.OptimizerRule { +public final class ReplaceStatsFilteredAggWithEval extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Aggregate aggregate) { int oldAggSize = aggregate.aggregates().size(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveEquals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveEquals.java index fbf3dcf8470a5..4ee4c813378b3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveEquals.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveEquals.java @@ -20,7 +20,7 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; -public class ReplaceStringCasingWithInsensitiveEquals extends OptimizerRules.OptimizerExpressionRule { +public final class ReplaceStringCasingWithInsensitiveEquals extends OptimizerRules.OptimizerExpressionRule.Sync { public ReplaceStringCasingWithInsensitiveEquals() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveRegexMatch.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveRegexMatch.java index fa43d51634efd..18fef877e10a2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveRegexMatch.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStringCasingWithInsensitiveRegexMatch.java @@ -19,7 +19,7 @@ import static org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceStringCasingWithInsensitiveEquals.unwrapCase; -public class ReplaceStringCasingWithInsensitiveRegexMatch extends OptimizerRules.OptimizerExpressionRule< +public final class ReplaceStringCasingWithInsensitiveRegexMatch extends OptimizerRules.OptimizerExpressionRule.Sync< RegexMatch> { public ReplaceStringCasingWithInsensitiveRegexMatch() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java index 8cb977f97b4b3..fe789b0a2ab28 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java @@ -18,7 +18,7 @@ * A following {@link ReplaceAliasingEvalWithProject} will effectively convert {@link ReferenceAttribute} into {@link FieldAttribute}, * something very useful in local physical planning. */ -public final class ReplaceTrivialTypeConversions extends OptimizerRules.OptimizerRule { +public final class ReplaceTrivialTypeConversions extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(Eval eval) { return eval.transformExpressionsOnly(AbstractConvertFunction.class, convert -> { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java index f31b1ef0ceaf4..eea967904faa2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.rule.Rule; -public final class SetAsOptimized extends Rule { +public final class SetAsOptimized extends Rule.Sync { @Override public LogicalPlan apply(LogicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java index 60ff161651f2d..15606bdc807af 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java @@ -34,7 +34,7 @@ /** * Simplifies arithmetic expressions with BinaryComparisons and fixed point fields, such as: (int + 2) / 3 > 4 => int > 10 */ -public final class SimplifyComparisonsArithmetics extends OptimizerRules.OptimizerExpressionRule { +public final class SimplifyComparisonsArithmetics extends OptimizerRules.OptimizerExpressionRule.Sync { BiFunction typesCompatible; public SimplifyComparisonsArithmetics(BiFunction typesCompatible) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java index d57a3de21b4a6..1475218fe1c45 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java @@ -12,7 +12,7 @@ import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; -public final class SkipQueryOnEmptyMappings extends OptimizerRules.OptimizerRule { +public final class SkipQueryOnEmptyMappings extends OptimizerRules.OptimizerRule.Sync { @Override protected LogicalPlan rule(EsRelation plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java index c6d62dee0ba42..443ad72334205 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java @@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.plan.logical.Limit; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -public final class SkipQueryOnLimitZero extends OptimizerRules.ParameterizedOptimizerRule { +public final class SkipQueryOnLimitZero extends OptimizerRules.ParameterizedOptimizerRule.Sync { public SkipQueryOnLimitZero() { super(OptimizerRules.TransformDirection.DOWN); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java index 534f118617d50..9c416da000475 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java @@ -19,7 +19,7 @@ /** * 3 in (field, 4, 5) --> 3 in (field) or 3 in (4, 5) */ -public final class SplitInWithFoldableValue extends OptimizerRules.OptimizerExpressionRule { +public final class SplitInWithFoldableValue extends OptimizerRules.OptimizerExpressionRule.Sync { public SplitInWithFoldableValue() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteFilteredExpression.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteFilteredExpression.java index 62a00b79d7333..295b0d9e5a95a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteFilteredExpression.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteFilteredExpression.java @@ -11,14 +11,13 @@ import org.elasticsearch.xpack.esql.expression.function.aggregate.FilteredExpression; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.OptimizerExpressionRule; -import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection; /** * This rule should not be needed - the substitute infrastructure should be enough. */ -public class SubstituteFilteredExpression extends OptimizerExpressionRule { +public class SubstituteFilteredExpression extends OptimizerExpressionRule.Sync { public SubstituteFilteredExpression() { - super(TransformDirection.UP); + super(OptimizerRules.TransformDirection.UP); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java index c762015dc597e..04a8c7b994666 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java @@ -32,7 +32,7 @@ import java.util.List; import java.util.Map; -public final class SubstituteSurrogateAggregations extends OptimizerRules.OptimizerRule { +public final class SubstituteSurrogateAggregations extends OptimizerRules.OptimizerRule.Sync { public SubstituteSurrogateAggregations() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateExpressions.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateExpressions.java index 307634f4e8983..adfada0969675 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateExpressions.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateExpressions.java @@ -14,7 +14,7 @@ /** * Replace {@link SurrogateExpression}s with their {@link SurrogateExpression#surrogate surrogates}. */ -public final class SubstituteSurrogateExpressions extends OptimizerRules.OptimizerExpressionRule { +public final class SubstituteSurrogateExpressions extends OptimizerRules.OptimizerExpressionRule.Sync { public SubstituteSurrogateExpressions() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogatePlans.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogatePlans.java index 05e725a22ccea..0ff83e2219fcd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogatePlans.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogatePlans.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.SurrogateLogicalPlan; -public final class SubstituteSurrogatePlans extends OptimizerRules.OptimizerRule { +public final class SubstituteSurrogatePlans extends OptimizerRules.OptimizerRule.Sync { public SubstituteSurrogatePlans() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateTimeSeriesAggregate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateTimeSeriesAggregate.java index 022ed8d372635..4ab0f07a16209 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateTimeSeriesAggregate.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateTimeSeriesAggregate.java @@ -144,7 +144,7 @@ * | STATS max(rate_$1 + rate_$2) BY host_values, time_bucket * */ -public final class TranslateTimeSeriesAggregate extends OptimizerRules.OptimizerRule { +public final class TranslateTimeSeriesAggregate extends OptimizerRules.OptimizerRule.Sync { public TranslateTimeSeriesAggregate() { super(OptimizerRules.TransformDirection.UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java index 18159c112914e..f3f5ef77d1ff6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java @@ -42,7 +42,7 @@ * Implementation-wise this rule goes bottom-up, keeping an alias up to date to the current plan * and then looks for replacing the target. */ -public class InferIsNotNull extends Rule { +public class InferIsNotNull extends Rule.Sync { @Override public LogicalPlan apply(LogicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java index 1d6aa2d9ba1d1..dc4224987232b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java @@ -34,7 +34,9 @@ * Unfortunately this optimization cannot be applied when grouping is necessary since it can filter out * groups containing only null values */ -public class InferNonNullAggConstraint extends OptimizerRules.ParameterizedOptimizerRule { +public final class InferNonNullAggConstraint extends OptimizerRules.ParameterizedOptimizerRule.Sync< + Aggregate, + LocalLogicalOptimizerContext> { public InferNonNullAggConstraint() { super(OptimizerRules.TransformDirection.UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/PruneLeftJoinOnNullMatchingField.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/PruneLeftJoinOnNullMatchingField.java index e46ab04e8fc91..d541b1cc18d9d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/PruneLeftJoinOnNullMatchingField.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/PruneLeftJoinOnNullMatchingField.java @@ -28,7 +28,7 @@ * of it. The rule can apply on the coordinator already, but it's more likely to be effective on the data nodes, where null aliasing is * inserted due to locally missing fields. This rule relies on that behavior -- see {@link ReplaceFieldWithConstantOrNull}. */ -public class PruneLeftJoinOnNullMatchingField extends OptimizerRules.ParameterizedOptimizerRule { +public final class PruneLeftJoinOnNullMatchingField extends OptimizerRules.ParameterizedOptimizerRule.Sync { public PruneLeftJoinOnNullMatchingField() { super(OptimizerRules.TransformDirection.DOWN); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceFieldWithConstantOrNull.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceFieldWithConstantOrNull.java index 5635cbba89c31..10351e7698e14 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceFieldWithConstantOrNull.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceFieldWithConstantOrNull.java @@ -36,7 +36,7 @@ * Look for any fields used in the plan that are missing and replaces them with null or look for fields that are constant. * This should minimize the plan execution, in the best scenario skipping its execution all together. */ -public class ReplaceFieldWithConstantOrNull extends ParameterizedRule { +public class ReplaceFieldWithConstantOrNull extends ParameterizedRule.Sync { @Override public LogicalPlan apply(LogicalPlan plan, LocalLogicalOptimizerContext localLogicalOptimizerContext) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java index 70bb401814e6c..5b9077c620974 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java @@ -18,7 +18,7 @@ /** * Break TopN back into Limit + OrderBy to allow the order rules to kick in. */ -public class ReplaceTopNWithLimitAndSort extends OptimizerRules.OptimizerRule { +public final class ReplaceTopNWithLimitAndSort extends OptimizerRules.OptimizerRule.Sync { public ReplaceTopNWithLimitAndSort() { super(UP); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java index 26cfbf40eb7ff..dbac127346466 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java @@ -35,7 +35,7 @@ * This is done here to localize the project close to the data source and simplify the upcoming field * extraction. */ -public class ProjectAwayColumns extends Rule { +public class ProjectAwayColumns extends Rule.Sync { @Override public PhysicalPlan apply(PhysicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java index 1e976ca2e6263..0969b1090cdb5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java @@ -69,7 +69,7 @@ * Note that the WHERE clause is both rewritten to an intersection and pushed down closer to the EsQueryExec, * which allows the predicate to be pushed down to Lucene in a later rule, PushFiltersToSource. */ -public class EnableSpatialDistancePushdown extends PhysicalOptimizerRules.ParameterizedOptimizerRule< +public class EnableSpatialDistancePushdown extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync< FilterExec, LocalPhysicalOptimizerContext> { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java index a4ec64b004a0c..9700a9550f37f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java @@ -34,7 +34,9 @@ * * @see ProjectAwayColumns */ -public class InsertFieldExtraction extends PhysicalOptimizerRules.ParameterizedOptimizerRule { +public class InsertFieldExtraction extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync< + PhysicalPlan, + LocalPhysicalOptimizerContext> { @Override public PhysicalPlan rule(PhysicalPlan plan, LocalPhysicalOptimizerContext context) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ParallelizeTimeSeriesSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ParallelizeTimeSeriesSource.java index d26599d7a96c9..64193bb1f9575 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ParallelizeTimeSeriesSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ParallelizeTimeSeriesSource.java @@ -33,7 +33,7 @@ * and time-series aggregation so that they can run parallel to speed up time-series query. * For the field-extraction part, it will use a specialized version for time-series indices. */ -public class ParallelizeTimeSeriesSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule< +public class ParallelizeTimeSeriesSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync< TimeSeriesAggregateExec, LocalPhysicalOptimizerContext> { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java index 1f8341c4768d2..74cb6e20d0b0e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java @@ -39,7 +39,7 @@ import static org.elasticsearch.xpack.esql.expression.predicate.Predicates.splitAnd; import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER; -public class PushFiltersToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule { +public class PushFiltersToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync { @Override protected PhysicalPlan rule(FilterExec filterExec, LocalPhysicalOptimizerContext ctx) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java index a6d3d5c1d537f..c007930a9b216 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java @@ -13,7 +13,7 @@ import org.elasticsearch.xpack.esql.plan.physical.LimitExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; -public class PushLimitToSource extends PhysicalOptimizerRules.OptimizerRule { +public class PushLimitToSource extends PhysicalOptimizerRules.OptimizerRule.Sync { @Override protected PhysicalPlan rule(LimitExec limitExec) { PhysicalPlan plan = limitExec; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushSampleToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushSampleToSource.java index 4e41b748ece08..6d0184db8ee15 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushSampleToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushSampleToSource.java @@ -18,7 +18,7 @@ import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.xpack.esql.planner.mapper.MapperUtils.hasScoreAttribute; -public class PushSampleToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule { +public class PushSampleToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync { @Override protected PhysicalPlan rule(SampleExec sample, LocalPhysicalOptimizerContext ctx) { PhysicalPlan plan = sample; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java index 0fff9e233e956..2ab1fad892155 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java @@ -41,7 +41,9 @@ /** * Looks for the case where certain stats exist right before the query and thus can be pushed down. */ -public class PushStatsToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule { +public class PushStatsToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync< + AggregateExec, + LocalPhysicalOptimizerContext> { @Override protected PhysicalPlan rule(AggregateExec aggregateExec, LocalPhysicalOptimizerContext context) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java index 02d2f49605ced..64f440516576b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java @@ -59,7 +59,7 @@ * * */ -public class PushTopNToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule { +public class PushTopNToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync { @Override protected PhysicalPlan rule(TopNExec topNExec, LocalPhysicalOptimizerContext ctx) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java index 4730f561348c9..cf091a7d46ad1 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java @@ -21,7 +21,7 @@ import static org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection.UP; -public class ReplaceSourceAttributes extends PhysicalOptimizerRules.OptimizerRule { +public class ReplaceSourceAttributes extends PhysicalOptimizerRules.OptimizerRule.Sync { public ReplaceSourceAttributes() { super(UP); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java index adcc8bc62acb4..55cefd1043653 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java @@ -68,7 +68,7 @@ * is the only place where this information is available. This also means that the knowledge of the usage of doc-values does not need * to be serialized between nodes, and is only used locally. */ -public class SpatialDocValuesExtraction extends PhysicalOptimizerRules.ParameterizedOptimizerRule< +public class SpatialDocValuesExtraction extends PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync< AggregateExec, LocalPhysicalOptimizerContext> { @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialShapeBoundsExtraction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialShapeBoundsExtraction.java index eb0d82a59079f..9500901e50a11 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialShapeBoundsExtraction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialShapeBoundsExtraction.java @@ -18,7 +18,7 @@ import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction; import org.elasticsearch.xpack.esql.expression.function.aggregate.SpatialExtent; import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext; -import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules.ParameterizedOptimizerRule; +import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules.ParameterizedOptimizerRule.Sync; import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; import org.elasticsearch.xpack.esql.plan.physical.EvalExec; import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; @@ -47,7 +47,7 @@ * not a check like {@code isNotNull}. * */ -public class SpatialShapeBoundsExtraction extends ParameterizedOptimizerRule { +public class SpatialShapeBoundsExtraction extends Sync { @Override protected PhysicalPlan rule(AggregateExec aggregate, LocalPhysicalOptimizerContext ctx) { Set foundAttributes = findSpatialShapeBoundsAttributes(aggregate, ctx); 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 7795bf5c2d9ff..de49630541599 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 @@ -5460,7 +5460,7 @@ public void testPushdownWithOverwrittenName() { record PushdownShadowingGeneratingPlanTestCase( BiFunction applyLogicalPlan, - OptimizerRules.OptimizerRule rule + OptimizerRules.OptimizerRule.Sync rule ) {}; static PushdownShadowingGeneratingPlanTestCase[] PUSHDOWN_SHADOWING_GENERATING_PLAN_TEST_CASES = { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java index b36cb3f6c6a42..8db3618f7f237 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java @@ -117,7 +117,7 @@ public void testFoldExcludingRangeWithDifferentTypesToFalse() { } public void testOptimizerExpressionRuleShouldNotVisitExcludedNodes() { - var rule = new OptimizerRules.OptimizerExpressionRule<>(randomFrom(OptimizerRules.TransformDirection.values())) { + var rule = new OptimizerRules.OptimizerExpressionRule.Sync<>(randomFrom(OptimizerRules.TransformDirection.values())) { private final List appliedTo = new ArrayList<>(); @Override From 40ab549c2a3f5c94ad2e001f53a991f19e296629 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 17:44:34 +0200 Subject: [PATCH 03/17] Append tests for the RuleExecutor and the ParameterizedRuleExecutor --- .../xpack/esql/rule/AbstractRuleTestCase.java | 287 ++++++++++++++++ .../rule/ParameterizedRuleExecutorTests.java | 310 ++++++++++++++++++ .../xpack/esql/rule/RuleExecutorTests.java | 244 ++++++++++++++ 3 files changed, 841 insertions(+) create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java new file mode 100644 index 0000000000000..122a7976261a1 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -0,0 +1,287 @@ +/* + * 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.rule; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.tree.Node; +import org.elasticsearch.xpack.esql.core.tree.NodeInfo; +import org.elasticsearch.xpack.esql.core.tree.Source; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.hamcrest.Matchers.containsString; + +/** + * Abstract base class for rule execution tests providing common test infrastructure. + */ +public abstract class AbstractRuleTestCase extends ESTestCase { + + // Test node implementation + protected static class TestNode extends Node { + private final String value; + private final List children; + + public TestNode(String value) { + this(Source.EMPTY, value, Collections.emptyList()); + } + + public TestNode(String value, List children) { + this(Source.EMPTY, value, children); + } + + public TestNode(Source source, String value, List children) { + super(source, children); + this.value = value; + this.children = children; + } + + public String value() { + return value; + } + + @Override + public TestNode replaceChildren(List newChildren) { + return new TestNode(source(), value, newChildren); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, TestNode::new, value, children); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + // Not needed for tests + } + + @Override + public String getWriteableName() { + return "test-node"; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (!(obj instanceof TestNode)) return false; + TestNode other = (TestNode) obj; + return value.equals(other.value) && children.equals(other.children); + } + + @Override + public int hashCode() { + return value.hashCode() * 31 + children.hashCode(); + } + + @Override + public String toString() { + return value + (children.isEmpty() ? "" : "(" + children + ")"); + } + } + + // Test rule implementations + protected static class AppendRule extends Rule.Sync { + private final String suffix; + + public AppendRule(String suffix) { + this.suffix = suffix; + } + + @Override + public TestNode apply(TestNode node) { + return new TestNode(node.value() + suffix, node.children()); + } + + @Override + public String name() { + return "Append" + suffix; + } + } + + protected static class ConditionalRule extends Rule.Sync { + private final String trigger; + private final String replacement; + + public ConditionalRule(String trigger, String replacement) { + this.trigger = trigger; + this.replacement = replacement; + } + + @Override + public TestNode apply(TestNode node) { + if (node.value().equals(trigger)) { + return new TestNode(replacement, node.children()); + } + return node; // No change if condition not met + } + + @Override + public String name() { + return "Conditional" + trigger + "To" + replacement; + } + } + + protected static class CountingAsyncRule extends Rule.Async { + private final AtomicInteger callCount = new AtomicInteger(0); + private final String suffix; + + public CountingAsyncRule(String suffix) { + this.suffix = suffix; + } + + @Override + public void apply(TestNode node, ActionListener listener) { + callCount.incrementAndGet(); + // Simulate async processing + listener.onResponse(new TestNode(node.value() + suffix, node.children())); + } + + @Override + public String name() { + return "CountingAsync" + suffix; + } + + public int getCallCount() { + return callCount.get(); + } + } + + protected static class FailingRule extends Rule.Async { + private final String errorMessage; + + public FailingRule(String errorMessage) { + this.errorMessage = errorMessage; + } + + @Override + public void apply(TestNode node, ActionListener listener) { + listener.onFailure(new RuntimeException(errorMessage)); + } + + @Override + public String name() { + return "FailingRule"; + } + } + + protected static class TestParameterizedRule extends ParameterizedRule.Sync { + @Override + public TestNode apply(TestNode node, String param) { + return new TestNode(node.value() + "_" + param, node.children()); + } + + @Override + public String name() { + return "TestParameterizedRule"; + } + } + + protected static class TestContextParameterizedRule extends ParameterizedRule.Sync { + @Override + public TestNode apply(TestNode node, TestContext context) { + return new TestNode(context.prefix + node.value() + context.suffix, node.children()); + } + + @Override + public String name() { + return "TestContextParameterizedRule"; + } + } + + // Test context class + protected static class TestContext { + public final String prefix; + public final String suffix; + + public TestContext(String prefix, String suffix) { + this.prefix = prefix; + this.suffix = suffix; + } + + @Override + public String toString() { + return prefix + "..." + suffix; + } + } + + // Helper methods for async testing + protected static class AsyncResult { + private final CountDownLatch latch = new CountDownLatch(1); + private final AtomicReference result = new AtomicReference<>(); + private final AtomicReference error = new AtomicReference<>(); + + public ActionListener listener() { + return ActionListener.wrap( + res -> { + result.set(res); + latch.countDown(); + }, + err -> { + error.set(err); + latch.countDown(); + } + ); + } + + public boolean await() throws InterruptedException { + return latch.await(5, TimeUnit.SECONDS); + } + + public T get() { + return result.get(); + } + + public Throwable getError() { + return error.get(); + } + + public void assertSuccess() { + try { + assertTrue("Execution should complete within timeout", await()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + assertNull("Should not have error: " + getError(), getError()); + assertNotNull("Should have result", get()); + } + + public void assertFailure() { + try { + assertTrue("Execution should complete within timeout", await()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + assertNotNull("Should have error", getError()); + assertNull("Should not have result", get()); + } + + public void assertFailure(String expectedMessage) { + assertFailure(); + assertThat(getError().getMessage(), containsString(expectedMessage)); + } + } + + // Utility method to await async results + protected static boolean await(CountDownLatch latch) { + try { + return latch.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java new file mode 100644 index 0000000000000..f0c689b7b1708 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -0,0 +1,310 @@ +/* + * 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.rule; + +import org.elasticsearch.action.ActionListener; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class ParameterizedRuleExecutorTests extends AbstractRuleTestCase { + + // Test ParameterizedRuleExecutor implementation + static class TestParameterizedRuleExecutor extends ParameterizedRuleExecutor { + public List> batches = new ArrayList<>(); + + public TestParameterizedRuleExecutor(String context) { + super(context); + } + + @Override + public List> batches() { + return batches; + } + } + + static class TestContextParameterizedRuleExecutor extends ParameterizedRuleExecutor { + public List> batches = new ArrayList<>(); + + public TestContextParameterizedRuleExecutor(TestContext context) { + super(context); + } + + @Override + public List> batches() { + return batches; + } + } + + public void testBasicParameterizedRuleExecution() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param_value"); + TestNode input = new TestNode("test"); + + TestParameterizedRule rule = new TestParameterizedRule(); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ParamBatch", rule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test_param_value", result.get().after().value()); + } + + public void testParameterizedRuleWithComplexContext() { + TestContext context = new TestContext("start_", "_end"); + TestContextParameterizedRuleExecutor executor = new TestContextParameterizedRuleExecutor(context); + TestNode input = new TestNode("middle"); + + TestContextParameterizedRule rule = new TestContextParameterizedRule(); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("start_middle_end", result.get().after().value()); + } + + public void testMultipleParameterizedRulesInBatch() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("X"); + TestNode input = new TestNode("base"); + + TestParameterizedRule rule1 = new TestParameterizedRule(); + TestParameterizedRule rule2 = new TestParameterizedRule(); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MultiBatch", rule1, rule2); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("base_X_X", result.get().after().value()); + + // Check transformations + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); + assertThat(batchTransformations.size(), equalTo(2)); + assertEquals("TestParameterizedRule", batchTransformations.get(0).name()); + assertEquals("TestParameterizedRule", batchTransformations.get(1).name()); + } + + public void testMixedRulesInParameterizedExecutor() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param"); + TestNode input = new TestNode("test"); + + // Mix parameterized and non-parameterized rules + AppendRule nonParamRule = new AppendRule("_suffix"); + TestParameterizedRule paramRule = new TestParameterizedRule(); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MixedBatch", nonParamRule, paramRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + // Should apply non-parameterized rule first, then parameterized rule + assertEquals("test_suffix_param", result.get().after().value()); + } + + public void testParameterizedRuleFailure() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("error_param"); + TestNode input = new TestNode("test"); + + // Create a failing parameterized rule + ParameterizedRule failingRule = new ParameterizedRule.Async() { + @Override + public void apply(TestNode node, String param, ActionListener listener) { + listener.onFailure(new RuntimeException("Parameterized rule failed with: " + param)); + } + + @Override + public String name() { + return "FailingParameterizedRule"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", failingRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertFailure("Parameterized rule failed with: error_param"); + } + + public void testParameterizedRuleExecutionOrder() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ORDER"); + TestNode input = new TestNode(""); + + List executionOrder = new ArrayList<>(); + + ParameterizedRule rule1 = new ParameterizedRule.Sync() { + @Override + public TestNode apply(TestNode node, String param) { + executionOrder.add("rule1_" + param); + return new TestNode(node.value() + "1", node.children()); + } + + @Override + public String name() { + return "ParamRule1"; + } + }; + + ParameterizedRule rule2 = new ParameterizedRule.Sync() { + @Override + public TestNode apply(TestNode node, String param) { + executionOrder.add("rule2_" + param); + return new TestNode(node.value() + "2", node.children()); + } + + @Override + public String name() { + return "ParamRule2"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("12", result.get().after().value()); + assertEquals(Arrays.asList("rule1_ORDER", "rule2_ORDER"), executionOrder); + } + + public void testParameterizedRuleNoChange() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ignored"); + TestNode input = new TestNode("test"); + + ParameterizedRule noChangeRule = new ParameterizedRule.Sync() { + @Override + public TestNode apply(TestNode node, String param) { + return node; // No change regardless of parameter + } + + @Override + public String name() { + return "NoChangeParamRule"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test", result.get().after().value()); + assertEquals(input, result.get().after()); // Same instance since no change + + // Check that transformation was recorded but marked as no change + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); + assertThat(batchTransformations.size(), equalTo(1)); + assertFalse("Should not have changed", batchTransformations.get(0).hasChanged()); + } + + public void testParameterizedRuleWithMultipleBatches() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("middle"); + TestNode input = new TestNode("test"); + + // First batch with parameterized rule + TestParameterizedRule paramRule = new TestParameterizedRule(); + RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("PrependBatch", paramRule); + + // Second batch with non-parameterized rule + AppendRule appendRule = new AppendRule("_final"); + RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("AppendBatch", appendRule); + + executor.batches.add(batch1); + executor.batches.add(batch2); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test_middle_final", result.get().after().value()); + + // Should have transformations from both batches + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(2)); + } + + public void testParameterizedExecuteShortcut() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("shortcut"); + TestNode input = new TestNode("test"); + + TestParameterizedRule rule = new TestParameterizedRule(); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", rule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.execute(input, result.listener()); + + result.assertSuccess(); + assertEquals("test_shortcut", result.get().value()); + } + + public void testParameterizedAsyncRule() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("async_param"); + TestNode input = new TestNode("test"); + + ParameterizedRule asyncRule = new ParameterizedRule.Async() { + @Override + public void apply(TestNode node, String param, ActionListener listener) { + // Simulate async processing + listener.onResponse(new TestNode("async_" + node.value() + "_" + param, node.children())); + } + + @Override + public String name() { + return "AsyncParameterizedRule"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("async_test_async_param", result.get().after().value()); + } + + public void testParameterizedRuleContextAccess() { + TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("context_value"); + TestNode input = new TestNode("test"); + + // Verify that the executor's context is correctly passed to rules + assertEquals("context_value", executor.context()); + + TestParameterizedRule rule = new TestParameterizedRule(); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test_context_value", result.get().after().value()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java new file mode 100644 index 0000000000000..e03d30acc18fc --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java @@ -0,0 +1,244 @@ +/* + * 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.rule; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class RuleExecutorTests extends AbstractRuleTestCase { + + // Test RuleExecutor implementation + static class TestRuleExecutor extends RuleExecutor { + public List> batches = new ArrayList<>(); + + @Override + public List> batches() { + return batches; + } + } + + public void testBasicSyncRuleExecution() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("test"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "success")); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("success", result.get().after().value()); + } + + public void testMultipleRulesInBatch() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("start"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", + new ConditionalRule("start", "middle"), + new ConditionalRule("middle", "end") + ); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("end", result.get().after().value()); + + // Check transformations + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); + assertThat(batchTransformations.size(), equalTo(2)); + assertEquals("ConditionalstartTomiddle", batchTransformations.get(0).name()); + assertEquals("ConditionalmiddleToend", batchTransformations.get(1).name()); + } + + public void testAsyncRuleExecution() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("test"); + CountingAsyncRule asyncRule = new CountingAsyncRule("_async"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test_async", result.get().after().value()); + assertEquals(1, asyncRule.getCallCount()); + } + + public void testRuleFailure() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("test"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", new FailingRule("Test error")); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertFailure("Test error"); + } + + public void testRuleExecutionOrder() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("A"); + + List executionOrder = new ArrayList<>(); + + Rule rule1 = new Rule.Sync() { + @Override + public TestNode apply(TestNode node) { + executionOrder.add("rule1"); + return new TestNode("B", node.children()); + } + + @Override + public String name() { + return "Rule1"; + } + }; + + Rule rule2 = new Rule.Sync() { + @Override + public TestNode apply(TestNode node) { + executionOrder.add("rule2"); + return new TestNode("C", node.children()); + } + + @Override + public String name() { + return "Rule2"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("C", result.get().after().value()); + assertEquals(Arrays.asList("rule1", "rule2"), executionOrder); + } + + public void testNoChangeRule() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("test"); + + Rule noChangeRule = new Rule.Sync() { + @Override + public TestNode apply(TestNode node) { + return node; // No change + } + + @Override + public String name() { + return "NoChange"; + } + }; + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("test", result.get().after().value()); + assertEquals(input, result.get().after()); // Same instance since no change + + // Check that transformation was recorded but marked as no change + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); + assertThat(batchTransformations.size(), equalTo(1)); + assertFalse("Should not have changed", batchTransformations.get(0).hasChanged()); + } + + public void testExecuteShortcut() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("test"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "done")); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.execute(input, result.listener()); + + result.assertSuccess(); + assertEquals("done", result.get().value()); + } + + public void testMultipleBatches() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("start"); + + RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("Batch1", new ConditionalRule("start", "middle")); + RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("Batch2", new ConditionalRule("middle", "end")); + + executor.batches.add(batch1); + executor.batches.add(batch2); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + assertEquals("end", result.get().after().value()); + + // Should have transformations from both batches + var transformations = result.get().transformations(); + assertThat(transformations.keySet().size(), equalTo(2)); + } + + public void testTransformationTracking() { + TestRuleExecutor executor = new TestRuleExecutor(); + TestNode input = new TestNode("original"); + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TrackingBatch", + new ConditionalRule("original", "modified"), + new ConditionalRule("modified", "final") + ); + executor.batches.add(batch); + + AsyncResult result = new AsyncResult<>(); + executor.executeWithInfo(input, result.listener()); + + result.assertSuccess(); + + TestRuleExecutor.ExecutionInfo info = result.get(); + assertEquals("original", info.before().value()); + assertEquals("final", info.after().value()); + + var transformations = info.transformations(); + assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); + assertThat(batchTransformations.size(), equalTo(2)); + + TestRuleExecutor.Transformation first = batchTransformations.get(0); + assertEquals("ConditionaloriginalTomodified", first.name()); + assertEquals("original", first.before().value()); + assertEquals("modified", first.after().value()); + assertTrue("Should have changed", first.hasChanged()); + + TestRuleExecutor.Transformation second = batchTransformations.get(1); + assertEquals("ConditionalmodifiedTofinal", second.name()); + assertEquals("modified", second.before().value()); + assertEquals("final", second.after().value()); + assertTrue("Should have changed", second.hasChanged()); + } +} From c55951139320924255979199c1dd4325129bfefa Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 18:01:03 +0200 Subject: [PATCH 04/17] Add deprecation for synchronous execute method in the RuleExecutor --- .../java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java | 2 ++ 1 file changed, 2 insertions(+) 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 09b130d679a9c..00a1151984e86 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 @@ -138,6 +138,8 @@ public Map, List> transformations() { } } + + @Deprecated(forRemoval = true) protected final TreeType execute(TreeType plan) { // TODO: remove when all implementinc classes are using the async version. PlainActionFuture executionInfoFuture = new PlainActionFuture<>(); From ddb4f90cacfc1807d4092bc9af3beb88389a8575 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 18:06:46 +0200 Subject: [PATCH 05/17] Lint changes --- .../xpack/esql/rule/RuleExecutor.java | 1 - .../xpack/esql/rule/AbstractRuleTestCase.java | 17 ++- .../rule/ParameterizedRuleExecutorTests.java | 112 +++++++++--------- .../xpack/esql/rule/RuleExecutorTests.java | 92 +++++++------- 4 files changed, 110 insertions(+), 112 deletions(-) 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 00a1151984e86..373d725bc2eb9 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 @@ -138,7 +138,6 @@ public Map, List> transformations() { } } - @Deprecated(forRemoval = true) protected final TreeType execute(TreeType plan) { // TODO: remove when all implementinc classes are using the async version. diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java index 122a7976261a1..9d2444b300e8f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -223,16 +223,13 @@ protected static class AsyncResult { private final AtomicReference error = new AtomicReference<>(); public ActionListener listener() { - return ActionListener.wrap( - res -> { - result.set(res); - latch.countDown(); - }, - err -> { - error.set(err); - latch.countDown(); - } - ); + return ActionListener.wrap(res -> { + result.set(res); + latch.countDown(); + }, err -> { + error.set(err); + latch.countDown(); + }); } public boolean await() throws InterruptedException { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java index f0c689b7b1708..9c8802c52680d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -20,11 +20,11 @@ public class ParameterizedRuleExecutorTests extends AbstractRuleTestCase { // Test ParameterizedRuleExecutor implementation static class TestParameterizedRuleExecutor extends ParameterizedRuleExecutor { public List> batches = new ArrayList<>(); - + public TestParameterizedRuleExecutor(String context) { super(context); } - + @Override public List> batches() { return batches; @@ -33,11 +33,11 @@ public List> batches() { static class TestContextParameterizedRuleExecutor extends ParameterizedRuleExecutor { public List> batches = new ArrayList<>(); - + public TestContextParameterizedRuleExecutor(TestContext context) { super(context); } - + @Override public List> batches() { return batches; @@ -47,14 +47,14 @@ public List> batches() { public void testBasicParameterizedRuleExecution() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param_value"); TestNode input = new TestNode("test"); - + TestParameterizedRule rule = new TestParameterizedRule(); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ParamBatch", rule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test_param_value", result.get().after().value()); } @@ -63,14 +63,14 @@ public void testParameterizedRuleWithComplexContext() { TestContext context = new TestContext("start_", "_end"); TestContextParameterizedRuleExecutor executor = new TestContextParameterizedRuleExecutor(context); TestNode input = new TestNode("middle"); - + TestContextParameterizedRule rule = new TestContextParameterizedRule(); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("start_middle_end", result.get().after().value()); } @@ -78,19 +78,19 @@ public void testParameterizedRuleWithComplexContext() { public void testMultipleParameterizedRulesInBatch() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("X"); TestNode input = new TestNode("base"); - + TestParameterizedRule rule1 = new TestParameterizedRule(); TestParameterizedRule rule2 = new TestParameterizedRule(); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MultiBatch", rule1, rule2); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("base_X_X", result.get().after().value()); - + // Check transformations var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); @@ -103,17 +103,17 @@ public void testMultipleParameterizedRulesInBatch() { public void testMixedRulesInParameterizedExecutor() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param"); TestNode input = new TestNode("test"); - + // Mix parameterized and non-parameterized rules AppendRule nonParamRule = new AppendRule("_suffix"); TestParameterizedRule paramRule = new TestParameterizedRule(); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MixedBatch", nonParamRule, paramRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); // Should apply non-parameterized rule first, then parameterized rule assertEquals("test_suffix_param", result.get().after().value()); @@ -122,67 +122,67 @@ public void testMixedRulesInParameterizedExecutor() { public void testParameterizedRuleFailure() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("error_param"); TestNode input = new TestNode("test"); - + // Create a failing parameterized rule ParameterizedRule failingRule = new ParameterizedRule.Async() { @Override public void apply(TestNode node, String param, ActionListener listener) { listener.onFailure(new RuntimeException("Parameterized rule failed with: " + param)); } - + @Override public String name() { return "FailingParameterizedRule"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", failingRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertFailure("Parameterized rule failed with: error_param"); } public void testParameterizedRuleExecutionOrder() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ORDER"); TestNode input = new TestNode(""); - + List executionOrder = new ArrayList<>(); - + ParameterizedRule rule1 = new ParameterizedRule.Sync() { @Override public TestNode apply(TestNode node, String param) { executionOrder.add("rule1_" + param); return new TestNode(node.value() + "1", node.children()); } - + @Override public String name() { return "ParamRule1"; } }; - + ParameterizedRule rule2 = new ParameterizedRule.Sync() { @Override public TestNode apply(TestNode node, String param) { executionOrder.add("rule2_" + param); return new TestNode(node.value() + "2", node.children()); } - + @Override public String name() { return "ParamRule2"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("12", result.get().after().value()); assertEquals(Arrays.asList("rule1_ORDER", "rule2_ORDER"), executionOrder); @@ -191,29 +191,29 @@ public String name() { public void testParameterizedRuleNoChange() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ignored"); TestNode input = new TestNode("test"); - + ParameterizedRule noChangeRule = new ParameterizedRule.Sync() { @Override public TestNode apply(TestNode node, String param) { return node; // No change regardless of parameter } - + @Override public String name() { return "NoChangeParamRule"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test", result.get().after().value()); assertEquals(input, result.get().after()); // Same instance since no change - + // Check that transformation was recorded but marked as no change var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); @@ -225,24 +225,24 @@ public String name() { public void testParameterizedRuleWithMultipleBatches() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("middle"); TestNode input = new TestNode("test"); - + // First batch with parameterized rule TestParameterizedRule paramRule = new TestParameterizedRule(); RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("PrependBatch", paramRule); - + // Second batch with non-parameterized rule AppendRule appendRule = new AppendRule("_final"); RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("AppendBatch", appendRule); - + executor.batches.add(batch1); executor.batches.add(batch2); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test_middle_final", result.get().after().value()); - + // Should have transformations from both batches var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(2)); @@ -251,14 +251,14 @@ public void testParameterizedRuleWithMultipleBatches() { public void testParameterizedExecuteShortcut() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("shortcut"); TestNode input = new TestNode("test"); - + TestParameterizedRule rule = new TestParameterizedRule(); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", rule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.execute(input, result.listener()); - + result.assertSuccess(); assertEquals("test_shortcut", result.get().value()); } @@ -266,26 +266,26 @@ public void testParameterizedExecuteShortcut() { public void testParameterizedAsyncRule() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("async_param"); TestNode input = new TestNode("test"); - + ParameterizedRule asyncRule = new ParameterizedRule.Async() { @Override public void apply(TestNode node, String param, ActionListener listener) { // Simulate async processing listener.onResponse(new TestNode("async_" + node.value() + "_" + param, node.children())); } - + @Override public String name() { return "AsyncParameterizedRule"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("async_test_async_param", result.get().after().value()); } @@ -293,17 +293,17 @@ public String name() { public void testParameterizedRuleContextAccess() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("context_value"); TestNode input = new TestNode("test"); - + // Verify that the executor's context is correctly passed to rules assertEquals("context_value", executor.context()); - + TestParameterizedRule rule = new TestParameterizedRule(); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test_context_value", result.get().after().value()); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java index e03d30acc18fc..2e59eefe56a68 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java @@ -18,7 +18,7 @@ public class RuleExecutorTests extends AbstractRuleTestCase { // Test RuleExecutor implementation static class TestRuleExecutor extends RuleExecutor { public List> batches = new ArrayList<>(); - + @Override public List> batches() { return batches; @@ -28,13 +28,13 @@ public List> batches() { public void testBasicSyncRuleExecution() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("test"); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "success")); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("success", result.get().after().value()); } @@ -42,19 +42,20 @@ public void testBasicSyncRuleExecution() { public void testMultipleRulesInBatch() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("start"); - - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>( + "TestBatch", new ConditionalRule("start", "middle"), new ConditionalRule("middle", "end") ); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("end", result.get().after().value()); - + // Check transformations var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); @@ -68,13 +69,13 @@ public void testAsyncRuleExecution() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("test"); CountingAsyncRule asyncRule = new CountingAsyncRule("_async"); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test_async", result.get().after().value()); assertEquals(1, asyncRule.getCallCount()); @@ -83,54 +84,54 @@ public void testAsyncRuleExecution() { public void testRuleFailure() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("test"); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", new FailingRule("Test error")); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertFailure("Test error"); } public void testRuleExecutionOrder() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("A"); - + List executionOrder = new ArrayList<>(); - + Rule rule1 = new Rule.Sync() { @Override public TestNode apply(TestNode node) { executionOrder.add("rule1"); return new TestNode("B", node.children()); } - + @Override public String name() { return "Rule1"; } }; - + Rule rule2 = new Rule.Sync() { @Override public TestNode apply(TestNode node) { executionOrder.add("rule2"); return new TestNode("C", node.children()); } - + @Override public String name() { return "Rule2"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("C", result.get().after().value()); assertEquals(Arrays.asList("rule1", "rule2"), executionOrder); @@ -139,29 +140,29 @@ public String name() { public void testNoChangeRule() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("test"); - + Rule noChangeRule = new Rule.Sync() { @Override public TestNode apply(TestNode node) { return node; // No change } - + @Override public String name() { return "NoChange"; } }; - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("test", result.get().after().value()); assertEquals(input, result.get().after()); // Same instance since no change - + // Check that transformation was recorded but marked as no change var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); @@ -173,13 +174,13 @@ public String name() { public void testExecuteShortcut() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("test"); - + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "done")); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.execute(input, result.listener()); - + result.assertSuccess(); assertEquals("done", result.get().value()); } @@ -187,19 +188,19 @@ public void testExecuteShortcut() { public void testMultipleBatches() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("start"); - + RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("Batch1", new ConditionalRule("start", "middle")); RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("Batch2", new ConditionalRule("middle", "end")); - + executor.batches.add(batch1); executor.batches.add(batch2); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); assertEquals("end", result.get().after().value()); - + // Should have transformations from both batches var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(2)); @@ -208,33 +209,34 @@ public void testMultipleBatches() { public void testTransformationTracking() { TestRuleExecutor executor = new TestRuleExecutor(); TestNode input = new TestNode("original"); - - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TrackingBatch", + + RuleExecutor.Batch batch = new RuleExecutor.Batch<>( + "TrackingBatch", new ConditionalRule("original", "modified"), new ConditionalRule("modified", "final") ); executor.batches.add(batch); - + AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); - + result.assertSuccess(); - + TestRuleExecutor.ExecutionInfo info = result.get(); assertEquals("original", info.before().value()); assertEquals("final", info.after().value()); - + var transformations = info.transformations(); assertThat(transformations.keySet().size(), equalTo(1)); var batchTransformations = transformations.values().iterator().next(); assertThat(batchTransformations.size(), equalTo(2)); - + TestRuleExecutor.Transformation first = batchTransformations.get(0); assertEquals("ConditionaloriginalTomodified", first.name()); assertEquals("original", first.before().value()); assertEquals("modified", first.after().value()); assertTrue("Should have changed", first.hasChanged()); - + TestRuleExecutor.Transformation second = batchTransformations.get(1); assertEquals("ConditionalmodifiedTofinal", second.name()); assertEquals("modified", second.before().value()); From 36466fa643943ef9610c75dd8b7dbb0acb142b51 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 18:58:02 +0200 Subject: [PATCH 06/17] Lint changes --- .../org/elasticsearch/xpack/esql/rule/ParameterizedRule.java | 2 +- .../java/org/elasticsearch/xpack/esql/rule/RuleExecutor.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java index e0b4ce611c8f7..166d918168839 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/rule/ParameterizedRule.java @@ -12,7 +12,7 @@ public interface ParameterizedRule, P> extends Rule { - public abstract void apply(T t, P p, ActionListener listener); + void apply(T t, P p, ActionListener listener); /** * Abstract base class for asynchronous parameterized rules that use ActionListener callbacks. 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 373d725bc2eb9..a14cc63190af2 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 @@ -171,7 +171,7 @@ private void executeBatches( long totalDuration, ActionListener listener ) { - if (!batchIterator.hasNext()) { + if (batchIterator.hasNext() == false) { TreeType finalPlan = currentPlan.get(); if (false == finalPlan.equals(originalPlan) && log.isDebugEnabled()) { log.debug( From 56f989772772df0361c03793269b2e154342d1fa Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 18:58:28 +0200 Subject: [PATCH 07/17] Make analyzer async. --- .../_nightly/esql/QueryPlanningBenchmark.java | 23 +++++-- .../xpack/esql/analysis/Analyzer.java | 9 ++- .../xpack/esql/session/EsqlSession.java | 60 +++++++++++-------- 3 files changed, 59 insertions(+), 33 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java index d2811962dd29d..eb329e406c935 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java @@ -9,6 +9,9 @@ package org.elasticsearch.benchmark._nightly.esql; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.logging.LogConfigurator; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexMode; @@ -49,6 +52,7 @@ import java.util.LinkedHashMap; import java.util.Locale; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static java.util.Collections.emptyMap; @@ -115,15 +119,22 @@ public void setup() { defaultOptimizer = new LogicalPlanOptimizer(new LogicalOptimizerContext(config, FoldContext.small())); } - private LogicalPlan plan(EsqlParser parser, Analyzer analyzer, LogicalPlanOptimizer optimizer, String query) { + private void plan( + EsqlParser parser, + Analyzer analyzer, + LogicalPlanOptimizer optimizer, + String query, + ActionListener listener + ) { var parsed = parser.createStatement(query, new QueryParams(), telemetry); - var analyzed = analyzer.analyze(parsed); - var optimized = optimizer.optimize(analyzed); - return optimized; + SubscribableListener.newForked(analyzedPlanListener -> analyzer.analyze(parsed, analyzedPlanListener)) + .addListener(listener.map(optimizer::optimize)); } @Benchmark - public void manyFields(Blackhole blackhole) { - blackhole.consume(plan(defaultParser, manyFieldsAnalyzer, defaultOptimizer, "FROM test | LIMIT 10")); + public void manyFields(Blackhole blackhole) throws ExecutionException, InterruptedException { + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + plan(defaultParser, manyFieldsAnalyzer, defaultOptimizer, "FROM test | LIMIT 10", optimizedPlanFuture); + blackhole.consume(optimizedPlanFuture.get()); } } 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 d34d9a5b37735..b428780d5d64a 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.analysis; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.logging.HeaderWarning; import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.lucene.BytesRefs; @@ -194,9 +195,11 @@ public Analyzer(AnalyzerContext context, Verifier verifier) { this.verifier = verifier; } - public LogicalPlan analyze(LogicalPlan plan) { - BitSet partialMetrics = new BitSet(FeatureMetric.values().length); - return verify(execute(plan), gatherPreAnalysisMetrics(plan, partialMetrics)); + public void analyze(LogicalPlan plan, ActionListener listener) { + execute(plan, listener.map(analzedPlan -> { + BitSet partialMetrics = new BitSet(FeatureMetric.values().length); + return this.verify(analzedPlan, gatherPreAnalysisMetrics(plan, partialMetrics)); + })); } public LogicalPlan verify(LogicalPlan plan, BitSet partialMetrics) { 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 b6dd0c40f3481..599a53d8eb8f8 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 @@ -108,7 +108,7 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; @@ -342,14 +342,22 @@ public void analyzedPlan( return; } - Function analyzeAction = (l) -> { + BiConsumer> analyzeAction = (preAnalysisResult, listener) -> { Analyzer analyzer = new Analyzer( - new AnalyzerContext(configuration, functionRegistry, l.indices, l.lookupIndices, l.enrichResolution, l.inferenceResolution), + new AnalyzerContext( + configuration, + functionRegistry, + preAnalysisResult.indices, + preAnalysisResult.lookupIndices, + preAnalysisResult.enrichResolution, + preAnalysisResult.inferenceResolution + ), verifier ); - LogicalPlan plan = analyzer.analyze(parsed); - plan.setAnalyzed(); - return plan; + analyzer.analyze(parsed, listener.map(analyzedPlan -> { + analyzedPlan.setAnalyzed(); + return analyzedPlan; + })); }; // Capture configured remotes list to ensure consistency throughout the session configuredClusters = Set.copyOf(indicesExpressionGrouper.getConfiguredClusters()); @@ -404,13 +412,14 @@ public void analyzedPlan( // do we want to declare it successful or skipped? For now, unavailability takes precedence. EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, result.indices.unavailableClusters()); EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, result.indices, null); - plan = analyzeAction.apply(result); + analyzeAction.accept(result, l.delegateFailureAndWrap((analyzedPlanListener, analyzedPlan) -> { + LOGGER.debug("Analyzed plan (second attempt, without filter):\n{}", analyzedPlan); + analyzedPlanListener.onResponse(analyzedPlan); + })); } catch (Exception e) { l.onFailure(e); return; } - LOGGER.debug("Analyzed plan (second attempt, without filter):\n{}", plan); - l.onResponse(plan); }).addListener(logicalPlanListener); } @@ -528,7 +537,7 @@ private boolean allCCSClustersSkipped( } private static void analyzeAndMaybeRetry( - Function analyzeAction, + BiConsumer> analyzeAction, QueryBuilder requestFilter, PreAnalysisResult result, EsqlExecutionInfo executionInfo, @@ -540,14 +549,7 @@ private static void analyzeAndMaybeRetry( var attemptMessage = requestFilter == null ? "the only" : "first"; LOGGER.debug("Analyzing the plan ({} attempt, {} filter)", attemptMessage, filterPresentMessage); - try { - if (result.indices.isValid() || requestFilter != null) { - // We won't run this check with no filter and no valid indices since this may lead to false positive - missing index report - // when the resolution result is not valid for a different reason. - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, result.indices, requestFilter); - } - plan = analyzeAction.apply(result); - } catch (Exception e) { + logicalPlanListener = logicalPlanListener.delegateResponse((errorListener, e) -> { if (e instanceof VerificationException ve) { LOGGER.debug( "Analyzing the plan ({} attempt, {} filter) failed with {}", @@ -557,7 +559,7 @@ private static void analyzeAndMaybeRetry( ); if (requestFilter == null) { // if the initial request didn't have a filter, then just pass the exception back to the user - logicalPlanListener.onFailure(ve); + errorListener.onFailure(ve); } else { // interested only in a VerificationException, but this time we are taking out the index filter // to try and make the index resolution work without any index filtering. In the next step... to be continued @@ -565,13 +567,23 @@ private static void analyzeAndMaybeRetry( } } else { // if the query failed with any other type of exception, then just pass the exception back to the user - logicalPlanListener.onFailure(e); + errorListener.onFailure(e); } - return; + }); + + try { + if (result.indices.isValid() || requestFilter != null) { + // We won't run this check with no filter and no valid indices since this may lead to false positive - missing index report + // when the resolution result is not valid for a different reason. + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, result.indices, requestFilter); + } + analyzeAction.accept(result, ActionListener.runBefore(logicalPlanListener, () -> { + LOGGER.debug("Analyzed plan ({} attempt, {} filter):\n{}", attemptMessage, filterPresentMessage, plan); + // the analysis succeeded from the first attempt, irrespective if it had a filter or not, just continue with the planning + })); + } catch (Exception e) { + l.onFailure(e); } - LOGGER.debug("Analyzed plan ({} attempt, {} filter):\n{}", attemptMessage, filterPresentMessage, plan); - // the analysis succeeded from the first attempt, irrespective if it had a filter or not, just continue with the planning - logicalPlanListener.onResponse(plan); } private static void resolveFieldNames(LogicalPlan parsed, EnrichResolution enrichResolution, ActionListener l) { From d067c7863c09d3b4c1df115b8603b2dcd2567605 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 19:39:54 +0200 Subject: [PATCH 08/17] Update analysis tests. --- .../elasticsearch/xpack/esql/CsvTests.java | 4 +- .../esql/analysis/AnalyzerTestUtils.java | 13 ++++-- .../xpack/esql/analysis/AnalyzerTests.java | 18 +++++---- .../xpack/esql/analysis/ParsingTests.java | 3 +- .../xpack/esql/analysis/VerifierTests.java | 5 ++- .../function/CheckLicenseTests.java | 3 +- .../AbstractLogicalPlanOptimizerTests.java | 11 ++--- .../LocalLogicalPlanOptimizerTests.java | 5 ++- .../optimizer/LogicalPlanOptimizerTests.java | 40 +++++++++---------- .../optimizer/PhysicalPlanOptimizerTests.java | 2 +- .../esql/optimizer/TestPlannerOptimizer.java | 4 +- .../logical/PropagateInlineEvalsTests.java | 3 +- .../xpack/esql/planner/FilterTests.java | 3 +- .../esql/plugin/ClusterRequestTests.java | 3 +- .../DataNodeRequestSerializationTests.java | 3 +- .../esql/telemetry/VerifierMetricsTests.java | 3 +- 16 files changed, 73 insertions(+), 50 deletions(-) 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 9062bdef62d76..95e0a2e4fff2b 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 @@ -509,7 +509,9 @@ private LogicalPlan analyzedPlan(LogicalPlan parsed, CsvTestsDataLoader.MultiInd new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies, emptyInferenceResolution()), TEST_VERIFIER ); - LogicalPlan plan = analyzer.analyze(parsed); + PlainActionFuture analyzedPlanFuture = new PlainActionFuture<>(); + analyzer.analyze(parsed, analyzedPlanFuture); + LogicalPlan plan = analyzedPlanFuture.actionGet(); plan.setAnalyzed(); LOGGER.debug("Analyzed plan:\n{}", plan); return plan; 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 5e6c37545a396..6f9d00b4e2b58 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.analysis; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.index.IndexMode; import org.elasticsearch.inference.TaskType; import org.elasticsearch.xpack.core.enrich.EnrichPolicy; @@ -131,15 +132,19 @@ public static LogicalPlan analyze(String query, String index, String mapping) { public static LogicalPlan analyze(String query, Analyzer analyzer) { var plan = new EsqlParser().createStatement(query); // System.out.println(plan); - var analyzed = analyzer.analyze(plan); - // System.out.println(analyzed); - return analyzed; + return analyze(analyzer, plan); } public static LogicalPlan analyze(String query, String mapping, QueryParams params) { var plan = new EsqlParser().createStatement(query, params); var analyzer = analyzer(loadMapping(mapping, "test"), TEST_VERIFIER, configuration(query)); - return analyzer.analyze(plan); + return analyze(analyzer, plan); + } + + public static LogicalPlan analyze(Analyzer analyzer, LogicalPlan plan) { + PlainActionFuture analyzedPlanFuture = new PlainActionFuture<>(); + analyzer.analyze(plan, analyzedPlanFuture); + return analyzedPlanFuture.actionGet(); } public static IndexResolution loadMapping(String resource, String indexName, IndexMode indexMode) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index 6ae6c6eea3b6f..a44f7e9b92018 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -159,7 +159,7 @@ public class AnalyzerTests extends ESTestCase { public void testIndexResolution() { EsIndex idx = new EsIndex("idx", Map.of()); Analyzer analyzer = analyzer(IndexResolution.valid(idx)); - var plan = analyzer.analyze(UNRESOLVED_RELATION); + var plan = analyze(analyzer, UNRESOLVED_RELATION); var limit = as(plan, Limit.class); assertEquals(new EsRelation(EMPTY, idx.name(), IndexMode.STANDARD, idx.indexNameWithModes(), NO_FIELDS), limit.child()); @@ -168,7 +168,7 @@ public void testIndexResolution() { public void testFailOnUnresolvedIndex() { Analyzer analyzer = analyzer(IndexResolution.invalid("Unknown index [idx]")); - VerificationException e = expectThrows(VerificationException.class, () -> analyzer.analyze(UNRESOLVED_RELATION)); + VerificationException e = expectThrows(VerificationException.class, () -> analyze(analyzer, UNRESOLVED_RELATION)); assertThat(e.getMessage(), containsString("Unknown index [idx]")); } @@ -177,7 +177,7 @@ public void testIndexWithClusterResolution() { EsIndex idx = new EsIndex("cluster:idx", Map.of()); Analyzer analyzer = analyzer(IndexResolution.valid(idx)); - var plan = analyzer.analyze(UNRESOLVED_RELATION); + var plan = analyze(analyzer, UNRESOLVED_RELATION); var limit = as(plan, Limit.class); assertEquals(new EsRelation(EMPTY, idx.name(), IndexMode.STANDARD, idx.indexNameWithModes(), NO_FIELDS), limit.child()); @@ -187,7 +187,8 @@ public void testAttributeResolution() { EsIndex idx = new EsIndex("idx", LoadMapping.loadMapping("mapping-one-field.json")); Analyzer analyzer = analyzer(IndexResolution.valid(idx)); - var plan = analyzer.analyze( + var plan = analyze( + analyzer, new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no")))) ); @@ -208,7 +209,8 @@ public void testAttributeResolution() { public void testAttributeResolutionOfChainedReferences() { Analyzer analyzer = analyzer(loadMapping("mapping-one-field.json", "idx")); - var plan = analyzer.analyze( + var plan = analyze( + analyzer, new Eval( EMPTY, new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no")))), @@ -240,7 +242,8 @@ public void testRowAttributeResolution() { EsIndex idx = new EsIndex("idx", Map.of()); Analyzer analyzer = analyzer(IndexResolution.valid(idx)); - var plan = analyzer.analyze( + var plan = analyze( + analyzer, new Eval( EMPTY, new Row(EMPTY, List.of(new Alias(EMPTY, "emp_no", new Literal(EMPTY, 1, DataType.INTEGER)))), @@ -271,7 +274,8 @@ public void testUnresolvableAttribute() { VerificationException ve = expectThrows( VerificationException.class, - () -> analyzer.analyze( + () -> analyze( + analyzer, new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_nos")))) ) ); 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 348c4d4266b8e..0f07dc3b0fea5 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 @@ -36,6 +36,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyInferenceResolution; import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyPolicyResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -182,7 +183,7 @@ public void testInvalidSample() { } private String error(String query) { - ParsingException e = expectThrows(ParsingException.class, () -> defaultAnalyzer.analyze(parser.createStatement(query))); + ParsingException e = expectThrows(ParsingException.class, () -> analyze(defaultAnalyzer, parser.createStatement(query))); String message = e.getMessage(); assertTrue(message.startsWith("line ")); return message.substring("line ".length()); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 6555a303592f3..02b18934b7f76 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -39,6 +39,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.paramAsConstant; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.esql.core.type.DataType.CARTESIAN_POINT; @@ -2241,7 +2242,7 @@ private void query(String query) { } private void query(String query, Analyzer analyzer) { - analyzer.analyze(parser.createStatement(query)); + analyze(analyzer, parser.createStatement(query)); } private String error(String query) { @@ -2272,7 +2273,7 @@ private String error(String query, Analyzer analyzer, Class Throwable e = expectThrows( exception, "Expected error for query [" + query + "] but no error was raised", - () -> analyzer.analyze(parser.createStatement(query, new QueryParams(parameters))) + () -> analyze(analyzer, parser.createStatement(query, new QueryParams(parameters))) ); assertThat(e, instanceOf(exception)); 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 68a6f38cdd69a..041ad59301cec 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 @@ -19,6 +19,7 @@ import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils; import org.elasticsearch.xpack.esql.analysis.Verifier; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FoldContext; @@ -86,7 +87,7 @@ public EsqlFunctionRegistry snapshotRegistry() { ? new LicensedLimit(l.source(), l.limit(), l.child(), functionLicenseFeature) : l ); - return analyzer(registry, operationMode).analyze(plan); + return AnalyzerTestUtils.analyze(analyzer(registry, operationMode), plan); } private static Analyzer analyzer(EsqlFunctionRegistry registry, License.OperationMode operationMode) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java index bff9843a89e1f..8ecad1200c661 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java @@ -32,6 +32,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; @@ -180,7 +181,7 @@ protected LogicalPlan plan(String query) { } protected LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { - var analyzed = analyzer.analyze(parser.createStatement(query)); + var analyzed = analyze(analyzer, parser.createStatement(query)); // System.out.println(analyzed); var optimized = optimizer.optimize(analyzed); // System.out.println(optimized); @@ -188,7 +189,7 @@ protected LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { } protected LogicalPlan planAirports(String query) { - var analyzed = analyzerAirports.analyze(parser.createStatement(query)); + var analyzed = analyze(analyzerAirports, parser.createStatement(query)); // System.out.println(analyzed); var optimized = logicalOptimizer.optimize(analyzed); // System.out.println(optimized); @@ -196,7 +197,7 @@ protected LogicalPlan planAirports(String query) { } protected LogicalPlan planExtra(String query) { - var analyzed = analyzerExtra.analyze(parser.createStatement(query)); + var analyzed = analyze(analyzerExtra, parser.createStatement(query)); // System.out.println(analyzed); var optimized = logicalOptimizer.optimize(analyzed); // System.out.println(optimized); @@ -204,11 +205,11 @@ protected LogicalPlan planExtra(String query) { } protected LogicalPlan planTypes(String query) { - return logicalOptimizer.optimize(analyzerTypes.analyze(parser.createStatement(query))); + return logicalOptimizer.optimize(analyze(analyzerTypes, parser.createStatement(query))); } protected LogicalPlan planMultiIndex(String query) { - return logicalOptimizer.optimize(multiIndexAnalyzer.analyze(parser.createStatement(query))); + return logicalOptimizer.optimize(analyze(multiIndexAnalyzer, parser.createStatement(query))); } @Override 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 f2b70c99253b8..7d87814065671 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 @@ -82,6 +82,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.statsForMissingField; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; @@ -508,7 +509,7 @@ public void testSparseDocument() throws Exception { TEST_VERIFIER ); - var analyzed = analyzer.analyze(parser.createStatement(query)); + var analyzed = analyze(analyzer, 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); @@ -785,7 +786,7 @@ private LocalRelation asEmptyRelation(Object o) { } private LogicalPlan plan(String query, Analyzer analyzer) { - var analyzed = analyzer.analyze(parser.createStatement(query)); + var analyzed = analyze(analyzer, parser.createStatement(query)); // System.out.println(analyzed); var optimized = logicalOptimizer.optimize(analyzed); // System.out.println(optimized); 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 de49630541599..1b412e7655806 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 @@ -5254,20 +5254,20 @@ public void testEmptyMappingIndex() { TEST_VERIFIER ); - var plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test"))); + var plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test"))); as(plan, LocalRelation.class); assertThat(plan.output(), equalTo(NO_FIELDS)); - plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test metadata _id | eval x = 1"))); + plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test metadata _id | eval x = 1"))); as(plan, LocalRelation.class); assertThat(Expressions.names(plan.output()), contains("_id", "x")); - plan = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement("from empty_test metadata _id, _version | limit 5"))); + plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test metadata _id, _version | limit 5"))); as(plan, LocalRelation.class); assertThat(Expressions.names(plan.output()), contains("_id", "_version")); plan = logicalOptimizer.optimize( - analyzer.analyze(parser.createStatement("from empty_test | eval x = \"abc\" | enrich languages_idx on x")) + analyze(analyzer, parser.createStatement("from empty_test | eval x = \"abc\" | enrich languages_idx on x")) ); LocalRelation local = as(plan, LocalRelation.class); assertThat(Expressions.names(local.output()), contains(NO_FIELDS.get(0).name(), "x", "language_code", "language_name")); @@ -5962,7 +5962,7 @@ private void doTestSimplifyComparisonArithmetics( private void assertSemanticMatching(String expected, String provided) { BinaryComparison bc = extractPlannedBinaryComparison(provided); - LogicalPlan exp = analyzerTypes.analyze(parser.createStatement("FROM types | WHERE " + expected)); + LogicalPlan exp = analyze(analyzerTypes, parser.createStatement("FROM types | WHERE " + expected)); assertSemanticMatching(bc, extractPlannedBinaryComparison(exp)); } @@ -5990,7 +5990,7 @@ private Expression getComparisonFromLogicalPlan(LogicalPlan plan) { private void assertNotSimplified(String comparison) { String query = "FROM types | WHERE " + comparison; Expression optimized = getComparisonFromLogicalPlan(planTypes(query)); - Expression raw = getComparisonFromLogicalPlan(analyzerTypes.analyze(parser.createStatement(query))); + Expression raw = getComparisonFromLogicalPlan(analyze(analyzerTypes, parser.createStatement(query))); assertTrue(raw.semanticEquals(optimized)); } @@ -6683,7 +6683,7 @@ public void testMultipleLookupShadowing() { public void testTranslateMetricsWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in))"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAggs = as(limit.child(), Aggregate.class); assertThat(finalAggs, not(instanceOf(TimeSeriesAggregate.class))); @@ -6704,7 +6704,7 @@ public void testTranslateMetricsWithoutGrouping() { public void testTranslateMixedAggsWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in)), max(network.cost)"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAggs = as(limit.child(), Aggregate.class); assertThat(finalAggs, not(instanceOf(TimeSeriesAggregate.class))); @@ -6729,7 +6729,7 @@ public void testTranslateMixedAggsWithoutGrouping() { public void testTranslateMixedAggsWithMathWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in)), max(network.cost + 0.2) * 1.1"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); Eval mulEval = as(project.child(), Eval.class); assertThat(mulEval.fields(), hasSize(1)); @@ -6767,7 +6767,7 @@ public void testTranslateMixedAggsWithMathWithoutGrouping() { public void testTranslateMetricsGroupedByOneDimension() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(rate(network.total_bytes_in)) BY cluster | SORT cluster | LIMIT 10"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); TopN topN = as(plan, TopN.class); Aggregate aggsByCluster = as(topN.child(), Aggregate.class); assertThat(aggsByCluster, not(instanceOf(TimeSeriesAggregate.class))); @@ -6792,7 +6792,7 @@ public void testTranslateMetricsGroupedByOneDimension() { public void testTranslateMetricsGroupedByTwoDimension() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS avg(rate(network.total_bytes_in)) BY cluster, pod"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); Eval eval = as(project.child(), Eval.class); assertThat(eval.fields(), hasSize(1)); @@ -6832,7 +6832,7 @@ public void testTranslateMetricsGroupedByTwoDimension() { public void testTranslateMetricsGroupedByTimeBucket() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(rate(network.total_bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -6866,7 +6866,7 @@ public void testTranslateMetricsGroupedByTimeBucketAndDimensions() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval eval = as(topN.child(), Eval.class); @@ -6908,7 +6908,7 @@ public void testTranslateSumOfTwoRates() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); TopN topN = as(plan, TopN.class); Aggregate finalAgg = as(topN.child(), Aggregate.class); Eval eval = as(finalAgg.child(), Eval.class); @@ -6929,7 +6929,7 @@ public void testTranslateMixedAggsGroupedByTimeBucketAndDimensions() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval eval = as(topN.child(), Eval.class); @@ -6981,7 +6981,7 @@ public void testAdjustMetricsRateBeforeFinalAgg() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval evalDiv = as(topN.child(), Eval.class); @@ -7034,7 +7034,7 @@ public void testAdjustMetricsRateBeforeFinalAgg() { public void testTranslateMaxOverTime() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(max_over_time(network.bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -7063,7 +7063,7 @@ public void testTranslateMaxOverTime() { public void testTranslateAvgOverTime() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(avg_over_time(network.bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -7103,7 +7103,7 @@ public void testMetricsWithoutRate() { """); List plans = new ArrayList<>(); for (String query : queries) { - var plan = logicalOptimizer.optimize(metricsAnalyzer.analyze(parser.createStatement(query))); + var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); plans.add(plan); } for (LogicalPlan plan : plans) { @@ -7694,7 +7694,7 @@ public void testPruneRedundantOrderBy() { | mv_expand x | sort y """; - LogicalPlan analyzed = analyzer.analyze(parser.createStatement(query)); + LogicalPlan analyzed = analyze(analyzer, parser.createStatement(query)); LogicalPlan optimized = rule.apply(analyzed); // check that all the redundant SORTs are removed in a single run 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 99eded20b1687..7fc9a0f8acd09 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 @@ -8247,7 +8247,7 @@ private PhysicalPlan physicalPlan(String query, TestDataSource dataSource) { } private PhysicalPlan physicalPlan(String query, TestDataSource dataSource, boolean assertSerialization) { - var logical = logicalOptimizer.optimize(dataSource.analyzer.analyze(parser.createStatement(query))); + var logical = logicalOptimizer.optimize(analyze(dataSource.analyzer, parser.createStatement(query))); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); // System.out.println("Physical\n" + physical); 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..7f9ded50cc93b 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 @@ -18,6 +18,8 @@ import org.elasticsearch.xpack.esql.session.Configuration; import org.elasticsearch.xpack.esql.stats.SearchStats; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; + public class TestPlannerOptimizer { private final EsqlParser parser; private final Analyzer analyzer; @@ -79,7 +81,7 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { } private PhysicalPlan physicalPlan(String query, Analyzer analyzer) { - var logical = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement(query))); + var logical = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query))); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); return physical; 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 088b5c1c9205e..9ac8bcbff6b9c 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 @@ -40,6 +40,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultInferenceResolution; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; import static org.hamcrest.Matchers.contains; @@ -169,7 +170,7 @@ public void testGroupingAliasingMoved_To_LeftSideOfJoin_WithExpression() { } private LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { - return optimizer.optimize(analyzer.analyze(parser.createStatement(query))); + return optimizer.optimize(analyze(analyzer, parser.createStatement(query))); } @Override 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 69060f9cee0f2..41278840f28e4 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 @@ -58,6 +58,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.core.querydsl.query.Query.unscore; import static org.elasticsearch.xpack.esql.core.util.Queries.Clause.FILTER; import static org.elasticsearch.xpack.esql.core.util.Queries.Clause.MUST; @@ -378,7 +379,7 @@ public static QueryBuilder singleValueQuery(String query, QueryBuilder inner, St } private PhysicalPlan plan(String query, QueryBuilder restFilter) { - var logical = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement(query))); + var logical = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query))); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); // System.out.println("physical\n" + physical); 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 ababc8ed37657..10dcb10e3bb98 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 @@ -44,6 +44,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.hamcrest.Matchers.equalTo; public class ClusterRequestTests extends AbstractWireSerializingTestCase { @@ -201,7 +202,7 @@ static LogicalPlan parse(String query) { ), TEST_VERIFIER ); - return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); + return logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query))); } @Override 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 1fc481711df97..70d5173ad848e 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 @@ -48,6 +48,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyPolicyResolution; import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; public class DataNodeRequestSerializationTests extends AbstractWireSerializingTestCase { @@ -302,7 +303,7 @@ static LogicalPlan parse(String query) { ), TEST_VERIFIER ); - return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); + return logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query))); } static PhysicalPlan mapAndMaybeOptimize(LogicalPlan logicalPlan) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java index 0fee82f4f6ee7..38df3fe0a7e01 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java @@ -21,6 +21,7 @@ import java.util.Set; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyzer; import static org.elasticsearch.xpack.esql.telemetry.FeatureMetric.DISSECT; import static org.elasticsearch.xpack.esql.telemetry.FeatureMetric.DROP; @@ -554,7 +555,7 @@ private Counters esql(String esql, Verifier v) { metrics = new Metrics(new EsqlFunctionRegistry()); verifier = new Verifier(metrics, new XPackLicenseState(() -> 0L)); } - analyzer(verifier).analyze(parser.createStatement(esql)); + analyze(analyzer(verifier), parser.createStatement(esql)); return metrics == null ? null : metrics.stats(); } From 179d782420512867189ff901ec8cba9a7445d0b9 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 21:30:10 +0200 Subject: [PATCH 09/17] Lint --- .../elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java | 2 +- .../xpack/esql/rule/ParameterizedRuleExecutorTests.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java index 9d2444b300e8f..6de0f4020a6f8 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -75,7 +75,7 @@ public String getWriteableName() { @Override public boolean equals(Object obj) { if (this == obj) return true; - if (!(obj instanceof TestNode)) return false; + if ((obj instanceof TestNode) == false) return false; TestNode other = (TestNode) obj; return value.equals(other.value) && children.equals(other.children); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java index 9c8802c52680d..563c38916db4b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -21,7 +21,7 @@ public class ParameterizedRuleExecutorTests extends AbstractRuleTestCase { static class TestParameterizedRuleExecutor extends ParameterizedRuleExecutor { public List> batches = new ArrayList<>(); - public TestParameterizedRuleExecutor(String context) { + TestParameterizedRuleExecutor(String context) { super(context); } @@ -34,7 +34,7 @@ public List> batches() { static class TestContextParameterizedRuleExecutor extends ParameterizedRuleExecutor { public List> batches = new ArrayList<>(); - public TestContextParameterizedRuleExecutor(TestContext context) { + TestContextParameterizedRuleExecutor(TestContext context) { super(context); } From 5851dc378f449f2242b7f6f89c75777ffd999864 Mon Sep 17 00:00:00 2001 From: afoucret Date: Mon, 7 Jul 2025 22:57:49 +0200 Subject: [PATCH 10/17] Make LogicalPlanOptimizer::optimize async. --- .../_nightly/esql/QueryPlanningBenchmark.java | 3 +- .../esql/optimizer/LogicalPlanOptimizer.java | 20 +++---- .../xpack/esql/session/EsqlSession.java | 20 +++---- .../elasticsearch/xpack/esql/CsvTests.java | 53 ++++++++++--------- .../AbstractLogicalPlanOptimizerTests.java | 33 ++++++------ .../LocalLogicalPlanOptimizerTests.java | 22 +++++--- .../optimizer/LogicalPlanOptimizerTests.java | 44 ++++++++------- .../optimizer/PhysicalPlanOptimizerTests.java | 17 ++++-- .../esql/optimizer/TestPlannerOptimizer.java | 10 +++- .../logical/PropagateInlineEvalsTests.java | 5 +- .../xpack/esql/planner/FilterTests.java | 6 ++- .../PlanConcurrencyCalculatorTests.java | 8 ++- .../esql/plugin/ClusterRequestTests.java | 5 +- .../DataNodeRequestSerializationTests.java | 5 +- 14 files changed, 150 insertions(+), 101 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java index eb329e406c935..52c388a998c9b 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java @@ -128,7 +128,8 @@ private void plan( ) { var parsed = parser.createStatement(query, new QueryParams(), telemetry); SubscribableListener.newForked(analyzedPlanListener -> analyzer.analyze(parsed, analyzedPlanListener)) - .addListener(listener.map(optimizer::optimize)); + .andThen((optimizedPlanListener, analyzedPlan) -> optimizer.optimize(analyzedPlan, optimizedPlanListener)) + .addListener(listener); } @Benchmark diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java index 14a858f85fd2a..61679d7db3d98 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.common.Failures; import org.elasticsearch.xpack.esql.core.type.DataType; @@ -109,15 +110,16 @@ public LogicalPlanOptimizer(LogicalOptimizerContext optimizerContext) { super(optimizerContext); } - public LogicalPlan optimize(LogicalPlan verified) { - var optimized = execute(verified); - - Failures failures = verifier.verify(optimized); - if (failures.hasFailures()) { - throw new VerificationException(failures); - } - optimized.setOptimized(); - return optimized; + public void optimize(LogicalPlan verified, ActionListener listener) { + execute(verified, listener.delegateFailureAndWrap((l, optimized) -> { + Failures failures = verifier.verify(optimized); + if (failures.hasFailures()) { + l.onFailure(new VerificationException(failures)); + return; + } + optimized.setOptimized(); + l.onResponse(optimized); + })); } @Override 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 599a53d8eb8f8..6846d4cf2c75b 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 @@ -198,10 +198,10 @@ public void execute(EsqlQueryRequest request, EsqlExecutionInfo executionInfo, P analyzedPlan(parsed, executionInfo, request.filter(), new EsqlCCSUtils.CssPartialErrorsActionListener(executionInfo, listener) { @Override public void onResponse(LogicalPlan analyzedPlan) { - preMapper.preMapper( - analyzedPlan, - listener.delegateFailureAndWrap((l, p) -> executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(p), l)) - ); + SubscribableListener.newForked(l -> preMapper.preMapper(analyzedPlan, l)) + .andThen((l, p) -> optimizedPlan(p, l)) + .andThen((l, p) -> executeOptimizedPlan(request, executionInfo, planRunner, p, l)) + .addListener(listener); } }); } @@ -827,13 +827,15 @@ private PhysicalPlan logicalPlanToPhysicalPlan(LogicalPlan optimizedPlan, EsqlQu return EstimatesRowSize.estimateRowSize(0, physicalPlan); } - public LogicalPlan optimizedPlan(LogicalPlan logicalPlan) { + public void optimizedPlan(LogicalPlan logicalPlan, ActionListener listener) { if (logicalPlan.analyzed() == false) { - throw new IllegalStateException("Expected analyzed plan"); + listener.onFailure(new IllegalStateException("Expected analyzed plan")); + return; } - var plan = logicalPlanOptimizer.optimize(logicalPlan); - LOGGER.debug("Optimized logicalPlan plan:\n{}", plan); - return plan; + logicalPlanOptimizer.optimize(logicalPlan, listener.map(optimizedPlan -> { + LOGGER.debug("Optimized logicalPlan plan:\n{}", optimizedPlan); + return optimizedPlan; + })); } public PhysicalPlan physicalPlan(LogicalPlan optimizedPlan) { 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 95e0a2e4fff2b..50dfbddae3a4d 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 @@ -502,19 +502,18 @@ private static EnrichPolicy loadEnrichPolicyMapping(String policyFileName) { } } - private LogicalPlan analyzedPlan(LogicalPlan parsed, CsvTestsDataLoader.MultiIndexTestDataset datasets) { + private void analyzedPlan(LogicalPlan parsed, CsvTestsDataLoader.MultiIndexTestDataset datasets, ActionListener listener) { var indexResolution = loadIndexResolution(datasets); var enrichPolicies = loadEnrichPolicies(); var analyzer = new Analyzer( new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies, emptyInferenceResolution()), TEST_VERIFIER ); - PlainActionFuture analyzedPlanFuture = new PlainActionFuture<>(); - analyzer.analyze(parsed, analyzedPlanFuture); - LogicalPlan plan = analyzedPlanFuture.actionGet(); - plan.setAnalyzed(); - LOGGER.debug("Analyzed plan:\n{}", plan); - return plan; + analyzer.analyze(parsed, listener.map(analyzedPlan -> { + analyzedPlan.setAnalyzed(); + LOGGER.debug("Analyzed plan:\n{}", analyzedPlan); + return analyzedPlan; + })); } private static CsvTestsDataLoader.MultiIndexTestDataset testDatasets(LogicalPlan parsed) { @@ -570,7 +569,6 @@ private static TestPhysicalOperationProviders testOperationProviders( private ActualResults executePlan(BigArrays bigArrays) throws Exception { LogicalPlan parsed = parser.createStatement(testCase.query); var testDatasets = testDatasets(parsed); - LogicalPlan analyzed = analyzedPlan(parsed, testDatasets); FoldContext foldCtx = FoldContext.small(); EsqlSession session = new EsqlSession( @@ -591,24 +589,29 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { PlainActionFuture listener = new PlainActionFuture<>(); - session.executeOptimizedPlan( - new EsqlQueryRequest(), - new EsqlExecutionInfo(randomBoolean()), - planRunner(bigArrays, foldCtx, physicalOperationProviders), - session.optimizedPlan(analyzed), - listener.delegateFailureAndWrap( - // Wrap so we can capture the warnings in the calling thread - (next, result) -> next.onResponse( - new ActualResults( - result.schema().stream().map(Attribute::name).toList(), - result.schema().stream().map(a -> Type.asType(a.dataType().nameUpper())).toList(), - result.schema().stream().map(Attribute::dataType).toList(), - result.pages(), - threadPool.getThreadContext().getResponseHeaders() + analyzedPlan(parsed, testDatasets, listener.delegateFailureAndWrap((analyzedPlanListener, analyzedPlan) -> { + session.optimizedPlan(analyzedPlan, listener.delegateFailureAndWrap((optimizedPlanListener, optimizedPlan) -> { + session.executeOptimizedPlan( + new EsqlQueryRequest(), + new EsqlExecutionInfo(randomBoolean()), + planRunner(bigArrays, foldCtx, physicalOperationProviders), + optimizedPlan, + listener.delegateFailureAndWrap( + // Wrap so we can capture the warnings in the calling thread + (next, result) -> next.onResponse( + new ActualResults( + result.schema().stream().map(Attribute::name).toList(), + result.schema().stream().map(a -> Type.asType(a.dataType().nameUpper())).toList(), + result.schema().stream().map(Attribute::dataType).toList(), + result.pages(), + threadPool.getThreadContext().getResponseHeaders() + ) + ) ) - ) - ) - ); + ); + })); + })); + return listener.get(); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java index 8ecad1200c661..c481fb513cafb 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.index.IndexMode; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.EsqlTestUtils; @@ -181,35 +182,33 @@ protected LogicalPlan plan(String query) { } protected LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { - var analyzed = analyze(analyzer, parser.createStatement(query)); - // System.out.println(analyzed); - var optimized = optimizer.optimize(analyzed); - // System.out.println(optimized); - return optimized; + return optimizedPlan(analyze(analyzer, parser.createStatement(query)), optimizer); } protected LogicalPlan planAirports(String query) { - var analyzed = analyze(analyzerAirports, parser.createStatement(query)); - // System.out.println(analyzed); - var optimized = logicalOptimizer.optimize(analyzed); - // System.out.println(optimized); - return optimized; + return optimizedPlan(analyze(analyzerAirports, parser.createStatement(query))); } protected LogicalPlan planExtra(String query) { - var analyzed = analyze(analyzerExtra, parser.createStatement(query)); - // System.out.println(analyzed); - var optimized = logicalOptimizer.optimize(analyzed); - // System.out.println(optimized); - return optimized; + return optimizedPlan(analyze(analyzerExtra, parser.createStatement(query))); } protected LogicalPlan planTypes(String query) { - return logicalOptimizer.optimize(analyze(analyzerTypes, parser.createStatement(query))); + return optimizedPlan(analyze(analyzerTypes, parser.createStatement(query))); } protected LogicalPlan planMultiIndex(String query) { - return logicalOptimizer.optimize(analyze(multiIndexAnalyzer, parser.createStatement(query))); + return optimizedPlan(analyze(multiIndexAnalyzer, parser.createStatement(query))); + } + + protected LogicalPlan optimizedPlan(LogicalPlan logicalPlan) { + return optimizedPlan(logicalPlan, logicalOptimizer); + } + + protected LogicalPlan optimizedPlan(LogicalPlan logicalPlan, LogicalPlanOptimizer optimizer) { + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + optimizer.optimize(logicalPlan, optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); } @Override 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 7d87814065671..77c92599398c3 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 @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.optimizer; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.Maps; import org.elasticsearch.index.IndexMode; @@ -251,7 +252,7 @@ public void testMissingFieldInSort() { * EsqlProject[[first_name{f}#7, last_name{r}#17]] * \_Limit[1000[INTEGER],true] * \_MvExpand[last_name{f}#10,last_name{r}#17] - * \_Project[[_meta_field{f}#12, emp_no{f}#6, first_name{f}#7, gender{f}#8, hire_date{f}#13, job{f}#14, job.raw{f}#15, lang + * \_Project[[_meta_field{f}#12, emp_no{f}#6, first_name{f}#7, gender{f}#8, hire_date{f}#13, job{f}#14, job.raw{f}#15, langu * uages{f}#9, last_name{r}#10, long_noidx{f}#16, salary{f}#11]] * \_Eval[[null[KEYWORD] AS last_name]] * \_Limit[1000[INTEGER],false] @@ -510,7 +511,7 @@ public void testSparseDocument() throws Exception { ); var analyzed = analyze(analyzer, parser.createStatement(query)); - var optimized = logicalOptimizer.optimize(analyzed); + var optimized = optimizedPlan(analyzed); var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); var plan = new LocalLogicalPlanOptimizer(localContext).localOptimize(optimized); @@ -786,17 +787,24 @@ private LocalRelation asEmptyRelation(Object o) { } private LogicalPlan plan(String query, Analyzer analyzer) { - var analyzed = analyze(analyzer, parser.createStatement(query)); - // System.out.println(analyzed); - var optimized = logicalOptimizer.optimize(analyzed); - // System.out.println(optimized); - return optimized; + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + analyzer.analyze( + parser.createStatement(query), + optimizedPlanFuture.delegateFailureAndWrap((l, analyzedPlan) -> logicalOptimizer.optimize(analyzedPlan, l)) + ); + return optimizedPlanFuture.actionGet(); } private LogicalPlan plan(String query) { return plan(query, analyzer); } + private LogicalPlan optimizedPlan(LogicalPlan logicalPlan) { + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(logicalPlan, optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); + } + private LogicalPlan localPlan(LogicalPlan plan, SearchStats searchStats) { var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); // System.out.println(plan); 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 1b412e7655806..4fc52fb08dadf 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 @@ -1276,7 +1276,7 @@ public void testMultipleCombineLimits() { var value = i == limitWithMinimum ? minimum : randomIntBetween(100, 1000); plan = new Limit(EMPTY, L(value), plan); } - assertEquals(new Limit(EMPTY, L(minimum), relation), logicalOptimizer.optimize(plan)); + assertEquals(new Limit(EMPTY, L(minimum), relation), optimizedPlan(plan)); } @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/115311") @@ -5254,21 +5254,19 @@ public void testEmptyMappingIndex() { TEST_VERIFIER ); - var plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test"))); + var plan = optimizedPlan(analyze(analyzer, parser.createStatement("from empty_test"))); as(plan, LocalRelation.class); assertThat(plan.output(), equalTo(NO_FIELDS)); - plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test metadata _id | eval x = 1"))); + plan = optimizedPlan(analyze(analyzer, parser.createStatement("from empty_test metadata _id | eval x = 1"))); as(plan, LocalRelation.class); assertThat(Expressions.names(plan.output()), contains("_id", "x")); - plan = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement("from empty_test metadata _id, _version | limit 5"))); + plan = optimizedPlan(analyze(analyzer, parser.createStatement("from empty_test metadata _id, _version | limit 5"))); as(plan, LocalRelation.class); assertThat(Expressions.names(plan.output()), contains("_id", "_version")); - plan = logicalOptimizer.optimize( - analyze(analyzer, parser.createStatement("from empty_test | eval x = \"abc\" | enrich languages_idx on x")) - ); + plan = optimizedPlan(analyze(analyzer, parser.createStatement("from empty_test | eval x = \"abc\" | enrich languages_idx on x"))); LocalRelation local = as(plan, LocalRelation.class); assertThat(Expressions.names(local.output()), contains(NO_FIELDS.get(0).name(), "x", "language_code", "language_name")); } @@ -5291,7 +5289,7 @@ public void testPlanSanityCheck() throws Exception { asList(new Order(limit.source(), salary, Order.OrderDirection.ASC, Order.NullsPosition.FIRST)) ); - IllegalStateException e = expectThrows(IllegalStateException.class, () -> logicalOptimizer.optimize(invalidPlan)); + IllegalStateException e = expectThrows(IllegalStateException.class, () -> optimizedPlan(invalidPlan)); assertThat(e.getMessage(), containsString("Plan [OrderBy[[Order[salary")); assertThat(e.getMessage(), containsString(" optimized incorrectly due to missing references [salary")); } @@ -5322,11 +5320,11 @@ public void testPlanSanityCheckWithBinaryPlans() { var join = as(upperLimit.child(), Join.class); var joinWithInvalidLeftPlan = join.replaceChildren(join.right(), join.right()); - IllegalStateException e = expectThrows(IllegalStateException.class, () -> logicalOptimizer.optimize(joinWithInvalidLeftPlan)); + IllegalStateException e = expectThrows(IllegalStateException.class, () -> optimizedPlan(joinWithInvalidLeftPlan)); assertThat(e.getMessage(), containsString(" optimized incorrectly due to missing references from left hand side [languages")); var joinWithInvalidRightPlan = join.replaceChildren(join.left(), join.left()); - e = expectThrows(IllegalStateException.class, () -> logicalOptimizer.optimize(joinWithInvalidRightPlan)); + e = expectThrows(IllegalStateException.class, () -> optimizedPlan(joinWithInvalidRightPlan)); assertThat(e.getMessage(), containsString(" optimized incorrectly due to missing references from right hand side [language_code")); } @@ -6683,7 +6681,7 @@ public void testMultipleLookupShadowing() { public void testTranslateMetricsWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in))"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAggs = as(limit.child(), Aggregate.class); assertThat(finalAggs, not(instanceOf(TimeSeriesAggregate.class))); @@ -6704,7 +6702,7 @@ public void testTranslateMetricsWithoutGrouping() { public void testTranslateMixedAggsWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in)), max(network.cost)"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAggs = as(limit.child(), Aggregate.class); assertThat(finalAggs, not(instanceOf(TimeSeriesAggregate.class))); @@ -6729,7 +6727,7 @@ public void testTranslateMixedAggsWithoutGrouping() { public void testTranslateMixedAggsWithMathWithoutGrouping() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS max(rate(network.total_bytes_in)), max(network.cost + 0.2) * 1.1"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); Eval mulEval = as(project.child(), Eval.class); assertThat(mulEval.fields(), hasSize(1)); @@ -6767,7 +6765,7 @@ public void testTranslateMixedAggsWithMathWithoutGrouping() { public void testTranslateMetricsGroupedByOneDimension() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(rate(network.total_bytes_in)) BY cluster | SORT cluster | LIMIT 10"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); TopN topN = as(plan, TopN.class); Aggregate aggsByCluster = as(topN.child(), Aggregate.class); assertThat(aggsByCluster, not(instanceOf(TimeSeriesAggregate.class))); @@ -6792,7 +6790,7 @@ public void testTranslateMetricsGroupedByOneDimension() { public void testTranslateMetricsGroupedByTwoDimension() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS avg(rate(network.total_bytes_in)) BY cluster, pod"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); Eval eval = as(project.child(), Eval.class); assertThat(eval.fields(), hasSize(1)); @@ -6832,7 +6830,7 @@ public void testTranslateMetricsGroupedByTwoDimension() { public void testTranslateMetricsGroupedByTimeBucket() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(rate(network.total_bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -6866,7 +6864,7 @@ public void testTranslateMetricsGroupedByTimeBucketAndDimensions() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval eval = as(topN.child(), Eval.class); @@ -6908,7 +6906,7 @@ public void testTranslateSumOfTwoRates() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); TopN topN = as(plan, TopN.class); Aggregate finalAgg = as(topN.child(), Aggregate.class); Eval eval = as(finalAgg.child(), Eval.class); @@ -6929,7 +6927,7 @@ public void testTranslateMixedAggsGroupedByTimeBucketAndDimensions() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval eval = as(topN.child(), Eval.class); @@ -6981,7 +6979,7 @@ public void testAdjustMetricsRateBeforeFinalAgg() { | SORT cluster | LIMIT 10 """; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Project project = as(plan, Project.class); TopN topN = as(project.child(), TopN.class); Eval evalDiv = as(topN.child(), Eval.class); @@ -7034,7 +7032,7 @@ public void testAdjustMetricsRateBeforeFinalAgg() { public void testTranslateMaxOverTime() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(max_over_time(network.bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -7063,7 +7061,7 @@ public void testTranslateMaxOverTime() { public void testTranslateAvgOverTime() { assumeTrue("requires snapshot builds", Build.current().isSnapshot()); var query = "TS k8s | STATS sum(avg_over_time(network.bytes_in)) BY bucket(@timestamp, 1h)"; - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); Limit limit = as(plan, Limit.class); Aggregate finalAgg = as(limit.child(), Aggregate.class); assertThat(finalAgg, not(instanceOf(TimeSeriesAggregate.class))); @@ -7103,7 +7101,7 @@ public void testMetricsWithoutRate() { """); List plans = new ArrayList<>(); for (String query : queries) { - var plan = logicalOptimizer.optimize(analyze(metricsAnalyzer, parser.createStatement(query))); + var plan = optimizedPlan(analyze(metricsAnalyzer, parser.createStatement(query))); plans.add(plan); } for (LogicalPlan plan : plans) { 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 7fc9a0f8acd09..09a679836bd23 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 @@ -11,6 +11,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Build; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.settings.Settings; @@ -7558,7 +7559,9 @@ public void testLookupThenTopN() { ExchangeExec exchange = as(outerTopN.child(), ExchangeExec.class); FragmentExec frag = as(exchange.child(), FragmentExec.class); - LogicalPlan opt = logicalOptimizer.optimize(frag.fragment()); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(frag.fragment(), optimizedPlanFuture); + LogicalPlan opt = optimizedPlanFuture.actionGet(); TopN innerTopN = as(opt, TopN.class); assertMap( innerTopN.order().stream().map(o -> o.child().toString()).toList(), @@ -7899,7 +7902,9 @@ public void testScore() { ExchangeExec exchange = as(limitExec.child(), ExchangeExec.class); FragmentExec frag = as(exchange.child(), FragmentExec.class); - LogicalPlan opt = logicalOptimizer.optimize(frag.fragment()); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(frag.fragment(), optimizedPlanFuture); + LogicalPlan opt = optimizedPlanFuture.actionGet(); Limit limit = as(opt, Limit.class); Filter filter = as(limit.child(), Filter.class); @@ -7926,7 +7931,9 @@ public void testScoreTopN() { ExchangeExec exchange = as(topNExec.child(), ExchangeExec.class); FragmentExec frag = as(exchange.child(), FragmentExec.class); - LogicalPlan opt = logicalOptimizer.optimize(frag.fragment()); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(frag.fragment(), optimizedPlanFuture); + LogicalPlan opt = optimizedPlanFuture.actionGet(); TopN topN = as(opt, TopN.class); List order = topN.order(); Order scoreOrer = order.getFirst(); @@ -8247,7 +8254,9 @@ private PhysicalPlan physicalPlan(String query, TestDataSource dataSource) { } private PhysicalPlan physicalPlan(String query, TestDataSource dataSource, boolean assertSerialization) { - var logical = logicalOptimizer.optimize(analyze(dataSource.analyzer, parser.createStatement(query))); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(analyze(dataSource.analyzer, parser.createStatement(query)), optimizedPlanFuture); + var logical = optimizedPlanFuture.actionGet(); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); // System.out.println("Physical\n" + physical); 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 7f9ded50cc93b..bfa4488df9550 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 @@ -7,10 +7,12 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.xpack.esql.EsqlTestUtils; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.PlannerUtils; @@ -81,9 +83,15 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { } private PhysicalPlan physicalPlan(String query, Analyzer analyzer) { - var logical = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query))); + var logical = optimizeLogical(analyze(analyzer, parser.createStatement(query))); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); return physical; } + + private LogicalPlan optimizeLogical(LogicalPlan logicalPlan) { + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(logicalPlan, optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); + } } 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 9ac8bcbff6b9c..050dd9d2c922e 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.index.IndexMode; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.EsqlTestUtils; @@ -170,7 +171,9 @@ public void testGroupingAliasingMoved_To_LeftSideOfJoin_WithExpression() { } private LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { - return optimizer.optimize(analyze(analyzer, parser.createStatement(query))); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + optimizer.optimize(analyze(analyzer, parser.createStatement(query)), optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); } @Override 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 41278840f28e4..88b3bc6df1e37 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 @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.planner; import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.ByteBufferStreamInput; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -36,6 +37,7 @@ import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; @@ -379,7 +381,9 @@ public static QueryBuilder singleValueQuery(String query, QueryBuilder inner, St } private PhysicalPlan plan(String query, QueryBuilder restFilter) { - var logical = logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query))); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query)), optimizedPlanFuture); + var logical = optimizedPlanFuture.actionGet(); // System.out.println("Logical\n" + logical); var physical = mapper.map(logical); // System.out.println("physical\n" + physical); 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..42b3ae500d7f1 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.planner; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.analysis.Analyzer; @@ -239,7 +240,12 @@ 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); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + new LogicalPlanOptimizer(new LogicalOptimizerContext(configuration, FoldContext.small())).optimize( + logicalPlan, + optimizedPlanFuture + ); + logicalPlan = optimizedPlanFuture.actionGet(); PhysicalPlan physicalPlan = new Mapper().map(logicalPlan); physicalPlan = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)).optimize(physicalPlan); 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 10dcb10e3bb98..ca1e335433b5b 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 @@ -11,6 +11,7 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; @@ -202,7 +203,9 @@ static LogicalPlan parse(String query) { ), TEST_VERIFIER ); - return logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query))); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query)), optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); } @Override 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 70d5173ad848e..74c726861bdd5 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 @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.plugin; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; @@ -303,7 +304,9 @@ static LogicalPlan parse(String query) { ), TEST_VERIFIER ); - return logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query))); + PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); + logicalOptimizer.optimize(analyze(analyzer, new EsqlParser().createStatement(query)), optimizedPlanFuture); + return optimizedPlanFuture.actionGet(); } static PhysicalPlan mapAndMaybeOptimize(LogicalPlan logicalPlan) { From 5f7fb3676ec632e3f24f623ac6788afa3167853e Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 07:39:35 +0200 Subject: [PATCH 11/17] Fix tests --- .../xpack/esql/rule/AbstractRuleTestCase.java | 33 +++++++++++++++++-- .../rule/ParameterizedRuleExecutorTests.java | 26 ++++++++------- .../xpack/esql/rule/RuleExecutorTests.java | 12 +++++-- 3 files changed, 56 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java index 6de0f4020a6f8..d720164c40cfe 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -179,7 +179,11 @@ public String name() { protected static class TestParameterizedRule extends ParameterizedRule.Sync { @Override public TestNode apply(TestNode node, String param) { - return new TestNode(node.value() + "_" + param, node.children()); + // Only apply to specific values to prevent infinite loops + if (node.value().equals("test")) { + return new TestNode(node.value() + "_" + param, node.children()); + } + return node; } @Override @@ -191,7 +195,11 @@ public String name() { protected static class TestContextParameterizedRule extends ParameterizedRule.Sync { @Override public TestNode apply(TestNode node, TestContext context) { - return new TestNode(context.prefix + node.value() + context.suffix, node.children()); + // Only apply to specific values to prevent infinite loops + if (node.value().equals("middle")) { + return new TestNode(context.prefix + node.value() + context.suffix, node.children()); + } + return node; } @Override @@ -200,6 +208,27 @@ public String name() { } } + protected static class ConditionalParameterizedRule extends ParameterizedRule.Sync { + private final String trigger; + + public ConditionalParameterizedRule(String trigger) { + this.trigger = trigger; + } + + @Override + public TestNode apply(TestNode node, String param) { + if (node.value().equals(trigger)) { + return new TestNode(node.value() + "_" + param, node.children()); + } + return node; + } + + @Override + public String name() { + return "ConditionalParameterized_" + trigger; + } + } + // Test context class protected static class TestContext { public final String prefix; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java index 563c38916db4b..d2f98b5dc4593 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -77,10 +77,10 @@ public void testParameterizedRuleWithComplexContext() { public void testMultipleParameterizedRulesInBatch() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("X"); - TestNode input = new TestNode("base"); + TestNode input = new TestNode("test"); // Use "test" as trigger for TestParameterizedRule - TestParameterizedRule rule1 = new TestParameterizedRule(); - TestParameterizedRule rule2 = new TestParameterizedRule(); + ConditionalParameterizedRule rule1 = new ConditionalParameterizedRule("test"); + ConditionalParameterizedRule rule2 = new ConditionalParameterizedRule("test_X"); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MultiBatch", rule1, rule2); executor.batches.add(batch); @@ -89,15 +89,15 @@ public void testMultipleParameterizedRulesInBatch() { executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("base_X_X", result.get().after().value()); + assertEquals("test_X_X", result.get().after().value()); // Check transformations var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); var batchTransformations = transformations.values().iterator().next(); assertThat(batchTransformations.size(), equalTo(2)); - assertEquals("TestParameterizedRule", batchTransformations.get(0).name()); - assertEquals("TestParameterizedRule", batchTransformations.get(1).name()); + assertEquals("ConditionalParameterized_test", batchTransformations.get(0).name()); + assertEquals("ConditionalParameterized_test_X", batchTransformations.get(1).name()); } public void testMixedRulesInParameterizedExecutor() { @@ -105,7 +105,7 @@ public void testMixedRulesInParameterizedExecutor() { TestNode input = new TestNode("test"); // Mix parameterized and non-parameterized rules - AppendRule nonParamRule = new AppendRule("_suffix"); + ConditionalRule nonParamRule = new ConditionalRule("test", "test_suffix"); TestParameterizedRule paramRule = new TestParameterizedRule(); RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MixedBatch", nonParamRule, paramRule); @@ -230,8 +230,8 @@ public void testParameterizedRuleWithMultipleBatches() { TestParameterizedRule paramRule = new TestParameterizedRule(); RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("PrependBatch", paramRule); - // Second batch with non-parameterized rule - AppendRule appendRule = new AppendRule("_final"); + // Second batch with non-parameterized rule that triggers on the result of first batch + ConditionalRule appendRule = new ConditionalRule("test_middle", "test_middle_final"); RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("AppendBatch", appendRule); executor.batches.add(batch1); @@ -270,8 +270,12 @@ public void testParameterizedAsyncRule() { ParameterizedRule asyncRule = new ParameterizedRule.Async() { @Override public void apply(TestNode node, String param, ActionListener listener) { - // Simulate async processing - listener.onResponse(new TestNode("async_" + node.value() + "_" + param, node.children())); + // Only apply to "test" nodes to prevent infinite loops + if (node.value().equals("test")) { + listener.onResponse(new TestNode("async_" + node.value() + "_" + param, node.children())); + } else { + listener.onResponse(node); + } } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java index 2e59eefe56a68..b451eb2c965ae 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java @@ -104,7 +104,11 @@ public void testRuleExecutionOrder() { @Override public TestNode apply(TestNode node) { executionOrder.add("rule1"); - return new TestNode("B", node.children()); + // Only transform A to B, avoid infinite loops + if ("A".equals(node.value())) { + return new TestNode("B", node.children()); + } + return node; } @Override @@ -117,7 +121,11 @@ public String name() { @Override public TestNode apply(TestNode node) { executionOrder.add("rule2"); - return new TestNode("C", node.children()); + // Only transform B to C, avoid infinite loops + if ("B".equals(node.value())) { + return new TestNode("C", node.children()); + } + return node; } @Override From 8af4def037aaa2835334d5c23d70e0c77dcdfad0 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 08:52:08 +0200 Subject: [PATCH 12/17] Test improvements. --- .../xpack/esql/rule/AbstractRuleTestCase.java | 108 +++++-------- .../rule/ParameterizedRuleExecutorTests.java | 150 ++++++++++-------- .../xpack/esql/rule/RuleExecutorTests.java | 117 ++++++++------ 3 files changed, 195 insertions(+), 180 deletions(-) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java index d720164c40cfe..f8e5fd26e5b47 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -8,13 +8,11 @@ package org.elasticsearch.xpack.esql.rule; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.core.tree.NodeInfo; +import org.elasticsearch.xpack.esql.core.tree.NodeTests.Dummy; import org.elasticsearch.xpack.esql.core.tree.Source; -import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -29,70 +27,42 @@ */ public abstract class AbstractRuleTestCase extends ESTestCase { - // Test node implementation - protected static class TestNode extends Node { - private final String value; - private final List children; - + // Test node implementation extending Dummy to avoid EsqlNodeSubclassTests scanning + protected static class TestNode extends Dummy { public TestNode(String value) { - this(Source.EMPTY, value, Collections.emptyList()); + this(Source.EMPTY, Collections.emptyList(), value); } - public TestNode(String value, List children) { - this(Source.EMPTY, value, children); + public TestNode(String value, List children) { + this(Source.EMPTY, children, value); } - public TestNode(Source source, String value, List children) { - super(source, children); - this.value = value; - this.children = children; + public TestNode(Source source, List children, String value) { + super(source, children, value); } public String value() { - return value; - } - - @Override - public TestNode replaceChildren(List newChildren) { - return new TestNode(source(), value, newChildren); + return thing(); // Delegate to Dummy's thing() method } @Override protected NodeInfo info() { - return NodeInfo.create(this, TestNode::new, value, children); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - // Not needed for tests + return NodeInfo.create(this, TestNode::new, children(), value()); } @Override - public String getWriteableName() { - return "test-node"; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if ((obj instanceof TestNode) == false) return false; - TestNode other = (TestNode) obj; - return value.equals(other.value) && children.equals(other.children); - } - - @Override - public int hashCode() { - return value.hashCode() * 31 + children.hashCode(); + public TestNode replaceChildren(List newChildren) { + return new TestNode(source(), newChildren, value()); } @Override public String toString() { - return value + (children.isEmpty() ? "" : "(" + children + ")"); + return value() + (children().isEmpty() ? "" : "(" + children() + ")"); } } // Test rule implementations - protected static class AppendRule extends Rule.Sync { + protected static class AppendRule extends Rule.Sync { private final String suffix; public AppendRule(String suffix) { @@ -100,8 +70,8 @@ public AppendRule(String suffix) { } @Override - public TestNode apply(TestNode node) { - return new TestNode(node.value() + suffix, node.children()); + public Dummy apply(Dummy node) { + return new TestNode(((TestNode) node).value() + suffix, node.children()); } @Override @@ -110,7 +80,7 @@ public String name() { } } - protected static class ConditionalRule extends Rule.Sync { + protected static class ConditionalRule extends Rule.Sync { private final String trigger; private final String replacement; @@ -120,8 +90,8 @@ public ConditionalRule(String trigger, String replacement) { } @Override - public TestNode apply(TestNode node) { - if (node.value().equals(trigger)) { + public Dummy apply(Dummy node) { + if (((TestNode) node).value().equals(trigger)) { return new TestNode(replacement, node.children()); } return node; // No change if condition not met @@ -133,7 +103,7 @@ public String name() { } } - protected static class CountingAsyncRule extends Rule.Async { + protected static class CountingAsyncRule extends Rule.Async { private final AtomicInteger callCount = new AtomicInteger(0); private final String suffix; @@ -142,10 +112,14 @@ public CountingAsyncRule(String suffix) { } @Override - public void apply(TestNode node, ActionListener listener) { + public void apply(Dummy node, ActionListener listener) { callCount.incrementAndGet(); - // Simulate async processing - listener.onResponse(new TestNode(node.value() + suffix, node.children())); + // Only apply to "test" nodes to prevent infinite loops + if (((TestNode) node).value().equals("test")) { + listener.onResponse(new TestNode(((TestNode) node).value() + suffix, node.children())); + } else { + listener.onResponse(node); + } } @Override @@ -158,7 +132,7 @@ public int getCallCount() { } } - protected static class FailingRule extends Rule.Async { + protected static class FailingRule extends Rule.Async { private final String errorMessage; public FailingRule(String errorMessage) { @@ -166,7 +140,7 @@ public FailingRule(String errorMessage) { } @Override - public void apply(TestNode node, ActionListener listener) { + public void apply(Dummy node, ActionListener listener) { listener.onFailure(new RuntimeException(errorMessage)); } @@ -176,12 +150,12 @@ public String name() { } } - protected static class TestParameterizedRule extends ParameterizedRule.Sync { + protected static class TestParameterizedRule extends ParameterizedRule.Sync { @Override - public TestNode apply(TestNode node, String param) { + public Dummy apply(Dummy node, String param) { // Only apply to specific values to prevent infinite loops - if (node.value().equals("test")) { - return new TestNode(node.value() + "_" + param, node.children()); + if (((TestNode) node).value().equals("test")) { + return new TestNode(((TestNode) node).value() + "_" + param, node.children()); } return node; } @@ -192,12 +166,12 @@ public String name() { } } - protected static class TestContextParameterizedRule extends ParameterizedRule.Sync { + protected static class TestContextParameterizedRule extends ParameterizedRule.Sync { @Override - public TestNode apply(TestNode node, TestContext context) { + public Dummy apply(Dummy node, TestContext context) { // Only apply to specific values to prevent infinite loops - if (node.value().equals("middle")) { - return new TestNode(context.prefix + node.value() + context.suffix, node.children()); + if (((TestNode) node).value().equals("middle")) { + return new TestNode(context.prefix + ((TestNode) node).value() + context.suffix, node.children()); } return node; } @@ -208,7 +182,7 @@ public String name() { } } - protected static class ConditionalParameterizedRule extends ParameterizedRule.Sync { + protected static class ConditionalParameterizedRule extends ParameterizedRule.Sync { private final String trigger; public ConditionalParameterizedRule(String trigger) { @@ -216,9 +190,9 @@ public ConditionalParameterizedRule(String trigger) { } @Override - public TestNode apply(TestNode node, String param) { - if (node.value().equals(trigger)) { - return new TestNode(node.value() + "_" + param, node.children()); + public Dummy apply(Dummy node, String param) { + if (((TestNode) node).value().equals(trigger)) { + return new TestNode(((TestNode) node).value() + "_" + param, node.children()); } return node; } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java index d2f98b5dc4593..79f550d6d90ba 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.rule; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.core.tree.NodeTests.Dummy; import java.util.ArrayList; import java.util.Arrays; @@ -17,98 +18,114 @@ public class ParameterizedRuleExecutorTests extends AbstractRuleTestCase { - // Test ParameterizedRuleExecutor implementation - static class TestParameterizedRuleExecutor extends ParameterizedRuleExecutor { - public List> batches = new ArrayList<>(); + // Test parameterized executor implementation + protected static class TestParameterizedRuleExecutor extends ParameterizedRuleExecutor { + public List> batches = new ArrayList<>(); - TestParameterizedRuleExecutor(String context) { + public TestParameterizedRuleExecutor(String context) { super(context); } @Override - public List> batches() { + protected Iterable> batches() { return batches; } } - static class TestContextParameterizedRuleExecutor extends ParameterizedRuleExecutor { - public List> batches = new ArrayList<>(); + protected static class TestContextParameterizedRuleExecutor extends ParameterizedRuleExecutor { + public List> batches = new ArrayList<>(); - TestContextParameterizedRuleExecutor(TestContext context) { + public TestContextParameterizedRuleExecutor(TestContext context) { super(context); } @Override - public List> batches() { + protected Iterable> batches() { return batches; } } public void testBasicParameterizedRuleExecution() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param_value"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); TestParameterizedRule rule = new TestParameterizedRule(); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ParamBatch", rule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ParamBatch", rule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test_param_value", result.get().after().value()); + assertEquals("test_param_value", ((TestNode) result.get().after()).value()); } public void testParameterizedRuleWithComplexContext() { TestContext context = new TestContext("start_", "_end"); TestContextParameterizedRuleExecutor executor = new TestContextParameterizedRuleExecutor(context); - TestNode input = new TestNode("middle"); + Dummy input = new TestNode("middle"); TestContextParameterizedRule rule = new TestContextParameterizedRule(); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("start_middle_end", result.get().after().value()); + assertEquals("start_middle_end", ((TestNode) result.get().after()).value()); } public void testMultipleParameterizedRulesInBatch() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("X"); - TestNode input = new TestNode("test"); // Use "test" as trigger for TestParameterizedRule + Dummy input = new TestNode("test"); // Use "test" as trigger for TestParameterizedRule ConditionalParameterizedRule rule1 = new ConditionalParameterizedRule("test"); ConditionalParameterizedRule rule2 = new ConditionalParameterizedRule("test_X"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MultiBatch", rule1, rule2); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MultiBatch", rule1, rule2); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test_X_X", result.get().after().value()); + assertEquals("test_X_X", ((TestNode) result.get().after()).value()); - // Check transformations + // Check transformations - the batch runs twice: + // 1st iteration: rule1 applies (test -> test_X), then rule2 applies (test_X -> test_X_X) + // 2nd iteration: no rules apply (no changes), so execution stops var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); var batchTransformations = transformations.values().iterator().next(); - assertThat(batchTransformations.size(), equalTo(2)); + + // We should have 4 transformations total: 2 from first iteration, 2 from second (no-change) + assertThat(batchTransformations.size(), equalTo(4)); + + // First iteration transformations assertEquals("ConditionalParameterized_test", batchTransformations.get(0).name()); - assertEquals("ConditionalParameterized_test_X", batchTransformations.get(1).name()); + assertTrue("First rule should have changed", batchTransformations.get(0).hasChanged()); + assertEquals("ConditionalParameterized_test_X", batchTransformations.get(1).name()); + assertTrue("Second rule should have changed", batchTransformations.get(1).hasChanged()); + + // Second iteration transformations (no changes) + assertEquals("ConditionalParameterized_test", batchTransformations.get(2).name()); + assertFalse("First rule should not change in second iteration", batchTransformations.get(2).hasChanged()); + assertEquals("ConditionalParameterized_test_X", batchTransformations.get(3).name()); + assertFalse("Second rule should not change in second iteration", batchTransformations.get(3).hasChanged()); } public void testMixedRulesInParameterizedExecutor() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("param"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); // Mix parameterized and non-parameterized rules ConditionalRule nonParamRule = new ConditionalRule("test", "test_suffix"); - TestParameterizedRule paramRule = new TestParameterizedRule(); + + // Use ConditionalParameterizedRule that triggers on the result of the first rule + ConditionalParameterizedRule paramRule = new ConditionalParameterizedRule("test_suffix"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MixedBatch", nonParamRule, paramRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("MixedBatch", nonParamRule, paramRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); @@ -116,17 +133,17 @@ public void testMixedRulesInParameterizedExecutor() { result.assertSuccess(); // Should apply non-parameterized rule first, then parameterized rule - assertEquals("test_suffix_param", result.get().after().value()); + assertEquals("test_suffix_param", ((TestNode) result.get().after()).value()); } public void testParameterizedRuleFailure() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("error_param"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); // Create a failing parameterized rule - ParameterizedRule failingRule = new ParameterizedRule.Async() { + ParameterizedRule failingRule = new ParameterizedRule.Async() { @Override - public void apply(TestNode node, String param, ActionListener listener) { + public void apply(Dummy node, String param, ActionListener listener) { listener.onFailure(new RuntimeException("Parameterized rule failed with: " + param)); } @@ -136,7 +153,7 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", failingRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", failingRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); @@ -147,15 +164,19 @@ public String name() { public void testParameterizedRuleExecutionOrder() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ORDER"); - TestNode input = new TestNode(""); + Dummy input = new TestNode("start"); List executionOrder = new ArrayList<>(); - ParameterizedRule rule1 = new ParameterizedRule.Sync() { + ParameterizedRule rule1 = new ParameterizedRule.Sync() { @Override - public TestNode apply(TestNode node, String param) { + public Dummy apply(Dummy node, String param) { executionOrder.add("rule1_" + param); - return new TestNode(node.value() + "1", node.children()); + // Only apply to "start" to prevent infinite loops + if (((TestNode) node).value().equals("start")) { + return new TestNode("1", ((TestNode) node).children()); + } + return node; } @Override @@ -164,11 +185,15 @@ public String name() { } }; - ParameterizedRule rule2 = new ParameterizedRule.Sync() { + ParameterizedRule rule2 = new ParameterizedRule.Sync() { @Override - public TestNode apply(TestNode node, String param) { + public Dummy apply(Dummy node, String param) { executionOrder.add("rule2_" + param); - return new TestNode(node.value() + "2", node.children()); + // Only apply to "1" to prevent infinite loops + if (((TestNode) node).value().equals("1")) { + return new TestNode("12", ((TestNode) node).children()); + } + return node; } @Override @@ -177,24 +202,25 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("12", result.get().after().value()); - assertEquals(Arrays.asList("rule1_ORDER", "rule2_ORDER"), executionOrder); + assertEquals("12", ((TestNode) result.get().after()).value()); + // Execution order: rule1, rule2 (first iteration), rule1, rule2 (second iteration - no changes) + assertEquals(Arrays.asList("rule1_ORDER", "rule2_ORDER", "rule1_ORDER", "rule2_ORDER"), executionOrder); } public void testParameterizedRuleNoChange() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("ignored"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - ParameterizedRule noChangeRule = new ParameterizedRule.Sync() { + ParameterizedRule noChangeRule = new ParameterizedRule.Sync() { @Override - public TestNode apply(TestNode node, String param) { + public Dummy apply(Dummy node, String param) { return node; // No change regardless of parameter } @@ -204,14 +230,14 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test", result.get().after().value()); + assertEquals("test", ((TestNode) result.get().after()).value()); assertEquals(input, result.get().after()); // Same instance since no change // Check that transformation was recorded but marked as no change @@ -224,15 +250,15 @@ public String name() { public void testParameterizedRuleWithMultipleBatches() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("middle"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); // First batch with parameterized rule TestParameterizedRule paramRule = new TestParameterizedRule(); - RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("PrependBatch", paramRule); + RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("PrependBatch", paramRule); // Second batch with non-parameterized rule that triggers on the result of first batch ConditionalRule appendRule = new ConditionalRule("test_middle", "test_middle_final"); - RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("AppendBatch", appendRule); + RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("AppendBatch", appendRule); executor.batches.add(batch1); executor.batches.add(batch2); @@ -241,7 +267,7 @@ public void testParameterizedRuleWithMultipleBatches() { executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test_middle_final", result.get().after().value()); + assertEquals("test_middle_final", ((TestNode) result.get().after()).value()); // Should have transformations from both batches var transformations = result.get().transformations(); @@ -250,29 +276,29 @@ public void testParameterizedRuleWithMultipleBatches() { public void testParameterizedExecuteShortcut() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("shortcut"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); TestParameterizedRule rule = new TestParameterizedRule(); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", rule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", rule); executor.batches.add(batch); - AsyncResult result = new AsyncResult<>(); + AsyncResult result = new AsyncResult<>(); executor.execute(input, result.listener()); result.assertSuccess(); - assertEquals("test_shortcut", result.get().value()); + assertEquals("test_shortcut", ((TestNode) result.get()).value()); } public void testParameterizedAsyncRule() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("async_param"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - ParameterizedRule asyncRule = new ParameterizedRule.Async() { + ParameterizedRule asyncRule = new ParameterizedRule.Async() { @Override - public void apply(TestNode node, String param, ActionListener listener) { + public void apply(Dummy node, String param, ActionListener listener) { // Only apply to "test" nodes to prevent infinite loops - if (node.value().equals("test")) { - listener.onResponse(new TestNode("async_" + node.value() + "_" + param, node.children())); + if (((TestNode) node).value().equals("test")) { + listener.onResponse(new TestNode("async_" + ((TestNode) node).value() + "_" + param, ((TestNode) node).children())); } else { listener.onResponse(node); } @@ -284,31 +310,31 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("async_test_async_param", result.get().after().value()); + assertEquals("async_test_async_param", ((TestNode) result.get().after()).value()); } public void testParameterizedRuleContextAccess() { TestParameterizedRuleExecutor executor = new TestParameterizedRuleExecutor("context_value"); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); // Verify that the executor's context is correctly passed to rules assertEquals("context_value", executor.context()); TestParameterizedRule rule = new TestParameterizedRule(); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("ContextBatch", rule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test_context_value", result.get().after().value()); + assertEquals("test_context_value", ((TestNode) result.get().after()).value()); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java index b451eb2c965ae..d2ebb6f569452 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java @@ -7,6 +7,9 @@ package org.elasticsearch.xpack.esql.rule; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.tree.NodeTests.Dummy; + import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -16,34 +19,34 @@ public class RuleExecutorTests extends AbstractRuleTestCase { // Test RuleExecutor implementation - static class TestRuleExecutor extends RuleExecutor { - public List> batches = new ArrayList<>(); + static class TestRuleExecutor extends RuleExecutor { + public List> batches = new ArrayList<>(); @Override - public List> batches() { + public List> batches() { return batches; } } public void testBasicSyncRuleExecution() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "success")); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "success")); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("success", result.get().after().value()); + assertEquals("success", ((TestNode) result.get().after()).value()); } public void testMultipleRulesInBatch() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("start"); + Dummy input = new TestNode("start"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>( + RuleExecutor.Batch batch = new RuleExecutor.Batch<>( "TestBatch", new ConditionalRule("start", "middle"), new ConditionalRule("middle", "end") @@ -54,38 +57,39 @@ public void testMultipleRulesInBatch() { executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("end", result.get().after().value()); + assertEquals("end", ((TestNode) result.get().after()).value()); - // Check transformations + // Check transformations - batch runs twice: + // 1st iteration: both rules apply, 2nd iteration: no changes var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); + var batchTransformations = transformations.values().iterator().next(); - assertThat(batchTransformations.size(), equalTo(2)); - assertEquals("ConditionalstartTomiddle", batchTransformations.get(0).name()); - assertEquals("ConditionalmiddleToend", batchTransformations.get(1).name()); + assertThat(batchTransformations.size(), equalTo(4)); // 2 rules × 2 iterations } public void testAsyncRuleExecution() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); CountingAsyncRule asyncRule = new CountingAsyncRule("_async"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("AsyncBatch", asyncRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test_async", result.get().after().value()); - assertEquals(1, asyncRule.getCallCount()); + assertEquals("test_async", ((TestNode) result.get().after()).value()); + // Rule is called twice: once in first iteration (applies), once in second iteration (no change) + assertEquals(2, asyncRule.getCallCount()); } public void testRuleFailure() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", new FailingRule("Test error")); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("FailingBatch", new FailingRule("Test error")); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); @@ -96,16 +100,16 @@ public void testRuleFailure() { public void testRuleExecutionOrder() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("A"); + Dummy input = new TestNode("A"); List executionOrder = new ArrayList<>(); - Rule rule1 = new Rule.Sync() { + Rule rule1 = new Rule.Sync() { @Override - public TestNode apply(TestNode node) { + public Dummy apply(Dummy node) { executionOrder.add("rule1"); // Only transform A to B, avoid infinite loops - if ("A".equals(node.value())) { + if ("A".equals(((TestNode) node).value())) { return new TestNode("B", node.children()); } return node; @@ -117,12 +121,12 @@ public String name() { } }; - Rule rule2 = new Rule.Sync() { + Rule rule2 = new Rule.Sync() { @Override - public TestNode apply(TestNode node) { + public Dummy apply(Dummy node) { executionOrder.add("rule2"); // Only transform B to C, avoid infinite loops - if ("B".equals(node.value())) { + if ("B".equals(((TestNode) node).value())) { return new TestNode("C", node.children()); } return node; @@ -134,24 +138,25 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("OrderBatch", rule1, rule2); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("C", result.get().after().value()); - assertEquals(Arrays.asList("rule1", "rule2"), executionOrder); + assertEquals("C", ((TestNode) result.get().after()).value()); + // Execution order: rule1, rule2 (first iteration), rule1, rule2 (second iteration - no changes) + assertEquals(Arrays.asList("rule1", "rule2", "rule1", "rule2"), executionOrder); } public void testNoChangeRule() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - Rule noChangeRule = new Rule.Sync() { + Rule noChangeRule = new Rule.Sync() { @Override - public TestNode apply(TestNode node) { + public Dummy apply(Dummy node) { return node; // No change } @@ -161,14 +166,14 @@ public String name() { } }; - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("NoChangeBatch", noChangeRule); executor.batches.add(batch); AsyncResult result = new AsyncResult<>(); executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("test", result.get().after().value()); + assertEquals("test", ((TestNode) result.get().after()).value()); assertEquals(input, result.get().after()); // Same instance since no change // Check that transformation was recorded but marked as no change @@ -181,24 +186,24 @@ public String name() { public void testExecuteShortcut() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("test"); + Dummy input = new TestNode("test"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "done")); + RuleExecutor.Batch batch = new RuleExecutor.Batch<>("TestBatch", new ConditionalRule("test", "done")); executor.batches.add(batch); - AsyncResult result = new AsyncResult<>(); + AsyncResult result = new AsyncResult<>(); executor.execute(input, result.listener()); result.assertSuccess(); - assertEquals("done", result.get().value()); + assertEquals("done", ((TestNode) result.get()).value()); } public void testMultipleBatches() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("start"); + Dummy input = new TestNode("start"); - RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("Batch1", new ConditionalRule("start", "middle")); - RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("Batch2", new ConditionalRule("middle", "end")); + RuleExecutor.Batch batch1 = new RuleExecutor.Batch<>("Batch1", new ConditionalRule("start", "middle")); + RuleExecutor.Batch batch2 = new RuleExecutor.Batch<>("Batch2", new ConditionalRule("middle", "end")); executor.batches.add(batch1); executor.batches.add(batch2); @@ -207,7 +212,7 @@ public void testMultipleBatches() { executor.executeWithInfo(input, result.listener()); result.assertSuccess(); - assertEquals("end", result.get().after().value()); + assertEquals("end", ((TestNode) result.get().after()).value()); // Should have transformations from both batches var transformations = result.get().transformations(); @@ -216,9 +221,9 @@ public void testMultipleBatches() { public void testTransformationTracking() { TestRuleExecutor executor = new TestRuleExecutor(); - TestNode input = new TestNode("original"); + Dummy input = new TestNode("original"); - RuleExecutor.Batch batch = new RuleExecutor.Batch<>( + RuleExecutor.Batch batch = new RuleExecutor.Batch<>( "TrackingBatch", new ConditionalRule("original", "modified"), new ConditionalRule("modified", "final") @@ -231,24 +236,34 @@ public void testTransformationTracking() { result.assertSuccess(); TestRuleExecutor.ExecutionInfo info = result.get(); - assertEquals("original", info.before().value()); - assertEquals("final", info.after().value()); + assertEquals("original", ((TestNode) info.before()).value()); + assertEquals("final", ((TestNode) info.after()).value()); var transformations = info.transformations(); assertThat(transformations.keySet().size(), equalTo(1)); var batchTransformations = transformations.values().iterator().next(); - assertThat(batchTransformations.size(), equalTo(2)); + assertThat(batchTransformations.size(), equalTo(4)); // 2 rules × 2 iterations + // First iteration transformations TestRuleExecutor.Transformation first = batchTransformations.get(0); assertEquals("ConditionaloriginalTomodified", first.name()); - assertEquals("original", first.before().value()); - assertEquals("modified", first.after().value()); + assertEquals("original", ((TestNode) first.before()).value()); + assertEquals("modified", ((TestNode) first.after()).value()); assertTrue("Should have changed", first.hasChanged()); TestRuleExecutor.Transformation second = batchTransformations.get(1); assertEquals("ConditionalmodifiedTofinal", second.name()); - assertEquals("modified", second.before().value()); - assertEquals("final", second.after().value()); + assertEquals("modified", ((TestNode) second.before()).value()); + assertEquals("final", ((TestNode) second.after()).value()); assertTrue("Should have changed", second.hasChanged()); + + // Second iteration transformations (no changes) + TestRuleExecutor.Transformation third = batchTransformations.get(2); + assertEquals("ConditionaloriginalTomodified", third.name()); + assertFalse("Should not have changed in second iteration", third.hasChanged()); + + TestRuleExecutor.Transformation fourth = batchTransformations.get(3); + assertEquals("ConditionalmodifiedTofinal", fourth.name()); + assertFalse("Should not have changed in second iteration", fourth.hasChanged()); } } From 37f07e350c9245ad3c88662bb35b9805a4ae6511 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 09:30:59 +0200 Subject: [PATCH 13/17] Fix tests. --- .../xpack/esql/rule/AbstractRuleTestCase.java | 6 +++--- .../esql/rule/ParameterizedRuleExecutorTests.java | 12 ++++++------ .../xpack/esql/rule/RuleExecutorTests.java | 7 +++---- .../xpack/esql/tree/EsqlNodeSubclassTests.java | 5 ++++- 4 files changed, 16 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java index f8e5fd26e5b47..693e369edb96b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/AbstractRuleTestCase.java @@ -29,15 +29,15 @@ public abstract class AbstractRuleTestCase extends ESTestCase { // Test node implementation extending Dummy to avoid EsqlNodeSubclassTests scanning protected static class TestNode extends Dummy { - public TestNode(String value) { + protected TestNode(String value) { this(Source.EMPTY, Collections.emptyList(), value); } - public TestNode(String value, List children) { + protected TestNode(String value, List children) { this(Source.EMPTY, children, value); } - public TestNode(Source source, List children, String value) { + protected TestNode(Source source, List children, String value) { super(source, children, value); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java index 79f550d6d90ba..73dc827c5d401 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/ParameterizedRuleExecutorTests.java @@ -93,21 +93,21 @@ public void testMultipleParameterizedRulesInBatch() { assertEquals("test_X_X", ((TestNode) result.get().after()).value()); // Check transformations - the batch runs twice: - // 1st iteration: rule1 applies (test -> test_X), then rule2 applies (test_X -> test_X_X) + // 1st iteration: rule1 applies (test -> test_X), then rule2 applies (test_X -> test_X_X) // 2nd iteration: no rules apply (no changes), so execution stops var transformations = result.get().transformations(); assertThat(transformations.keySet().size(), equalTo(1)); var batchTransformations = transformations.values().iterator().next(); - + // We should have 4 transformations total: 2 from first iteration, 2 from second (no-change) assertThat(batchTransformations.size(), equalTo(4)); - + // First iteration transformations assertEquals("ConditionalParameterized_test", batchTransformations.get(0).name()); assertTrue("First rule should have changed", batchTransformations.get(0).hasChanged()); - assertEquals("ConditionalParameterized_test_X", batchTransformations.get(1).name()); + assertEquals("ConditionalParameterized_test_X", batchTransformations.get(1).name()); assertTrue("Second rule should have changed", batchTransformations.get(1).hasChanged()); - + // Second iteration transformations (no changes) assertEquals("ConditionalParameterized_test", batchTransformations.get(2).name()); assertFalse("First rule should not change in second iteration", batchTransformations.get(2).hasChanged()); @@ -121,7 +121,7 @@ public void testMixedRulesInParameterizedExecutor() { // Mix parameterized and non-parameterized rules ConditionalRule nonParamRule = new ConditionalRule("test", "test_suffix"); - + // Use ConditionalParameterizedRule that triggers on the result of the first rule ConditionalParameterizedRule paramRule = new ConditionalParameterizedRule("test_suffix"); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java index d2ebb6f569452..96810fb64e713 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/rule/RuleExecutorTests.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.esql.rule; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.core.tree.NodeTests.Dummy; import java.util.ArrayList; @@ -256,13 +255,13 @@ public void testTransformationTracking() { assertEquals("modified", ((TestNode) second.before()).value()); assertEquals("final", ((TestNode) second.after()).value()); assertTrue("Should have changed", second.hasChanged()); - + // Second iteration transformations (no changes) TestRuleExecutor.Transformation third = batchTransformations.get(2); assertEquals("ConditionaloriginalTomodified", third.name()); assertFalse("Should not have changed in second iteration", third.hasChanged()); - - TestRuleExecutor.Transformation fourth = batchTransformations.get(3); + + TestRuleExecutor.Transformation fourth = batchTransformations.get(3); assertEquals("ConditionalmodifiedTofinal", fourth.name()); assertFalse("Should not have changed in second iteration", fourth.hasChanged()); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java index 716ae56d85d7a..ad6f9f8ab0e1a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java @@ -829,7 +829,10 @@ private static void maybeLoadClass( throw new IOException("Couldn't load " + location, e); } - if (false == Modifier.isAbstract(c.getModifiers()) && false == c.isAnonymousClass() && clazz.isAssignableFrom(c)) { + if (false == Modifier.isAbstract(c.getModifiers()) + && false == c.isAnonymousClass() + && false == Modifier.isProtected(c.getModifiers()) + && clazz.isAssignableFrom(c)) { Class s = c.asSubclass(clazz); results.add(s); } From b01605a782430689afbe864309ffe9ca174dd568 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 11:05:24 +0200 Subject: [PATCH 14/17] Make local physical plan optimizer async --- .../xpack/esql/core/tree/Node.java | 105 +++- .../esql/core/tree/NodeTransformTests.java | 450 ++++++++++++++++++ .../optimizer/LocalLogicalPlanOptimizer.java | 5 +- .../optimizer/LocalPhysicalPlanOptimizer.java | 5 +- .../esql/optimizer/PhysicalPlanOptimizer.java | 5 +- .../xpack/esql/planner/PlannerUtils.java | 69 +-- .../xpack/esql/plugin/ComputeService.java | 76 +-- .../xpack/esql/session/EsqlSession.java | 136 ++++-- .../elasticsearch/xpack/esql/CsvTests.java | 5 +- .../LocalLogicalPlanOptimizerTests.java | 8 +- .../optimizer/PhysicalPlanOptimizerTests.java | 43 +- .../esql/optimizer/TestPlannerOptimizer.java | 45 +- .../xpack/esql/planner/FilterTests.java | 35 +- .../PlanConcurrencyCalculatorTests.java | 5 +- .../DataNodeRequestSerializationTests.java | 4 +- 15 files changed, 825 insertions(+), 171 deletions(-) create mode 100644 x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/tree/NodeTransformTests.java diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/tree/Node.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/tree/Node.java index 613f5b0ae76c2..5d3c7eed3e6ab 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/tree/Node.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/tree/Node.java @@ -6,6 +6,9 @@ */ package org.elasticsearch.xpack.esql.core.tree; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.CountDownActionListener; import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException; @@ -14,6 +17,8 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; @@ -187,16 +192,45 @@ public T transformDown(Function rule) { return node.transformChildren(child -> child.transformDown(rule)); } + @SuppressWarnings("unchecked") + public void transformDown(BiConsumer> rule, ActionListener listener) { + // First apply the rule to the current node (top-down) + rule.accept((T) this, listener.delegateFailureAndWrap((l, transformedNode) -> { + // Then recursively transform the children with the same rule + transformedNode.transformChildren((child, childListener) -> child.transformDown(rule, childListener), l); + })); + } + @SuppressWarnings("unchecked") public T transformDown(Class typeToken, Function rule) { return transformDown((t) -> (typeToken.isInstance(t) ? rule.apply((E) t) : t)); } + @SuppressWarnings("unchecked") + public void transformDown(Class typeToken, BiConsumer> rule, ActionListener listener) { + transformDown(typeToken::isInstance, rule, listener); + } + @SuppressWarnings("unchecked") public T transformDown(Predicate> nodePredicate, Function rule) { return transformDown((t) -> (nodePredicate.test(t) ? rule.apply((E) t) : t)); } + @SuppressWarnings("unchecked") + public void transformDown( + Predicate> nodePredicate, + BiConsumer> rule, + ActionListener listener + ) { + transformDown((T node, ActionListener l) -> { + if (nodePredicate.test(node)) { + rule.accept((E) node, l); + } else { + l.onResponse(node); + } + }, listener); + } + @SuppressWarnings("unchecked") public T transformUp(Function rule) { T transformed = transformChildren(child -> child.transformUp(rule)); @@ -205,8 +239,25 @@ public T transformUp(Function rule) { } @SuppressWarnings("unchecked") + public void transformUp(BiConsumer> rule, ActionListener listener) { + // First, recursively transform the children (depth-first, bottom-up) using the same async rule + transformChildren( + // traversal operation applied to each child + (child, childListener) -> child.transformUp(rule, childListener), + // After all children are transformed, apply the rule to the (possibly) new current node + listener.delegateFailureAndWrap((l, transformedChildrenNode) -> { + T node = transformedChildrenNode.equals(this) ? (T) this : transformedChildrenNode; + rule.accept(node, l); + }) + ); + } + public T transformUp(Class typeToken, Function rule) { - return transformUp((t) -> (typeToken.isInstance(t) ? rule.apply((E) t) : t)); + return transformUp(typeToken::isInstance, rule); + } + + public void transformUp(Class typeToken, BiConsumer> rule, ActionListener listener) { + transformUp(typeToken::isInstance, rule, listener); } @SuppressWarnings("unchecked") @@ -214,6 +265,22 @@ public T transformUp(Predicate> nodePredicate, Function (nodePredicate.test(t) ? rule.apply((E) t) : t)); } + @SuppressWarnings("unchecked") + public void transformUp( + Predicate> nodePredicate, + BiConsumer> rule, + ActionListener listener + ) { + transformUp((T node, ActionListener l) -> { + if (nodePredicate.test(node)) { + E typedNode = (E) node; + rule.accept((E) node, l); + } else { + l.onResponse(node); + } + }, listener); + } + @SuppressWarnings("unchecked") protected > T transformChildren(Function traversalOperation) { boolean childrenChanged = false; @@ -238,6 +305,42 @@ public T transformUp(Predicate> nodePredicate, Function> traversalOperation, ActionListener listener) { + if (children.isEmpty()) { + listener.onResponse((T) this); + return; + } + + final SetOnce> transformedChildren = new SetOnce<>(); + final AtomicBoolean childrenChanged = new AtomicBoolean(false); + + CountDownActionListener countDownListener = new CountDownActionListener( + children.size(), + listener.delegateFailureIgnoreResponseAndWrap((l) -> { + if (childrenChanged.get()) { + l.onResponse(replaceChildrenSameSize(transformedChildren.get())); + } else { + l.onResponse((T) this); + } + }) + ); + + for (int i = 0, s = children.size(); i < s; i++) { + T child = children.get(i); + final int childId = i; + traversalOperation.accept(child, countDownListener.map(next -> { + if (child.equals(next) == false) { + if (childrenChanged.compareAndSet(false, true) && transformedChildren.get() == null) { + transformedChildren.trySet(new ArrayList<>(children)); + } + transformedChildren.get().set(childId, next); + } + return null; + })); + } + } + public final T replaceChildrenSameSize(List newChildren) { if (newChildren.size() != children.size()) { throw new QlIllegalArgumentException( diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/tree/NodeTransformTests.java b/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/tree/NodeTransformTests.java new file mode 100644 index 0000000000000..533b9de862bdc --- /dev/null +++ b/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/tree/NodeTransformTests.java @@ -0,0 +1,450 @@ +/* + * 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.core.tree; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.test.ESTestCase; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Predicate; + +import static org.elasticsearch.xpack.esql.core.tree.SourceTests.randomSource; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class NodeTransformTests extends ESTestCase { + // Transform Up Tests + public void testTransformUpSimpleLeafTransformation() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Function transformer = createLeafTransformer(); + + NodeTests.Dummy result = tree.transformUp(transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + assertAsyncTransformMatches(tree, transformer, result); + } + + public void testTransformUpWithTypeToken() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Function transformer = n -> new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + + NodeTests.Dummy result = tree.transformUp(NodeTests.NoChildren.class, transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + SetOnce asyncResult = new SetOnce<>(); + tree.transformUp( + NodeTests.NoChildren.class, + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(result))); + } + + public void testTransformUpWithPredicate() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Predicate> predicate = n -> n instanceof NodeTests.NoChildren && ((NodeTests.NoChildren) n).thing().equals("leaf1"); + Function transformer = n -> new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + + NodeTests.Dummy result = tree.transformUp(predicate, transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2")); // Not transformed + + // Verify async version matches + SetOnce asyncResult = new SetOnce<>(); + tree.transformUp( + predicate, + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(result))); + } + + public void testTransformUpErrorHandling() { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + + RuntimeException e = expectThrows( + RuntimeException.class, + () -> tree.transformUp(n -> { throw new RuntimeException("test error"); }) + ); + assertThat(e.getMessage(), equalTo("test error")); + } + + public void testTransformUpAsyncErrorHandling() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + SetOnce exception = new SetOnce<>(); + + tree.transformUp( + (n, l) -> l.onFailure(new RuntimeException("test error")), + ActionListener.wrap(r -> fail("should not be called"), e -> exception.set(asInstanceOf(RuntimeException.class, e))) + ); + + assertBusy(() -> assertThat(exception.get().getMessage(), equalTo("test error"))); + } + + public void testTransformUpNestedStructures() throws Exception { + NodeTests.ChildrenAreAProperty tree = createNestedTree(); + Function transformer = createAllNodesTransformer(); + + NodeTests.Dummy result = tree.transformUp(transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformedOuter = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformedOuter.thing(), equalTo("outer_transformed")); + + NodeTests.Dummy innerResult = transformedOuter.children().get(0); + assertThat(innerResult, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformedInner = (NodeTests.ChildrenAreAProperty) innerResult; + assertThat(transformedInner.thing(), equalTo("inner_transformed")); + assertThat(transformedInner.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformedInner.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + assertAsyncTransformMatches(tree, transformer, result); + } + + // Transform Down Tests + public void testTransformDownSimpleLeafTransformation() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Function transformer = createLeafTransformer(); + + NodeTests.Dummy result = tree.transformDown(transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + assertAsyncTransformDownMatches(tree, transformer, result); + } + + public void testTransformDownWithTypeToken() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Function transformer = n -> new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + + NodeTests.Dummy result = tree.transformDown(NodeTests.NoChildren.class, transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + SetOnce asyncResult = new SetOnce<>(); + tree.transformDown( + NodeTests.NoChildren.class, + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(result))); + } + + public void testTransformDownWithPredicate() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + Predicate> predicate = n -> n instanceof NodeTests.NoChildren && ((NodeTests.NoChildren) n).thing().equals("leaf1"); + Function transformer = n -> new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + + NodeTests.Dummy result = tree.transformDown(predicate, transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformed = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformed.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformed.children().get(1).thing(), equalTo("leaf2")); // Not transformed + + // Verify async version matches + SetOnce asyncResult = new SetOnce<>(); + tree.transformDown( + predicate, + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(result))); + } + + public void testTransformDownErrorHandling() { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + + RuntimeException e = expectThrows( + RuntimeException.class, + () -> tree.transformDown(n -> { throw new RuntimeException("test error"); }) + ); + assertThat(e.getMessage(), equalTo("test error")); + } + + public void testTransformDownAsyncErrorHandling() throws Exception { + NodeTests.ChildrenAreAProperty tree = createSimpleTree(); + SetOnce exception = new SetOnce<>(); + + tree.transformDown((n, listener) -> { + if (n instanceof NodeTests.NoChildren) { + listener.onFailure(new RuntimeException("test error")); + } else { + listener.onResponse(n); + } + }, ActionListener.wrap(r -> fail("should not be called"), e -> exception.set(asInstanceOf(RuntimeException.class, e)))); + + assertBusy(() -> { + assertNotNull(exception.get()); + assertThat(exception.get().getMessage(), equalTo("test error")); + }); + } + + public void testTransformDownNestedStructures() throws Exception { + NodeTests.ChildrenAreAProperty tree = createNestedTree(); + Function transformer = createAllNodesTransformer(); + + NodeTests.Dummy result = tree.transformDown(transformer); + + assertThat(result, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformedOuter = (NodeTests.ChildrenAreAProperty) result; + assertThat(transformedOuter.thing(), equalTo("outer_transformed")); + + NodeTests.Dummy innerResult = transformedOuter.children().get(0); + assertThat(innerResult, instanceOf(NodeTests.ChildrenAreAProperty.class)); + NodeTests.ChildrenAreAProperty transformedInner = (NodeTests.ChildrenAreAProperty) innerResult; + assertThat(transformedInner.thing(), equalTo("inner_transformed")); + assertThat(transformedInner.children().get(0).thing(), equalTo("leaf1_transformed")); + assertThat(transformedInner.children().get(1).thing(), equalTo("leaf2_transformed")); + + // Verify async version matches + assertAsyncTransformDownMatches(tree, transformer, result); + } + + // Tests demonstrating behavioral differences between transformUp and transformDown + public void testTransformUpVsDownOrderDependentTransformation() { + NodeTests.NoChildren leaf1 = new NodeTests.NoChildren(randomSource(), "leaf"); + NodeTests.NoChildren leaf2 = new NodeTests.NoChildren(randomSource(), "leaf"); + NodeTests.ChildrenAreAProperty innerNode = new NodeTests.ChildrenAreAProperty(randomSource(), List.of(leaf1, leaf2), "inner"); + NodeTests.ChildrenAreAProperty outerNode = new NodeTests.ChildrenAreAProperty(randomSource(), List.of(innerNode), "outer"); + + Function transformerDown = n -> { + if (n instanceof NodeTests.ChildrenAreAProperty) { + NodeTests.ChildrenAreAProperty cn = (NodeTests.ChildrenAreAProperty) n; + return new NodeTests.ChildrenAreAProperty(cn.source(), cn.children(), cn.thing() + "_DOWN"); + } + return n; + }; + + Function transformerUp = n -> { + if (n instanceof NodeTests.ChildrenAreAProperty) { + NodeTests.ChildrenAreAProperty cn = (NodeTests.ChildrenAreAProperty) n; + return new NodeTests.ChildrenAreAProperty(cn.source(), cn.children(), cn.thing() + "_UP"); + } + return n; + }; + + // Transform down: parent first, then children + NodeTests.Dummy resultDown = outerNode.transformDown(transformerDown); + NodeTests.ChildrenAreAProperty outerDown = (NodeTests.ChildrenAreAProperty) resultDown; + NodeTests.ChildrenAreAProperty innerDown = (NodeTests.ChildrenAreAProperty) outerDown.children().get(0); + + // Transform up: children first, then parent + NodeTests.Dummy resultUp = outerNode.transformUp(transformerUp); + NodeTests.ChildrenAreAProperty outerUp = (NodeTests.ChildrenAreAProperty) resultUp; + NodeTests.ChildrenAreAProperty innerUp = (NodeTests.ChildrenAreAProperty) outerUp.children().get(0); + + // Verify transformation order is reflected in results + assertThat(outerDown.thing(), equalTo("outer_DOWN")); + assertThat(innerDown.thing(), equalTo("inner_DOWN")); + assertThat(outerUp.thing(), equalTo("outer_UP")); + assertThat(innerUp.thing(), equalTo("inner_UP")); + } + + public void testTransformUpVsDownChildDependentLogic() { + NodeTests.NoChildren leaf1 = new NodeTests.NoChildren(randomSource(), "A"); + NodeTests.NoChildren leaf2 = new NodeTests.NoChildren(randomSource(), "B"); + NodeTests.ChildrenAreAProperty node = new NodeTests.ChildrenAreAProperty(randomSource(), List.of(leaf1, leaf2), "parent"); + + // Transformer that changes parent based on children's current state + Function transformer = n -> { + if (n instanceof NodeTests.ChildrenAreAProperty) { + NodeTests.ChildrenAreAProperty cn = (NodeTests.ChildrenAreAProperty) n; + // Count how many children have "transformed" in their name + long transformedChildrenCount = cn.children().stream().filter(child -> child.thing().contains("transformed")).count(); + return new NodeTests.ChildrenAreAProperty( + cn.source(), + cn.children(), + cn.thing() + "_sees_" + transformedChildrenCount + "_transformed_children" + ); + } else if (n instanceof NodeTests.NoChildren) { + return new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + } + return n; + }; + + // Transform down: parent sees children in original state + NodeTests.Dummy resultDown = node.transformDown(transformer); + NodeTests.ChildrenAreAProperty parentDown = (NodeTests.ChildrenAreAProperty) resultDown; + + // Transform up: parent sees children after they've been transformed + NodeTests.Dummy resultUp = node.transformUp(transformer); + NodeTests.ChildrenAreAProperty parentUp = (NodeTests.ChildrenAreAProperty) resultUp; + + // Key difference: transformDown parent sees 0 transformed children, + // transformUp parent sees 2 transformed children + assertThat(parentDown.thing(), equalTo("parent_sees_0_transformed_children")); + assertThat(parentUp.thing(), equalTo("parent_sees_2_transformed_children")); + + // Both should have transformed children + assertThat(parentDown.children().get(0).thing(), equalTo("A_transformed")); + assertThat(parentDown.children().get(1).thing(), equalTo("B_transformed")); + assertThat(parentUp.children().get(0).thing(), equalTo("A_transformed")); + assertThat(parentUp.children().get(1).thing(), equalTo("B_transformed")); + } + + public void testTransformUpVsDownConditionalTransformation() { + NodeTests.NoChildren leaf1 = new NodeTests.NoChildren(randomSource(), "child1"); + NodeTests.NoChildren leaf2 = new NodeTests.NoChildren(randomSource(), "child2"); + NodeTests.ChildrenAreAProperty node = new NodeTests.ChildrenAreAProperty(randomSource(), List.of(leaf1, leaf2), "STOP"); + + // Transformer that stops transformation if parent has "STOP" in name + Function transformer = n -> { + if (n instanceof NodeTests.ChildrenAreAProperty) { + NodeTests.ChildrenAreAProperty cn = (NodeTests.ChildrenAreAProperty) n; + if (cn.thing().contains("STOP")) { + // Return node unchanged + return cn; + } else { + return new NodeTests.ChildrenAreAProperty(cn.source(), cn.children(), cn.thing() + "_processed"); + } + } else if (n instanceof NodeTests.NoChildren) { + return new NodeTests.NoChildren(n.source(), n.thing() + "_transformed"); + } + return n; + }; + + NodeTests.Dummy resultDown = node.transformDown(transformer); + NodeTests.ChildrenAreAProperty parentDown = (NodeTests.ChildrenAreAProperty) resultDown; + + NodeTests.Dummy resultUp = node.transformUp(transformer); + NodeTests.ChildrenAreAProperty parentUp = (NodeTests.ChildrenAreAProperty) resultUp; + + // Both parents should remain unchanged (contain "STOP") + assertThat(parentDown.thing(), equalTo("STOP")); + assertThat(parentUp.thing(), equalTo("STOP")); + + // Both should have transformed children + assertThat(parentDown.children().get(0).thing(), equalTo("child1_transformed")); + assertThat(parentUp.children().get(0).thing(), equalTo("child1_transformed")); + } + + public void testTransformUpVsDownAccumulativeChanges() { + NodeTests.NoChildren leaf = new NodeTests.NoChildren(randomSource(), "0"); + NodeTests.AChildIsAProperty innerNode = new NodeTests.AChildIsAProperty(randomSource(), leaf, "0"); + NodeTests.AChildIsAProperty outerNode = new NodeTests.AChildIsAProperty(randomSource(), innerNode, "0"); + + // Transformer that increments numeric values + Function transformer = n -> { + try { + int currentValue = Integer.parseInt(n.thing()); + String newValue = String.valueOf(currentValue + 1); + + if (n instanceof NodeTests.NoChildren) { + return new NodeTests.NoChildren(n.source(), newValue); + } else if (n instanceof NodeTests.AChildIsAProperty) { + NodeTests.AChildIsAProperty an = (NodeTests.AChildIsAProperty) n; + return new NodeTests.AChildIsAProperty(an.source(), an.child(), newValue); + } + } catch (NumberFormatException e) { + // If not a number, leave unchanged + } + return n; + }; + + NodeTests.Dummy resultDown = outerNode.transformDown(transformer); + NodeTests.Dummy resultUp = outerNode.transformUp(transformer); + + // Extract the final values + NodeTests.AChildIsAProperty outerDown = (NodeTests.AChildIsAProperty) resultDown; + NodeTests.AChildIsAProperty innerDown = (NodeTests.AChildIsAProperty) outerDown.child(); + NodeTests.NoChildren leafDown = (NodeTests.NoChildren) innerDown.child(); + + NodeTests.AChildIsAProperty outerUp = (NodeTests.AChildIsAProperty) resultUp; + NodeTests.AChildIsAProperty innerUp = (NodeTests.AChildIsAProperty) outerUp.child(); + NodeTests.NoChildren leafUp = (NodeTests.NoChildren) innerUp.child(); + + // All nodes should be incremented to "1" + assertThat(leafDown.thing(), equalTo("1")); + assertThat(leafUp.thing(), equalTo("1")); + assertThat(innerDown.thing(), equalTo("1")); + assertThat(innerUp.thing(), equalTo("1")); + assertThat(outerDown.thing(), equalTo("1")); + assertThat(outerUp.thing(), equalTo("1")); + } + + // Helper methods for transform tests + private NodeTests.ChildrenAreAProperty createSimpleTree() { + NodeTests.NoChildren leaf1 = new NodeTests.NoChildren(randomSource(), "leaf1"); + NodeTests.NoChildren leaf2 = new NodeTests.NoChildren(randomSource(), "leaf2"); + return new NodeTests.ChildrenAreAProperty(randomSource(), List.of(leaf1, leaf2), "node"); + } + + private NodeTests.ChildrenAreAProperty createNestedTree() { + NodeTests.NoChildren leaf1 = new NodeTests.NoChildren(randomSource(), "leaf1"); + NodeTests.NoChildren leaf2 = new NodeTests.NoChildren(randomSource(), "leaf2"); + NodeTests.ChildrenAreAProperty innerNode = new NodeTests.ChildrenAreAProperty(randomSource(), List.of(leaf1, leaf2), "inner"); + return new NodeTests.ChildrenAreAProperty(randomSource(), List.of(innerNode), "outer"); + } + + private Function createLeafTransformer() { + return n -> n instanceof NodeTests.NoChildren ? new NodeTests.NoChildren(n.source(), n.thing() + "_transformed") : n; + } + + private Function createAllNodesTransformer() { + return n -> { + if (n instanceof NodeTests.NoChildren) { + return new NodeTests.NoChildren(n.source(), ((NodeTests.NoChildren) n).thing() + "_transformed"); + } else if (n instanceof NodeTests.ChildrenAreAProperty) { + NodeTests.ChildrenAreAProperty cn = (NodeTests.ChildrenAreAProperty) n; + return new NodeTests.ChildrenAreAProperty(cn.source(), cn.children(), cn.thing() + "_transformed"); + } + return n; + }; + } + + private void assertAsyncTransformMatches( + NodeTests.Dummy node, + Function transformer, + NodeTests.Dummy expectedResult + ) throws Exception { + SetOnce asyncResult = new SetOnce<>(); + ((Node) node).transformUp( + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(expectedResult))); + } + + private void assertAsyncTransformDownMatches( + NodeTests.Dummy node, + Function transformer, + NodeTests.Dummy expectedResult + ) throws Exception { + SetOnce asyncResult = new SetOnce<>(); + ((Node) node).transformDown( + (n, listener) -> listener.onResponse(transformer.apply(n)), + ActionListener.wrap(asyncResult::set, ESTestCase::fail) + ); + assertBusy(() -> assertThat(asyncResult.get(), equalTo(expectedResult))); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java index dab2b35025aa6..0699851fd04fd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEmptyRelation; import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceStatsFilteredAggWithEval; import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceStringCasingWithInsensitiveRegexMatch; @@ -80,7 +81,7 @@ private static Batch localOperators() { return operators.with(newRules.toArray(Rule[]::new)); } - public LogicalPlan localOptimize(LogicalPlan plan) { - return execute(plan); + public void localOptimize(LogicalPlan plan, ActionListener listener) { + execute(plan, listener); } } 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 b3fb1aa7098e9..1c43577a2d08d 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.common.Failures; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.EnableSpatialDistancePushdown; @@ -41,8 +42,8 @@ public LocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context) { super(context); } - public PhysicalPlan localOptimize(PhysicalPlan plan) { - return verify(execute(plan)); + public void localOptimize(PhysicalPlan plan, ActionListener listener) { + execute(plan, listener.map(this::verify)); } PhysicalPlan verify(PhysicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java index 57e2c8cba4c32..c5072487bf592 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.common.Failures; import org.elasticsearch.xpack.esql.optimizer.rules.physical.ProjectAwayColumns; @@ -33,8 +34,8 @@ public PhysicalPlanOptimizer(PhysicalOptimizerContext context) { super(context); } - public PhysicalPlan optimize(PhysicalPlan plan) { - return verify(execute(plan)); + public void optimize(PhysicalPlan plan, ActionListener listener) { + execute(plan, listener.safeMap(this::verify)); } PhysicalPlan verify(PhysicalPlan plan) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java index 614d5901c519f..7e307494a6f55 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.planner; import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.util.BigArrays; @@ -162,52 +163,64 @@ private static void forEachRelation(PhysicalPlan plan, Consumer acti })); } - public static PhysicalPlan localPlan( + public static void localPlan( List searchContexts, Configuration configuration, FoldContext foldCtx, - PhysicalPlan plan + PhysicalPlan plan, + ActionListener listener ) { - return localPlan(configuration, foldCtx, plan, SearchContextStats.from(searchContexts)); + localPlan(configuration, foldCtx, plan, SearchContextStats.from(searchContexts), listener); } - public static PhysicalPlan localPlan(Configuration configuration, FoldContext foldCtx, PhysicalPlan plan, SearchStats searchStats) { + public static void localPlan( + Configuration configuration, + FoldContext foldCtx, + PhysicalPlan plan, + SearchStats searchStats, + ActionListener listener + ) { final var logicalOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(configuration, foldCtx, searchStats)); var physicalOptimizer = new LocalPhysicalPlanOptimizer(new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats)); - return localPlan(plan, logicalOptimizer, physicalOptimizer); + localPlan(plan, logicalOptimizer, physicalOptimizer, listener); } - public static PhysicalPlan localPlan( + public static void localPlan( PhysicalPlan plan, LocalLogicalPlanOptimizer logicalOptimizer, - LocalPhysicalPlanOptimizer physicalOptimizer + LocalPhysicalPlanOptimizer physicalOptimizer, + ActionListener listener ) { final LocalMapper localMapper = new LocalMapper(); - var isCoordPlan = new Holder<>(Boolean.TRUE); - var localPhysicalPlan = plan.transformUp(FragmentExec.class, f -> { - isCoordPlan.set(Boolean.FALSE); - var optimizedFragment = logicalOptimizer.localOptimize(f.fragment()); - var physicalFragment = localMapper.map(optimizedFragment); - var filter = f.esFilter(); - if (filter != null) { - physicalFragment = physicalFragment.transformUp( - EsSourceExec.class, - query -> new EsSourceExec( - Source.EMPTY, - query.indexPattern(), - query.indexMode(), - query.indexNameWithModes(), - query.output(), - filter + plan.transformUp(FragmentExec.class, (f, l) -> { + logicalOptimizer.localOptimize(f.fragment(), ActionListener.wrap(optimizedFragment -> { + var physicalFragment = localMapper.map(optimizedFragment); + var filter = f.esFilter(); + if (filter != null) { + physicalFragment = physicalFragment.transformUp( + EsSourceExec.class, + query -> new EsSourceExec( + Source.EMPTY, + query.indexPattern(), + query.indexMode(), + query.indexNameWithModes(), + query.output(), + filter + ) + ); + } + + physicalOptimizer.localOptimize( + physicalFragment, + ActionListener.wrap( + localOptimized -> l.onResponse(EstimatesRowSize.estimateRowSize(f.estimatedRowSize(), localOptimized)), + l::onFailure ) ); - } - var localOptimized = physicalOptimizer.localOptimize(physicalFragment); - return EstimatesRowSize.estimateRowSize(f.estimatedRowSize(), localOptimized); - }); - return isCoordPlan.get() ? plan : localPhysicalPlan; + }, l::onFailure)); + }, listener); } /** 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 4adc97d28fee0..56945a0f19782 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 @@ -576,39 +576,51 @@ public SourceProvider createSourceProvider() { LOGGER.debug("Received physical plan:\n{}", plan); - var localPlan = PlannerUtils.localPlan(context.searchExecutionContexts(), context.configuration(), context.foldCtx(), plan); - // 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 - LocalExecutionPlanner.LocalExecutionPlan localExecutionPlan = planner.plan(context.description(), context.foldCtx(), localPlan); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Local execution plan:\n{}", localExecutionPlan.describe()); - } - var drivers = localExecutionPlan.createDrivers(context.sessionId()); - // After creating the drivers (and therefore, the operators), we can safely decrement the reference count since the operators - // will hold a reference to the contexts where relevant. - contexts.forEach(RefCounted::decRef); - if (drivers.isEmpty()) { - throw new IllegalStateException("no drivers created"); - } - LOGGER.debug("using {} drivers", drivers.size()); - driverRunner.executeDrivers( - task, - drivers, - transportService.getThreadPool().executor(ESQL_WORKER_THREAD_POOL_NAME), - ActionListener.releaseAfter(listener.map(ignored -> { - if (context.configuration().profile()) { - return DriverCompletionInfo.includingProfiles( - drivers, - context.description(), - clusterService.getClusterName().value(), - transportService.getLocalNode().getName(), - localPlan.toString() - ); - } else { - return DriverCompletionInfo.excludingProfiles(drivers); + PlannerUtils.localPlan( + context.searchExecutionContexts(), + context.configuration(), + context.foldCtx(), + plan, + listener.delegateFailureAndWrap((l, localPlan) -> { + // 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 + LocalExecutionPlanner.LocalExecutionPlan localExecutionPlan = planner.plan( + context.description(), + context.foldCtx(), + localPlan + ); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Local execution plan:\n{}", localExecutionPlan.describe()); } - }), () -> Releasables.close(drivers)) + var drivers = localExecutionPlan.createDrivers(context.sessionId()); + // After creating the drivers (and therefore, the operators), we can safely decrement the reference count since the + // operators + // will hold a reference to the contexts where relevant. + contexts.forEach(RefCounted::decRef); + if (drivers.isEmpty()) { + throw new IllegalStateException("no drivers created"); + } + LOGGER.debug("using {} drivers", drivers.size()); + driverRunner.executeDrivers( + task, + drivers, + transportService.getThreadPool().executor(ESQL_WORKER_THREAD_POOL_NAME), + ActionListener.releaseAfter(l.map(ignored -> { + if (context.configuration().profile()) { + return DriverCompletionInfo.includingProfiles( + drivers, + context.description(), + clusterService.getClusterName().value(), + transportService.getLocalNode().getName(), + localPlan.toString() + ); + } else { + return DriverCompletionInfo.excludingProfiles(drivers); + } + }), () -> Releasables.close(drivers)) + ); + }) ); } catch (Exception e) { 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 6846d4cf2c75b..6a923e83b10bd 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 @@ -102,12 +102,15 @@ import org.elasticsearch.xpack.esql.telemetry.PlanTelemetry; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.stream.Collectors; @@ -217,25 +220,27 @@ public void executeOptimizedPlan( LogicalPlan optimizedPlan, ActionListener listener ) { - PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(optimizedPlan, request); - if (explainMode) { - String physicalPlanString = physicalPlan.toString(); - List fields = List.of( - new ReferenceAttribute(EMPTY, "role", DataType.KEYWORD), - new ReferenceAttribute(EMPTY, "type", DataType.KEYWORD), - new ReferenceAttribute(EMPTY, "plan", DataType.KEYWORD) - ); - List> values = new ArrayList<>(); - values.add(List.of("coordinator", "parsedPlan", parsedPlanString)); - values.add(List.of("coordinator", "optimizedLogicalPlan", optimizedLogicalPlanString)); - values.add(List.of("coordinator", "optimizedPhysicalPlan", physicalPlanString)); - var blocks = BlockUtils.fromList(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, values); - physicalPlan = new LocalSourceExec(Source.EMPTY, fields, LocalSupplier.of(blocks)); - } - // TODO: this could be snuck into the underlying listener - EsqlCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); - // execute any potential subplans - executeSubPlans(physicalPlan, planRunner, executionInfo, request, listener); + logicalPlanToPhysicalPlan(optimizedPlan, request, ActionListener.wrap(physicalPlan -> { + PhysicalPlan finalPhysicalPlan = physicalPlan; + if (explainMode) { + String physicalPlanString = physicalPlan.toString(); + List fields = List.of( + new ReferenceAttribute(EMPTY, "role", DataType.KEYWORD), + new ReferenceAttribute(EMPTY, "type", DataType.KEYWORD), + new ReferenceAttribute(EMPTY, "plan", DataType.KEYWORD) + ); + List> values = new ArrayList<>(); + values.add(List.of("coordinator", "parsedPlan", parsedPlanString)); + values.add(List.of("coordinator", "optimizedLogicalPlan", optimizedLogicalPlanString)); + values.add(List.of("coordinator", "optimizedPhysicalPlan", physicalPlanString)); + var blocks = BlockUtils.fromList(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, values); + finalPhysicalPlan = new LocalSourceExec(Source.EMPTY, fields, LocalSupplier.of(blocks)); + } + // TODO: this could be snuck into the underlying listener + EsqlCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); + // execute any potential subplans + executeSubPlans(finalPhysicalPlan, planRunner, executionInfo, request, listener); + }, listener::onFailure)); } private record PlanTuple(PhysicalPlan physical, LogicalPlan logical) {} @@ -247,7 +252,8 @@ private void executeSubPlans( EsqlQueryRequest request, ActionListener listener ) { - List subplans = new ArrayList<>(); + // Collect all logical subplans first + List logicalSubplans = new ArrayList<>(); // Currently the inlinestats are limited and supported as streaming operators, thus present inside the fragment as logical plans // Below they get collected, translated into a separate, coordinator based plan and the results 'broadcasted' as a local relation @@ -257,20 +263,50 @@ private void executeSubPlans( LogicalPlan subplan = InlineJoin.replaceStub(ij.left(), ij.right()); // mark the new root node as optimized subplan.setOptimized(); - PhysicalPlan subqueryPlan = logicalPlanToPhysicalPlan(subplan, request); - subplans.add(new PlanTuple(subqueryPlan, ij.right())); + logicalSubplans.add(subplan); }); }); - Iterator iterator = subplans.iterator(); - - // TODO: merge into one method - if (subplans.size() > 0) { - // code-path to execute subplans - executeSubPlan(new DriverCompletionInfo.Accumulator(), physicalPlan, iterator, executionInfo, runner, listener); - } else { + if (logicalSubplans.isEmpty()) { // execute main plan runner.run(physicalPlan, listener); + return; + } + + // Convert all logical subplans to physical plans asynchronously + List subplans = new ArrayList<>(Collections.nCopies(logicalSubplans.size(), null)); + AtomicInteger completed = new AtomicInteger(0); + AtomicReference error = new AtomicReference<>(); + + for (int i = 0; i < logicalSubplans.size(); i++) { + final int index = i; + LogicalPlan logicalSubplan = logicalSubplans.get(i); + + logicalPlanToPhysicalPlan(logicalSubplan, request, ActionListener.wrap(subqueryPlan -> { + synchronized (subplans) { + if (error.get() == null) { + // Find the corresponding InlineJoin to get the right side + physicalPlan.forEachUp(FragmentExec.class, f -> { + f.fragment().forEachUp(InlineJoin.class, ij -> { + LogicalPlan subplan = InlineJoin.replaceStub(ij.left(), ij.right()); + if (subplan.equals(logicalSubplan)) { + subplans.set(index, new PlanTuple(subqueryPlan, ij.right())); + } + }); + }); + + if (completed.incrementAndGet() == logicalSubplans.size()) { + // All subplans are ready, execute them + Iterator iterator = subplans.iterator(); + executeSubPlan(new DriverCompletionInfo.Accumulator(), physicalPlan, iterator, executionInfo, runner, listener); + } + } + } + }, e -> { + if (error.compareAndSet(null, e instanceof Exception ex ? ex : new RuntimeException(e))) { + listener.onFailure(error.get()); + } + })); } } @@ -809,22 +845,23 @@ private static Set subfields(Set names) { return names.stream().filter(name -> name.endsWith(WILDCARD) == false).map(name -> name + ".*").collect(Collectors.toSet()); } - private PhysicalPlan logicalPlanToPhysicalPlan(LogicalPlan optimizedPlan, EsqlQueryRequest request) { - PhysicalPlan physicalPlan = optimizedPhysicalPlan(optimizedPlan); - physicalPlan = physicalPlan.transformUp(FragmentExec.class, f -> { - QueryBuilder filter = request.filter(); - if (filter != null) { - var fragmentFilter = f.esFilter(); - // TODO: have an ESFilter and push down to EsQueryExec / EsSource - // This is an ugly hack to push the filter parameter to Lucene - // TODO: filter integration testing - filter = fragmentFilter != null ? boolQuery().filter(fragmentFilter).must(filter) : filter; - LOGGER.debug("Fold filter {} to EsQueryExec", filter); - f = f.withFilter(filter); - } - return f; - }); - return EstimatesRowSize.estimateRowSize(0, physicalPlan); + private void logicalPlanToPhysicalPlan(LogicalPlan optimizedPlan, EsqlQueryRequest request, ActionListener listener) { + optimizedPhysicalPlan(optimizedPlan, listener.map(physicalPlan -> { + physicalPlan = physicalPlan.transformUp(FragmentExec.class, f -> { + QueryBuilder filter = request.filter(); + if (filter != null) { + var fragmentFilter = f.esFilter(); + // TODO: have an ESFilter and push down to EsQueryExec / EsSource + // This is an ugly hack to push the filter parameter to Lucene + // TODO: filter integration testing + filter = fragmentFilter != null ? boolQuery().filter(fragmentFilter).must(filter) : filter; + LOGGER.debug("Fold filter {} to EsQueryExec", filter); + f = f.withFilter(filter); + } + return f; + }); + return EstimatesRowSize.estimateRowSize(0, physicalPlan); + })); } public void optimizedPlan(LogicalPlan logicalPlan, ActionListener listener) { @@ -848,10 +885,11 @@ public PhysicalPlan physicalPlan(LogicalPlan optimizedPlan) { return plan; } - public PhysicalPlan optimizedPhysicalPlan(LogicalPlan optimizedPlan) { - var plan = physicalPlanOptimizer.optimize(physicalPlan(optimizedPlan)); - LOGGER.debug("Optimized physical plan:\n{}", plan); - return plan; + public void optimizedPhysicalPlan(LogicalPlan optimizedPlan, ActionListener listener) { + physicalPlanOptimizer.optimize(physicalPlan(optimizedPlan), listener.map(plan -> { + LOGGER.debug("Optimized physical plan:\n{}", plan); + return plan; + })); } record PreAnalysisResult( 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 50dfbddae3a4d..e914a2e05cefb 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 @@ -723,7 +723,10 @@ void executeSubPlan( new LocalPhysicalOptimizerContext(configuration, foldCtx, searchStats) ); - var csvDataNodePhysicalPlan = PlannerUtils.localPlan(dataNodePlan, logicalTestOptimizer, physicalTestOptimizer); + PlainActionFuture localPlanFuture = new PlainActionFuture<>(); + PlannerUtils.localPlan(dataNodePlan, logicalTestOptimizer, physicalTestOptimizer, localPlanFuture); + var csvDataNodePhysicalPlan = localPlanFuture.actionGet(); + exchangeSource.addRemoteSink( exchangeSink::fetchPageAsync, Randomness.get().nextBoolean(), 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 77c92599398c3..f7d9212a6776a 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 @@ -513,7 +513,9 @@ public void testSparseDocument() throws Exception { var analyzed = analyze(analyzer, parser.createStatement(query)); var optimized = optimizedPlan(analyzed); var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, FoldContext.small(), searchStats); - var plan = new LocalLogicalPlanOptimizer(localContext).localOptimize(optimized); + PlainActionFuture planFuture = new PlainActionFuture<>(); + new LocalLogicalPlanOptimizer(localContext).localOptimize(optimized, planFuture); + var plan = planFuture.actionGet(); var project = as(plan, Project.class); assertThat(project.projections(), hasSize(10)); @@ -808,7 +810,9 @@ private LogicalPlan optimizedPlan(LogicalPlan logicalPlan) { 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); + PlainActionFuture localPlanFuture = new PlainActionFuture<>(); + new LocalLogicalPlanOptimizer(localContext).localOptimize(plan, localPlanFuture); + var localPlan = localPlanFuture.actionGet(); // System.out.println(localPlan); return localPlan; } 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 09a679836bd23..a787e34d7048a 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 @@ -11,6 +11,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Build; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.lucene.BytesRefs; @@ -7845,7 +7846,9 @@ 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(config, FoldContext.small(), plan, TEST_SEARCH_STATS); + PlainActionFuture localPlanFuture = new PlainActionFuture<>(); + PlannerUtils.localPlan(config, FoldContext.small(), plan, TEST_SEARCH_STATS, localPlanFuture); + plan = localPlanFuture.actionGet(); ExchangeSinkHandler exchangeSinkHandler = new ExchangeSinkHandler(null, 10, () -> 10); LocalExecutionPlanner planner = new LocalExecutionPlanner( "test", @@ -8209,21 +8212,29 @@ private PhysicalPlan optimizedPlan(PhysicalPlan plan) { private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { // System.out.println("* Physical Before\n" + plan); - var p = EstimatesRowSize.estimateRowSize(0, physicalPlanOptimizer.optimize(plan)); - // System.out.println("* Physical After\n" + p); - // the real execution breaks the plan at the exchange and then decouples the plan - // this is of no use in the unit tests, which checks the plan as a whole instead of each - // 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); - return EstimatesRowSize.estimateRowSize(fragment.estimatedRowSize(), localPlan); - }); - - // handle local reduction alignment - l = localRelationshipAlignment(l); - // System.out.println("* Localized DataNode Plan\n" + l); - return l; + PlainActionFuture result = new PlainActionFuture<>(); + + physicalPlanOptimizer.optimize(plan, ActionListener.wrap(optimizedPlan -> { + var p = EstimatesRowSize.estimateRowSize(0, optimizedPlan); + // System.out.println("* Physical After\n" + p); + // the real execution breaks the plan at the exchange and then decouples the plan + // this is of no use in the unit tests, which checks the plan as a whole instead of each + // individually hence why here the plan is kept as is + + var l = p.transformUp(FragmentExec.class, fragment -> { + PlainActionFuture localPlanFuture = new PlainActionFuture<>(); + PlannerUtils.localPlan(config, FoldContext.small(), fragment, searchStats, localPlanFuture); + var localPlan = localPlanFuture.actionGet(); + return EstimatesRowSize.estimateRowSize(fragment.estimatedRowSize(), localPlan); + }); + + // handle local reduction alignment + l = localRelationshipAlignment(l); + // System.out.println("* Localized DataNode Plan\n" + l); + result.onResponse(l); + }, result::onFailure)); + + return result.actionGet(); } static SearchStats statsWithIndexedFields(String... names) { 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 bfa4488df9550..653d0b6c38c3f 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 @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.optimizer; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.xpack.esql.EsqlTestUtils; import org.elasticsearch.xpack.esql.analysis.Analyzer; @@ -59,27 +60,33 @@ public PhysicalPlan plan(String query, SearchStats stats, Analyzer analyzer) { } private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { - // System.out.println("* Physical Before\n" + plan); - var physicalPlan = EstimatesRowSize.estimateRowSize(0, physicalPlanOptimizer.optimize(plan)); - // System.out.println("* Physical After\n" + physicalPlan); - // the real execution breaks the plan at the exchange and then decouples the plan - // this is of no use in the unit tests, which checks the plan as a whole instead of each - // individually hence why here the plan is kept as is - - var logicalTestOptimizer = new LocalLogicalPlanOptimizer( - new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats) - ); - var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( - new LocalPhysicalOptimizerContext(config, FoldContext.small(), searchStats), - true - ); - var l = PlannerUtils.localPlan(physicalPlan, logicalTestOptimizer, physicalTestOptimizer); - - // handle local reduction alignment - l = PhysicalPlanOptimizerTests.localRelationshipAlignment(l); + PlainActionFuture result = new PlainActionFuture<>(); + + physicalPlanOptimizer.optimize(plan, ActionListener.wrap(physicalPlan -> { + physicalPlan = EstimatesRowSize.estimateRowSize(0, physicalPlan); + // System.out.println("* Physical After\n" + p); + // the real execution breaks the plan at the exchange and then decouples the plan + // this is of no use in the unit tests, which checks the plan as a whole instead of each + // individually hence why here the plan is kept as is + + var logicalTestOptimizer = new LocalLogicalPlanOptimizer( + new LocalLogicalOptimizerContext(config, FoldContext.small(), searchStats) + ); + + var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer( + new LocalPhysicalOptimizerContext(config, FoldContext.small(), searchStats), + true + ); + + PlannerUtils.localPlan(physicalPlan, logicalTestOptimizer, physicalTestOptimizer, ActionListener.wrap(l -> { + // handle local reduction alignment + var alignedL = PhysicalPlanOptimizerTests.localRelationshipAlignment(l); + result.onResponse(alignedL); + }, result::onFailure)); + }, result::onFailure)); // System.out.println("* Localized DataNode Plan\n" + l); - return l; + return result.actionGet(); } private PhysicalPlan physicalPlan(String query, Analyzer analyzer) { 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 88b3bc6df1e37..48fa3c73f0b6f 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 @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.planner; import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.ByteBufferStreamInput; @@ -37,7 +38,6 @@ import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; @@ -381,20 +381,25 @@ public static QueryBuilder singleValueQuery(String query, QueryBuilder inner, St } private PhysicalPlan plan(String query, QueryBuilder restFilter) { - PlainActionFuture optimizedPlanFuture = new PlainActionFuture<>(); - logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query)), optimizedPlanFuture); - var logical = optimizedPlanFuture.actionGet(); - // System.out.println("Logical\n" + logical); - var physical = mapper.map(logical); - // System.out.println("physical\n" + physical); - physical = physical.transformUp( - FragmentExec.class, - f -> new FragmentExec(f.source(), f.fragment(), restFilter, f.estimatedRowSize()) - ); - physical = physicalPlanOptimizer.optimize(physical); - // System.out.println("optimized\n" + physical); - assertSerialization(physical); - return physical; + PlainActionFuture physicalPlanFuture = new PlainActionFuture<>(); + + logicalOptimizer.optimize(analyze(analyzer, parser.createStatement(query)), ActionListener.wrap(logical -> { + var physical = mapper.map(logical); + // System.out.println("physical\n" + physical); + physical = physical.transformUp( + FragmentExec.class, + f -> new FragmentExec(f.source(), f.fragment(), restFilter, f.estimatedRowSize()) + ); + + physicalPlanOptimizer.optimize(physical, ActionListener.wrap(optimized -> { + // System.out.println("optimized\n" + physical); + assertSerialization(optimized); + physicalPlanFuture.onResponse(optimized); + }, physicalPlanFuture::onFailure)); + + }, physicalPlanFuture::onFailure)); + + return physicalPlanFuture.actionGet(); } private QueryBuilder restFilterQuery(String field) { 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 42b3ae500d7f1..59502fd90b6c4 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 @@ -248,7 +248,10 @@ private void assertConcurrency(String query, Integer concurrencyPragmaValue, Int logicalPlan = optimizedPlanFuture.actionGet(); PhysicalPlan physicalPlan = new Mapper().map(logicalPlan); - physicalPlan = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)).optimize(physicalPlan); + + PlainActionFuture physicalPlanFuture = new PlainActionFuture<>(); + new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)).optimize(physicalPlan, physicalPlanFuture); + physicalPlan = physicalPlanFuture.actionGet(); PhysicalPlan dataNodePlan = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode(physicalPlan, configuration).v2(); 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 74c726861bdd5..4c10fc3ffada9 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 @@ -314,7 +314,9 @@ static PhysicalPlan mapAndMaybeOptimize(LogicalPlan logicalPlan) { var mapper = new Mapper(); var physical = mapper.map(logicalPlan); if (randomBoolean()) { - physical = physicalPlanOptimizer.optimize(physical); + PlainActionFuture optimizedPhysicalFuture = new PlainActionFuture<>(); + physicalPlanOptimizer.optimize(physical, optimizedPhysicalFuture); + physical = optimizedPhysicalFuture.actionGet(); } return physical; } From 6c5e4ec8fa853b859c36ded094ced24e902719a7 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 11:17:53 +0200 Subject: [PATCH 15/17] Who is using sync rule executor in 2025? --- .../org/elasticsearch/xpack/esql/rule/RuleExecutor.java | 9 --------- 1 file changed, 9 deletions(-) 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 a14cc63190af2..96dd206dd0014 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 @@ -9,7 +9,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.core.tree.NodeUtils; @@ -138,14 +137,6 @@ public Map, List> transformations() { } } - @Deprecated(forRemoval = true) - protected final TreeType execute(TreeType plan) { - // TODO: remove when all implementinc classes are using the async version. - PlainActionFuture executionInfoFuture = new PlainActionFuture<>(); - executeWithInfo(plan, executionInfoFuture); - return executionInfoFuture.actionGet().after; - } - protected final void execute(TreeType plan, ActionListener listener) { executeWithInfo(plan, ActionListener.wrap(executionInfo -> listener.onResponse(executionInfo.after()), listener::onFailure)); } From c6e864bd5fdb98781a5d65ecaa380a2d7a4fe603 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 12:21:14 +0200 Subject: [PATCH 16/17] Fix merge conflict --- .../xpack/esql/session/EsqlSession.java | 73 ++++++++++--------- 1 file changed, 37 insertions(+), 36 deletions(-) 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 053bfe505057e..e9d32ba84a3e6 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 @@ -273,43 +273,44 @@ private void executeSubPlan( ) { LOGGER.debug("Executing subplan:\n{}", subPlans.stubReplacedSubPlan()); // Create a physical plan out of the logical sub-plan - var physicalSubPlan = logicalPlanToPhysicalPlan(subPlans.stubReplacedSubPlan(), request); - - runner.run(physicalSubPlan, listener.delegateFailureAndWrap((next, result) -> { - try { - // Translate the subquery into a separate, coordinator based plan and the results 'broadcasted' as a local relation - completionInfoAccumulator.accumulate(result.completionInfo()); - LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan(), result); - - // replace the original logical plan with the backing result - LogicalPlan newLogicalPlan = optimizedPlan.transformUp( - InlineJoin.class, - // use object equality since the right-hand side shouldn't have changed in the optimizedPlan at this point - // and equals would have ignored name IDs anyway - ij -> ij.right() == subPlans.originalSubPlan() ? InlineJoin.inlineData(ij, resultWrapper) : ij - ); - // TODO: INLINESTATS can we do better here and further optimize the plan AFTER one of the subplans executed? - newLogicalPlan.setOptimized(); - LOGGER.debug("Plan after previous subplan execution:\n{}", newLogicalPlan); - // look for the next inlinejoin plan - var newSubPlan = firstSubPlan(newLogicalPlan); - - if (newSubPlan == null) {// run the final "main" plan - LOGGER.debug("Executing final plan:\n{}", newLogicalPlan); - var newPhysicalPlan = logicalPlanToPhysicalPlan(newLogicalPlan, request); - runner.run(newPhysicalPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { - completionInfoAccumulator.accumulate(finalResult.completionInfo()); - finalListener.onResponse( - new Result(finalResult.schema(), finalResult.pages(), completionInfoAccumulator.finish(), executionInfo) - ); - })); - } else {// continue executing the subplans - executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, listener); + logicalPlanToPhysicalPlan(subPlans.stubReplacedSubPlan(), request, ActionListener.wrap(physicalSubPlan -> { + runner.run(physicalSubPlan, listener.delegateFailureAndWrap((next, result) -> { + try { + // Translate the subquery into a separate, coordinator based plan and the results 'broadcasted' as a local relation + completionInfoAccumulator.accumulate(result.completionInfo()); + LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan(), result); + + // replace the original logical plan with the backing result + LogicalPlan newLogicalPlan = optimizedPlan.transformUp( + InlineJoin.class, + // use object equality since the right-hand side shouldn't have changed in the optimizedPlan at this point + // and equals would have ignored name IDs anyway + ij -> ij.right() == subPlans.originalSubPlan() ? InlineJoin.inlineData(ij, resultWrapper) : ij + ); + // TODO: INLINESTATS can we do better here and further optimize the plan AFTER one of the subplans executed? + newLogicalPlan.setOptimized(); + LOGGER.debug("Plan after previous subplan execution:\n{}", newLogicalPlan); + // look for the next inlinejoin plan + var newSubPlan = firstSubPlan(newLogicalPlan); + + if (newSubPlan == null) {// run the final "main" plan + LOGGER.debug("Executing final plan:\n{}", newLogicalPlan); + logicalPlanToPhysicalPlan(newLogicalPlan, request, ActionListener.wrap(newPhysicalPlan -> { + runner.run(newPhysicalPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { + completionInfoAccumulator.accumulate(finalResult.completionInfo()); + finalListener.onResponse( + new Result(finalResult.schema(), finalResult.pages(), completionInfoAccumulator.finish(), executionInfo) + ); + })); + }, listener::onFailure)); + } else {// continue executing the subplans + executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, listener); + } + } finally { + Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks))); } - } finally { - Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks))); - } - })); + })); + }, listener::onFailure)); } private static LocalRelation resultToPlan(LogicalPlan plan, Result result) { From 8519eccb76b48c642b2cf4faf8d447bab2db6690 Mon Sep 17 00:00:00 2001 From: afoucret Date: Tue, 8 Jul 2025 12:31:55 +0200 Subject: [PATCH 17/17] Remove useless class. --- .../xpack/esql/analysis/AnalyzerRules.java | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java index 21f1bca8d0841..990b8aa584599 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerRules.java @@ -11,7 +11,6 @@ import org.elasticsearch.xpack.esql.core.expression.UnresolvedAttribute; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.rule.ParameterizedRule; -import org.elasticsearch.xpack.esql.rule.Rule; import java.util.ArrayList; import java.util.Collection; @@ -21,24 +20,6 @@ import java.util.function.Supplier; public final class AnalyzerRules { - - public interface AnalyzerRule extends Rule { - abstract class Sync extends Rule.Sync { - // transformUp (post-order) - that is first children and then the node - // but with a twist; only if the tree is not resolved or analyzed - @Override - public final LogicalPlan apply(LogicalPlan plan) { - return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t)); - } - - protected abstract LogicalPlan rule(SubPlan plan); - - protected boolean skipResolved() { - return true; - } - } - } - public interface ParameterizedAnalyzerRule extends ParameterizedRule { abstract class Sync extends ParameterizedRule.Sync { // transformUp (post-order) - that is first children and then the node