Browse Source

ESQL: Reorganize optimizer rules (#112338)

- Organize the optimizer rules consistently for all 4 optimizers (logical, physical, local logical, local physical).
- Move helper methods meant for optimizer rules out of the optimizers into the relevant rules or into helper classes.
- Consolidate the 2 nearly identical logical ParameterizedRules into one.
Alexander Spies 1 year ago
parent
commit
2f08d7d8be
93 changed files with 1862 additions and 1613 deletions
  1. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java
  2. 8 295
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java
  3. 13 745
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java
  4. 54 298
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java
  5. 2 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalVerifier.java
  6. 0 53
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java
  7. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java
  8. 1 77
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java
  9. 2 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalVerifier.java
  10. 48 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PlanConsistencyChecker.java
  11. 5 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/AddDefaultTopN.java
  12. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java
  13. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java
  14. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java
  15. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java
  16. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java
  17. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java
  18. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java
  19. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConvertStringToByteRef.java
  20. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/DuplicateLimitAfterMvExpand.java
  21. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java
  22. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java
  23. 22 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java
  24. 2 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java
  25. 7 4
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java
  26. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java
  27. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java
  28. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java
  29. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java
  30. 8 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java
  31. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java
  32. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java
  33. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneOrderByBeforeStats.java
  34. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java
  35. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java
  36. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java
  37. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java
  38. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java
  39. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java
  40. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java
  41. 210 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownUtils.java
  42. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java
  43. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java
  44. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java
  45. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java
  46. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java
  47. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java
  48. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsAggExpressionWithEval.java
  49. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsNestedExpressionWithEval.java
  50. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java
  51. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java
  52. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java
  53. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java
  54. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java
  55. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java
  56. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSpatialSurrogates.java
  57. 2 3
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogates.java
  58. 40 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TemporaryNameUtils.java
  59. 1 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateMetricsAggregate.java
  60. 111 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java
  61. 75 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java
  62. 48 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/LocalPropagateEmptyRelation.java
  63. 99 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java
  64. 30 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java
  65. 91 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java
  66. 160 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java
  67. 86 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java
  68. 37 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushDownUtils.java
  69. 218 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java
  70. 28 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java
  71. 124 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java
  72. 64 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java
  73. 50 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java
  74. 131 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java
  75. 3 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java
  76. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java
  77. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java
  78. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java
  79. 0 17
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/FoldNull.java
  80. 6 5
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java
  81. 10 8
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java
  82. 2 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java
  83. 0 18
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PropagateNullable.java
  84. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsEliminationTests.java
  85. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplificationTests.java
  86. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisonsTests.java
  87. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctionsTests.java
  88. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java
  89. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNullTests.java
  90. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRightTests.java
  91. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEqualsTests.java
  92. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullableTests.java
  93. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java

@@ -61,7 +61,7 @@ import java.util.stream.Stream;
 import static org.elasticsearch.xpack.esql.common.Failure.fail;
 import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST;
 import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN;
-import static org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer.PushFiltersToSource.canPushToSource;
+import static org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushFiltersToSource.canPushToSource;
 
 /**
  * This class is part of the planner. Responsible for failing impossible queries with a human-readable error message.  In particular, this

+ 8 - 295
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java

@@ -7,63 +7,27 @@
 
 package org.elasticsearch.xpack.esql.optimizer;
 
-import org.elasticsearch.common.util.Maps;
-import org.elasticsearch.common.util.set.Sets;
-import org.elasticsearch.compute.data.Block;
-import org.elasticsearch.compute.data.BlockFactory;
-import org.elasticsearch.compute.data.BlockUtils;
-import org.elasticsearch.xpack.esql.core.expression.Alias;
-import org.elasticsearch.xpack.esql.core.expression.Attribute;
-import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
-import org.elasticsearch.xpack.esql.core.expression.Expression;
-import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
-import org.elasticsearch.xpack.esql.core.expression.Literal;
-import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
-import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
-import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull;
-import org.elasticsearch.xpack.esql.core.rule.ParameterizedRule;
 import org.elasticsearch.xpack.esql.core.rule.ParameterizedRuleExecutor;
-import org.elasticsearch.xpack.esql.core.rule.Rule;
-import org.elasticsearch.xpack.esql.core.type.DataType;
-import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
-import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
-import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
-import org.elasticsearch.xpack.esql.expression.function.scalar.nulls.Coalesce;
-import org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules;
-import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEmptyRelation;
-import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
-import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
-import org.elasticsearch.xpack.esql.plan.logical.Eval;
-import org.elasticsearch.xpack.esql.plan.logical.Filter;
-import org.elasticsearch.xpack.esql.plan.logical.Limit;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEmptyRelation;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.InferIsNotNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.InferNonNullAggConstraint;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.LocalPropagateEmptyRelation;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.ReplaceMissingFieldWithNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.ReplaceTopNWithLimitAndSort;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
-import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
-import org.elasticsearch.xpack.esql.plan.logical.Project;
-import org.elasticsearch.xpack.esql.plan.logical.RegexExtract;
-import org.elasticsearch.xpack.esql.plan.logical.TopN;
-import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation;
-import org.elasticsearch.xpack.esql.planner.AbstractPhysicalOperationProviders;
-import org.elasticsearch.xpack.esql.planner.PlannerUtils;
-import org.elasticsearch.xpack.esql.stats.SearchStats;
 
 import java.util.ArrayList;
-import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import static java.util.Arrays.asList;
-import static java.util.Collections.emptySet;
 import static org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer.cleanup;
 import static org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer.operators;
-import static org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules.TransformDirection.UP;
 
 /**
- * <p>This class is part of the planner. Data node level logical optimizations.  At this point we have access to
- * {@link org.elasticsearch.xpack.esql.stats.SearchStats} which provides access to metadata about the index. </p>
+ * This class is part of the planner. Data node level logical optimizations.  At this point we have access to
+ * {@link org.elasticsearch.xpack.esql.stats.SearchStats} which provides access to metadata about the index.
  *
  * <p>NB: This class also reapplies all the rules from {@link LogicalPlanOptimizer#operators()} and {@link LogicalPlanOptimizer#cleanup()}
- * </p>
  */
 public class LocalLogicalPlanOptimizer extends ParameterizedRuleExecutor<LogicalPlan, LocalLogicalOptimizerContext> {
 
@@ -105,255 +69,4 @@ public class LocalLogicalPlanOptimizer extends ParameterizedRuleExecutor<Logical
     public LogicalPlan localOptimize(LogicalPlan plan) {
         return execute(plan);
     }
-
-    /**
-     * Break TopN back into Limit + OrderBy to allow the order rules to kick in.
-     */
-    public static class ReplaceTopNWithLimitAndSort extends OptimizerRules.OptimizerRule<TopN> {
-        public ReplaceTopNWithLimitAndSort() {
-            super(UP);
-        }
-
-        @Override
-        protected LogicalPlan rule(TopN plan) {
-            return new Limit(plan.source(), plan.limit(), new OrderBy(plan.source(), plan.child(), plan.order()));
-        }
-    }
-
-    /**
-     * Look for any fields used in the plan that are missing locally and replace them with null.
-     * This should minimize the plan execution, in the best scenario skipping its execution all together.
-     */
-    private static class ReplaceMissingFieldWithNull extends ParameterizedRule<LogicalPlan, LogicalPlan, LocalLogicalOptimizerContext> {
-
-        @Override
-        public LogicalPlan apply(LogicalPlan plan, LocalLogicalOptimizerContext localLogicalOptimizerContext) {
-            return plan.transformUp(p -> missingToNull(p, localLogicalOptimizerContext.searchStats()));
-        }
-
-        private LogicalPlan missingToNull(LogicalPlan plan, SearchStats stats) {
-            if (plan instanceof EsRelation || plan instanceof LocalRelation) {
-                return plan;
-            }
-
-            if (plan instanceof Aggregate a) {
-                // don't do anything (for now)
-                return a;
-            }
-            // keep the aliased name
-            else if (plan instanceof Project project) {
-                var projections = project.projections();
-                List<NamedExpression> newProjections = new ArrayList<>(projections.size());
-                Map<DataType, Alias> nullLiteral = Maps.newLinkedHashMapWithExpectedSize(DataType.types().size());
-
-                for (NamedExpression projection : projections) {
-                    // Do not use the attribute name, this can deviate from the field name for union types.
-                    if (projection instanceof FieldAttribute f && stats.exists(f.fieldName()) == false) {
-                        DataType dt = f.dataType();
-                        Alias nullAlias = nullLiteral.get(f.dataType());
-                        // save the first field as null (per datatype)
-                        if (nullAlias == null) {
-                            Alias alias = new Alias(f.source(), f.name(), Literal.of(f, null), f.id());
-                            nullLiteral.put(dt, alias);
-                            projection = alias.toAttribute();
-                        }
-                        // otherwise point to it
-                        else {
-                            // since avoids creating field copies
-                            projection = new Alias(f.source(), f.name(), nullAlias.toAttribute(), f.id());
-                        }
-                    }
-
-                    newProjections.add(projection);
-                }
-                // add the first found field as null
-                if (nullLiteral.size() > 0) {
-                    plan = new Eval(project.source(), project.child(), new ArrayList<>(nullLiteral.values()));
-                    plan = new Project(project.source(), plan, newProjections);
-                }
-            } else if (plan instanceof Eval
-                || plan instanceof Filter
-                || plan instanceof OrderBy
-                || plan instanceof RegexExtract
-                || plan instanceof TopN) {
-                    plan = plan.transformExpressionsOnlyUp(
-                        FieldAttribute.class,
-                        // Do not use the attribute name, this can deviate from the field name for union types.
-                        f -> stats.exists(f.fieldName()) ? f : Literal.of(f, null)
-                    );
-                }
-
-            return plan;
-        }
-    }
-
-    /**
-     * Simplify IsNotNull targets by resolving the underlying expression to its root fields with unknown
-     * nullability.
-     * e.g.
-     * (x + 1) / 2 IS NOT NULL --> x IS NOT NULL AND (x+1) / 2 IS NOT NULL
-     * SUBSTRING(x, 3) > 4 IS NOT NULL --> x IS NOT NULL AND SUBSTRING(x, 3) > 4 IS NOT NULL
-     * When dealing with multiple fields, a conjunction/disjunction based on the predicate:
-     * (x + y) / 4 IS NOT NULL --> x IS NOT NULL AND y IS NOT NULL AND (x + y) / 4 IS NOT NULL
-     * This handles the case of fields nested inside functions or expressions in order to avoid:
-     * - having to evaluate the whole expression
-     * - not pushing down the filter due to expression evaluation
-     * IS NULL cannot be simplified since it leads to a disjunction which prevents the filter to be
-     * pushed down:
-     * (x + 1) IS NULL --> x IS NULL OR x + 1 IS NULL
-     * and x IS NULL cannot be pushed down
-     * <br/>
-     * Implementation-wise this rule goes bottom-up, keeping an alias up to date to the current plan
-     * and then looks for replacing the target.
-     */
-    static class InferIsNotNull extends Rule<LogicalPlan, LogicalPlan> {
-
-        @Override
-        public LogicalPlan apply(LogicalPlan plan) {
-            // the alias map is shared across the whole plan
-            AttributeMap<Expression> aliases = new AttributeMap<>();
-            // traverse bottom-up to pick up the aliases as we go
-            plan = plan.transformUp(p -> inspectPlan(p, aliases));
-            return plan;
-        }
-
-        private LogicalPlan inspectPlan(LogicalPlan plan, AttributeMap<Expression> aliases) {
-            // inspect just this plan properties
-            plan.forEachExpression(Alias.class, a -> aliases.put(a.toAttribute(), a.child()));
-            // now go about finding isNull/isNotNull
-            LogicalPlan newPlan = plan.transformExpressionsOnlyUp(IsNotNull.class, inn -> inferNotNullable(inn, aliases));
-            return newPlan;
-        }
-
-        private Expression inferNotNullable(IsNotNull inn, AttributeMap<Expression> aliases) {
-            Expression result = inn;
-            Set<Expression> refs = resolveExpressionAsRootAttributes(inn.field(), aliases);
-            // no refs found or could not detect - return the original function
-            if (refs.size() > 0) {
-                // add IsNull for the filters along with the initial inn
-                var innList = CollectionUtils.combine(refs.stream().map(r -> (Expression) new IsNotNull(inn.source(), r)).toList(), inn);
-                result = Predicates.combineAnd(innList);
-            }
-            return result;
-        }
-
-        /**
-         * Unroll the expression to its references to get to the root fields
-         * that really matter for filtering.
-         */
-        protected Set<Expression> resolveExpressionAsRootAttributes(Expression exp, AttributeMap<Expression> aliases) {
-            Set<Expression> resolvedExpressions = new LinkedHashSet<>();
-            boolean changed = doResolve(exp, aliases, resolvedExpressions);
-            return changed ? resolvedExpressions : emptySet();
-        }
-
-        private boolean doResolve(Expression exp, AttributeMap<Expression> aliases, Set<Expression> resolvedExpressions) {
-            boolean changed = false;
-            // check if the expression can be skipped or is not nullabe
-            if (skipExpression(exp)) {
-                resolvedExpressions.add(exp);
-            } else {
-                for (Expression e : exp.references()) {
-                    Expression resolved = aliases.resolve(e, e);
-                    // found a root attribute, bail out
-                    if (resolved instanceof Attribute a && resolved == e) {
-                        resolvedExpressions.add(a);
-                        // don't mark things as change if the original expression hasn't been broken down
-                        changed |= resolved != exp;
-                    } else {
-                        // go further
-                        changed |= doResolve(resolved, aliases, resolvedExpressions);
-                    }
-                }
-            }
-            return changed;
-        }
-
-        private static boolean skipExpression(Expression e) {
-            return e instanceof Coalesce;
-        }
-    }
-
-    /**
-     * Local aggregation can only produce intermediate state that get wired into the global agg.
-     */
-    private static class LocalPropagateEmptyRelation extends PropagateEmptyRelation {
-
-        /**
-         * Local variant of the aggregation that returns the intermediate value.
-         */
-        @Override
-        protected void aggOutput(NamedExpression agg, AggregateFunction aggFunc, BlockFactory blockFactory, List<Block> blocks) {
-            List<Attribute> output = AbstractPhysicalOperationProviders.intermediateAttributes(List.of(agg), List.of());
-            for (Attribute o : output) {
-                DataType dataType = o.dataType();
-                // boolean right now is used for the internal #seen so always return true
-                var value = dataType == DataType.BOOLEAN ? true
-                    // look for count(literal) with literal != null
-                    : aggFunc instanceof Count count && (count.foldable() == false || count.fold() != null) ? 0L
-                    // otherwise nullify
-                    : null;
-                var wrapper = BlockUtils.wrapperFor(blockFactory, PlannerUtils.toElementType(dataType), 1);
-                wrapper.accept(value);
-                blocks.add(wrapper.builder().build());
-            }
-        }
-    }
-
-    /**
-     * The vast majority of aggs ignore null entries - this rule adds a pushable filter, as it is cheap
-     * to execute, to filter this entries out to begin with.
-     * STATS x = min(a), y = sum(b)
-     * becomes
-     * | WHERE a IS NOT NULL OR b IS NOT NULL
-     * | STATS x = min(a), y = sum(b)
-     * <br>
-     * Unfortunately this optimization cannot be applied when grouping is necessary since it can filter out
-     * groups containing only null values
-     */
-    static class InferNonNullAggConstraint extends ParameterizedOptimizerRule<Aggregate, LocalLogicalOptimizerContext> {
-
-        @Override
-        protected LogicalPlan rule(Aggregate aggregate, LocalLogicalOptimizerContext context) {
-            // only look at aggregates with default grouping
-            if (aggregate.groupings().size() > 0) {
-                return aggregate;
-            }
-
-            SearchStats stats = context.searchStats();
-            LogicalPlan plan = aggregate;
-            var aggs = aggregate.aggregates();
-            Set<Expression> nonNullAggFields = Sets.newLinkedHashSetWithExpectedSize(aggs.size());
-            for (var agg : aggs) {
-                if (Alias.unwrap(agg) instanceof AggregateFunction af) {
-                    Expression field = af.field();
-                    // ignore literals (e.g. COUNT(1))
-                    // make sure the field exists at the source and is indexed (not runtime)
-                    if (field.foldable() == false && field instanceof FieldAttribute fa && stats.isIndexed(fa.name())) {
-                        nonNullAggFields.add(field);
-                    } else {
-                        // otherwise bail out since unless disjunction needs to cover _all_ fields, things get filtered out
-                        return plan;
-                    }
-                }
-            }
-
-            if (nonNullAggFields.size() > 0) {
-                Expression condition = Predicates.combineOr(
-                    nonNullAggFields.stream().map(f -> (Expression) new IsNotNull(aggregate.source(), f)).toList()
-                );
-                plan = aggregate.replaceChild(new Filter(aggregate.source(), aggregate.child(), condition));
-            }
-            return plan;
-        }
-    }
-
-    abstract static class ParameterizedOptimizerRule<SubPlan extends LogicalPlan, P> extends ParameterizedRule<SubPlan, LogicalPlan, P> {
-
-        public final LogicalPlan apply(LogicalPlan plan, P context) {
-            return plan.transformUp(typeToken(), t -> rule(t, context));
-        }
-
-        protected abstract LogicalPlan rule(SubPlan plan, P context);
-    }
 }

+ 13 - 745
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java

@@ -7,108 +7,31 @@
 
 package org.elasticsearch.xpack.esql.optimizer;
 
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.core.Tuple;
-import org.elasticsearch.geometry.Circle;
-import org.elasticsearch.geometry.Geometry;
-import org.elasticsearch.geometry.Point;
-import org.elasticsearch.geometry.utils.WellKnownBinary;
-import org.elasticsearch.index.IndexMode;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.xpack.esql.VerificationException;
 import org.elasticsearch.xpack.esql.common.Failure;
-import org.elasticsearch.xpack.esql.core.expression.Alias;
-import org.elasticsearch.xpack.esql.core.expression.Attribute;
-import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
-import org.elasticsearch.xpack.esql.core.expression.Expression;
-import org.elasticsearch.xpack.esql.core.expression.Expressions;
-import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
-import org.elasticsearch.xpack.esql.core.expression.Literal;
-import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
-import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
-import org.elasticsearch.xpack.esql.core.expression.TypedAttribute;
-import org.elasticsearch.xpack.esql.core.expression.function.scalar.UnaryScalarFunction;
-import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
-import org.elasticsearch.xpack.esql.core.expression.predicate.Range;
-import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate;
-import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.StringQueryPredicate;
-import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And;
-import org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic;
-import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not;
-import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull;
-import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNull;
-import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison;
-import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RegexMatch;
-import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardLike;
-import org.elasticsearch.xpack.esql.core.querydsl.query.Query;
 import org.elasticsearch.xpack.esql.core.rule.ParameterizedRuleExecutor;
 import org.elasticsearch.xpack.esql.core.rule.Rule;
-import org.elasticsearch.xpack.esql.core.tree.Source;
-import org.elasticsearch.xpack.esql.core.type.DataType;
-import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
-import org.elasticsearch.xpack.esql.core.util.Queries;
-import org.elasticsearch.xpack.esql.core.util.Queries.Clause;
-import org.elasticsearch.xpack.esql.core.util.StringUtils;
-import org.elasticsearch.xpack.esql.expression.Order;
-import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
-import org.elasticsearch.xpack.esql.expression.function.aggregate.SpatialAggregateFunction;
-import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch;
-import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialDisjoint;
-import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialIntersects;
-import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction;
-import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesUtils;
-import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.StDistance;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.InsensitiveBinaryComparison;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual;
-import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals;
-import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules.OptimizerRule;
-import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
-import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
-import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec;
-import org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec;
-import org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec.Stat;
-import org.elasticsearch.xpack.esql.plan.physical.EvalExec;
-import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
-import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
-import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
-import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.EnableSpatialDistancePushdown;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.InsertFieldExtraction;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushFiltersToSource;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushLimitToSource;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushStatsToSource;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushTopNToSource;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.ReplaceSourceAttributes;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.SpatialDocValuesExtraction;
 import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
-import org.elasticsearch.xpack.esql.plan.physical.TopNExec;
-import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
-import org.elasticsearch.xpack.esql.planner.AbstractPhysicalOperationProviders;
-import org.elasticsearch.xpack.esql.planner.EsqlTranslatorHandler;
-import org.elasticsearch.xpack.esql.stats.SearchStats;
 
-import java.nio.ByteOrder;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.Set;
-import java.util.function.Predicate;
 
 import static java.util.Arrays.asList;
-import static java.util.Collections.emptyList;
-import static java.util.Collections.singletonList;
-import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.splitAnd;
-import static org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules.TransformDirection.UP;
-import static org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec.StatsType.COUNT;
 
 /**
  * Manages field extraction and pushing parts of the query into Lucene. (Query elements that are not pushed into Lucene are executed via
  * the compute engine)
  */
 public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor<PhysicalPlan, LocalPhysicalOptimizerContext> {
-    public static final EsqlTranslatorHandler TRANSLATOR_HANDLER = new EsqlTranslatorHandler();
 
     private final PhysicalVerifier verifier = PhysicalVerifier.INSTANCE;
 
@@ -128,6 +51,11 @@ public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor<Physic
         return plan;
     }
 
+    @Override
+    protected List<Batch<PhysicalPlan>> batches() {
+        return rules(true);
+    }
+
     protected List<Batch<PhysicalPlan>> rules(boolean optimizeForEsSource) {
         List<Rule<?, PhysicalPlan>> esSourceRules = new ArrayList<>(4);
         esSourceRules.add(new ReplaceSourceAttributes());
@@ -149,664 +77,4 @@ public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor<Physic
         return asList(pushdown, fieldExtraction);
     }
 
-    @Override
-    protected List<Batch<PhysicalPlan>> batches() {
-        return rules(true);
-    }
-
-    private static class ReplaceSourceAttributes extends OptimizerRule<EsSourceExec> {
-
-        ReplaceSourceAttributes() {
-            super(UP);
-        }
-
-        @Override
-        protected PhysicalPlan rule(EsSourceExec plan) {
-            var docId = new FieldAttribute(plan.source(), EsQueryExec.DOC_ID_FIELD.getName(), EsQueryExec.DOC_ID_FIELD);
-            if (plan.indexMode() == IndexMode.TIME_SERIES) {
-                Attribute tsid = null, timestamp = null;
-                for (Attribute attr : plan.output()) {
-                    String name = attr.name();
-                    if (name.equals(MetadataAttribute.TSID_FIELD)) {
-                        tsid = attr;
-                    } else if (name.equals(MetadataAttribute.TIMESTAMP_FIELD)) {
-                        timestamp = attr;
-                    }
-                }
-                if (tsid == null || timestamp == null) {
-                    throw new IllegalStateException("_tsid or @timestamp are missing from the time-series source");
-                }
-                return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId, tsid, timestamp), plan.query());
-            } else {
-                return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId), plan.query());
-            }
-        }
-    }
-
-    // Materialize the concrete fields that need to be extracted from the storage until the last possible moment.
-    // Expects the local plan to already have a projection containing the fields needed upstream.
-    //
-    // 1. add the materialization right before usage inside the local plan
-    // 2. materialize any missing fields needed further up the chain
-    /**
-     * @see org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer.ProjectAwayColumns
-     */
-    static class InsertFieldExtraction extends Rule<PhysicalPlan, PhysicalPlan> {
-
-        @Override
-        public PhysicalPlan apply(PhysicalPlan plan) {
-            // apply the plan locally, adding a field extractor right before data is loaded
-            // by going bottom-up
-            plan = plan.transformUp(UnaryExec.class, p -> {
-                var missing = missingAttributes(p);
-
-                /*
-                 * If there is a single grouping then we'll try to use ords. Either way
-                 * it loads the field lazily. If we have more than one field we need to
-                 * make sure the fields are loaded for the standard hash aggregator.
-                 */
-                if (p instanceof AggregateExec agg && agg.groupings().size() == 1) {
-                    var leaves = new LinkedList<>();
-                    // TODO: this seems out of place
-                    agg.aggregates()
-                        .stream()
-                        .filter(a -> agg.groupings().contains(a) == false)
-                        .forEach(a -> leaves.addAll(a.collectLeaves()));
-                    var remove = agg.groupings().stream().filter(g -> leaves.contains(g) == false).toList();
-                    missing.removeAll(Expressions.references(remove));
-                }
-
-                // add extractor
-                if (missing.isEmpty() == false) {
-                    // collect source attributes and add the extractor
-                    var extractor = new FieldExtractExec(p.source(), p.child(), List.copyOf(missing));
-                    p = p.replaceChild(extractor);
-                }
-
-                return p;
-            });
-
-            return plan;
-        }
-
-        private static Set<Attribute> missingAttributes(PhysicalPlan p) {
-            var missing = new LinkedHashSet<Attribute>();
-            var input = p.inputSet();
-
-            // collect field attributes used inside expressions
-            p.forEachExpression(TypedAttribute.class, f -> {
-                if (f instanceof FieldAttribute || f instanceof MetadataAttribute) {
-                    if (input.contains(f) == false) {
-                        missing.add(f);
-                    }
-                }
-            });
-            return missing;
-        }
-    }
-
-    public static class PushFiltersToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<
-        FilterExec,
-        LocalPhysicalOptimizerContext> {
-
-        @Override
-        protected PhysicalPlan rule(FilterExec filterExec, LocalPhysicalOptimizerContext ctx) {
-            PhysicalPlan plan = filterExec;
-            if (filterExec.child() instanceof EsQueryExec queryExec) {
-                List<Expression> pushable = new ArrayList<>();
-                List<Expression> nonPushable = new ArrayList<>();
-                for (Expression exp : splitAnd(filterExec.condition())) {
-                    (canPushToSource(exp, x -> hasIdenticalDelegate(x, ctx.searchStats())) ? pushable : nonPushable).add(exp);
-                }
-                // Combine GT, GTE, LT and LTE in pushable to Range if possible
-                List<Expression> newPushable = combineEligiblePushableToRange(pushable);
-                if (newPushable.size() > 0) { // update the executable with pushable conditions
-                    Query queryDSL = TRANSLATOR_HANDLER.asQuery(Predicates.combineAnd(newPushable));
-                    QueryBuilder planQuery = queryDSL.asBuilder();
-                    var query = Queries.combine(Clause.FILTER, asList(queryExec.query(), planQuery));
-                    queryExec = new EsQueryExec(
-                        queryExec.source(),
-                        queryExec.index(),
-                        queryExec.indexMode(),
-                        queryExec.output(),
-                        query,
-                        queryExec.limit(),
-                        queryExec.sorts(),
-                        queryExec.estimatedRowSize()
-                    );
-                    if (nonPushable.size() > 0) { // update filter with remaining non-pushable conditions
-                        plan = new FilterExec(filterExec.source(), queryExec, Predicates.combineAnd(nonPushable));
-                    } else { // prune Filter entirely
-                        plan = queryExec;
-                    }
-                } // else: nothing changes
-            }
-
-            return plan;
-        }
-
-        private static List<Expression> combineEligiblePushableToRange(List<Expression> pushable) {
-            List<EsqlBinaryComparison> bcs = new ArrayList<>();
-            List<Range> ranges = new ArrayList<>();
-            List<Expression> others = new ArrayList<>();
-            boolean changed = false;
-
-            pushable.forEach(e -> {
-                if (e instanceof GreaterThan || e instanceof GreaterThanOrEqual || e instanceof LessThan || e instanceof LessThanOrEqual) {
-                    if (((EsqlBinaryComparison) e).right().foldable()) {
-                        bcs.add((EsqlBinaryComparison) e);
-                    } else {
-                        others.add(e);
-                    }
-                } else {
-                    others.add(e);
-                }
-            });
-
-            for (int i = 0, step = 1; i < bcs.size() - 1; i += step, step = 1) {
-                BinaryComparison main = bcs.get(i);
-                for (int j = i + 1; j < bcs.size(); j++) {
-                    BinaryComparison other = bcs.get(j);
-                    if (main.left().semanticEquals(other.left())) {
-                        // >/>= AND </<=
-                        if ((main instanceof GreaterThan || main instanceof GreaterThanOrEqual)
-                            && (other instanceof LessThan || other instanceof LessThanOrEqual)) {
-                            bcs.remove(j);
-                            bcs.remove(i);
-
-                            ranges.add(
-                                new Range(
-                                    main.source(),
-                                    main.left(),
-                                    main.right(),
-                                    main instanceof GreaterThanOrEqual,
-                                    other.right(),
-                                    other instanceof LessThanOrEqual,
-                                    main.zoneId()
-                                )
-                            );
-
-                            changed = true;
-                            step = 0;
-                            break;
-                        }
-                        // </<= AND >/>=
-                        else if ((other instanceof GreaterThan || other instanceof GreaterThanOrEqual)
-                            && (main instanceof LessThan || main instanceof LessThanOrEqual)) {
-                                bcs.remove(j);
-                                bcs.remove(i);
-
-                                ranges.add(
-                                    new Range(
-                                        main.source(),
-                                        main.left(),
-                                        other.right(),
-                                        other instanceof GreaterThanOrEqual,
-                                        main.right(),
-                                        main instanceof LessThanOrEqual,
-                                        main.zoneId()
-                                    )
-                                );
-
-                                changed = true;
-                                step = 0;
-                                break;
-                            }
-                    }
-                }
-            }
-            return changed ? CollectionUtils.combine(others, bcs, ranges) : pushable;
-        }
-
-        public static boolean canPushToSource(Expression exp, Predicate<FieldAttribute> hasIdenticalDelegate) {
-            if (exp instanceof BinaryComparison bc) {
-                return isAttributePushable(bc.left(), bc, hasIdenticalDelegate) && bc.right().foldable();
-            } else if (exp instanceof InsensitiveBinaryComparison bc) {
-                return isAttributePushable(bc.left(), bc, hasIdenticalDelegate) && bc.right().foldable();
-            } else if (exp instanceof BinaryLogic bl) {
-                return canPushToSource(bl.left(), hasIdenticalDelegate) && canPushToSource(bl.right(), hasIdenticalDelegate);
-            } else if (exp instanceof In in) {
-                return isAttributePushable(in.value(), null, hasIdenticalDelegate) && Expressions.foldable(in.list());
-            } else if (exp instanceof Not not) {
-                return canPushToSource(not.field(), hasIdenticalDelegate);
-            } else if (exp instanceof UnaryScalarFunction usf) {
-                if (usf instanceof RegexMatch<?> || usf instanceof IsNull || usf instanceof IsNotNull) {
-                    if (usf instanceof IsNull || usf instanceof IsNotNull) {
-                        if (usf.field() instanceof FieldAttribute fa && fa.dataType().equals(DataType.TEXT)) {
-                            return true;
-                        }
-                    }
-                    return isAttributePushable(usf.field(), usf, hasIdenticalDelegate);
-                }
-            } else if (exp instanceof CIDRMatch cidrMatch) {
-                return isAttributePushable(cidrMatch.ipField(), cidrMatch, hasIdenticalDelegate)
-                    && Expressions.foldable(cidrMatch.matches());
-            } else if (exp instanceof SpatialRelatesFunction bc) {
-                return bc.canPushToSource(LocalPhysicalPlanOptimizer::isAggregatable);
-            } else if (exp instanceof MatchQueryPredicate mqp) {
-                return mqp.field() instanceof FieldAttribute && DataType.isString(mqp.field().dataType());
-            } else if (exp instanceof StringQueryPredicate) {
-                return true;
-            }
-            return false;
-        }
-
-        private static boolean isAttributePushable(
-            Expression expression,
-            Expression operation,
-            Predicate<FieldAttribute> hasIdenticalDelegate
-        ) {
-            if (isPushableFieldAttribute(expression, hasIdenticalDelegate)) {
-                return true;
-            }
-            if (expression instanceof MetadataAttribute ma && ma.searchable()) {
-                return operation == null
-                    // no range or regex queries supported with metadata fields
-                    || operation instanceof Equals
-                    || operation instanceof NotEquals
-                    || operation instanceof WildcardLike;
-            }
-            return false;
-        }
-    }
-
-    /**
-     * this method is supposed to be used to define if a field can be used for exact push down (eg. sort or filter).
-     * "aggregatable" is the most accurate information we can have from field_caps as of now.
-     * Pushing down operations on fields that are not aggregatable would result in an error.
-     */
-    private static boolean isAggregatable(FieldAttribute f) {
-        return f.exactAttribute().field().isAggregatable();
-    }
-
-    private static class PushLimitToSource extends OptimizerRule<LimitExec> {
-        @Override
-        protected PhysicalPlan rule(LimitExec limitExec) {
-            PhysicalPlan plan = limitExec;
-            PhysicalPlan child = limitExec.child();
-            if (child instanceof EsQueryExec queryExec) { // add_task_parallelism_above_query: false
-                plan = queryExec.withLimit(limitExec.limit());
-            } else if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
-                plan = exchangeExec.replaceChild(queryExec.withLimit(limitExec.limit()));
-            }
-            return plan;
-        }
-    }
-
-    private static class PushTopNToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<
-        TopNExec,
-        LocalPhysicalOptimizerContext> {
-        @Override
-        protected PhysicalPlan rule(TopNExec topNExec, LocalPhysicalOptimizerContext ctx) {
-            PhysicalPlan plan = topNExec;
-            PhysicalPlan child = topNExec.child();
-            if (canPushSorts(child) && canPushDownOrders(topNExec.order(), x -> hasIdenticalDelegate(x, ctx.searchStats()))) {
-                var sorts = buildFieldSorts(topNExec.order());
-                var limit = topNExec.limit();
-
-                if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
-                    plan = exchangeExec.replaceChild(queryExec.withSorts(sorts).withLimit(limit));
-                } else {
-                    plan = ((EsQueryExec) child).withSorts(sorts).withLimit(limit);
-                }
-            }
-            return plan;
-        }
-
-        private boolean canPushDownOrders(List<Order> orders, Predicate<FieldAttribute> hasIdenticalDelegate) {
-            // allow only exact FieldAttributes (no expressions) for sorting
-            return orders.stream().allMatch(o -> isPushableFieldAttribute(o.child(), hasIdenticalDelegate));
-        }
-
-        private List<EsQueryExec.FieldSort> buildFieldSorts(List<Order> orders) {
-            List<EsQueryExec.FieldSort> sorts = new ArrayList<>(orders.size());
-            for (Order o : orders) {
-                sorts.add(new EsQueryExec.FieldSort(((FieldAttribute) o.child()).exactAttribute(), o.direction(), o.nullsPosition()));
-            }
-            return sorts;
-        }
-    }
-
-    private static boolean canPushSorts(PhysicalPlan plan) {
-        if (plan instanceof EsQueryExec queryExec) {
-            return queryExec.canPushSorts();
-        }
-        if (plan instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
-            return queryExec.canPushSorts();
-        }
-        return false;
-    }
-
-    /**
-     * Looks for the case where certain stats exist right before the query and thus can be pushed down.
-     */
-    private static class PushStatsToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<
-        AggregateExec,
-        LocalPhysicalOptimizerContext> {
-
-        @Override
-        protected PhysicalPlan rule(AggregateExec aggregateExec, LocalPhysicalOptimizerContext context) {
-            PhysicalPlan plan = aggregateExec;
-            if (aggregateExec.child() instanceof EsQueryExec queryExec) {
-                var tuple = pushableStats(aggregateExec, context);
-
-                // for the moment support pushing count just for one field
-                List<Stat> stats = tuple.v2();
-                if (stats.size() > 1) {
-                    return aggregateExec;
-                }
-
-                // TODO: handle case where some aggs cannot be pushed down by breaking the aggs into two sources (regular + stats) + union
-                // use the stats since the attributes are larger in size (due to seen)
-                if (tuple.v2().size() == aggregateExec.aggregates().size()) {
-                    plan = new EsStatsQueryExec(
-                        aggregateExec.source(),
-                        queryExec.index(),
-                        queryExec.query(),
-                        queryExec.limit(),
-                        tuple.v1(),
-                        tuple.v2()
-                    );
-                }
-            }
-            return plan;
-        }
-
-        private Tuple<List<Attribute>, List<Stat>> pushableStats(AggregateExec aggregate, LocalPhysicalOptimizerContext context) {
-            AttributeMap<Stat> stats = new AttributeMap<>();
-            Tuple<List<Attribute>, List<Stat>> tuple = new Tuple<>(new ArrayList<>(), new ArrayList<>());
-
-            if (aggregate.groupings().isEmpty()) {
-                for (NamedExpression agg : aggregate.aggregates()) {
-                    var attribute = agg.toAttribute();
-                    Stat stat = stats.computeIfAbsent(attribute, a -> {
-                        if (agg instanceof Alias as) {
-                            Expression child = as.child();
-                            if (child instanceof Count count) {
-                                var target = count.field();
-                                String fieldName = null;
-                                QueryBuilder query = null;
-                                // TODO: add count over field (has to be field attribute)
-                                if (target.foldable()) {
-                                    fieldName = StringUtils.WILDCARD;
-                                }
-                                // check if regular field
-                                else {
-                                    if (target instanceof FieldAttribute fa) {
-                                        var fName = fa.name();
-                                        if (context.searchStats().isSingleValue(fName)) {
-                                            fieldName = fa.name();
-                                            query = QueryBuilders.existsQuery(fieldName);
-                                        }
-                                    }
-                                }
-                                if (fieldName != null) {
-                                    return new Stat(fieldName, COUNT, query);
-                                }
-                            }
-                        }
-                        return null;
-                    });
-                    if (stat != null) {
-                        List<Attribute> intermediateAttributes = AbstractPhysicalOperationProviders.intermediateAttributes(
-                            singletonList(agg),
-                            emptyList()
-                        );
-                        // TODO: the attributes have been recreated here; they will have wrong name ids, and the dependency check will
-                        // probably fail when we fix https://github.com/elastic/elasticsearch/issues/105436.
-                        // We may need to refactor AbstractPhysicalOperationProviders.intermediateAttributes so it doesn't return just
-                        // a list of attributes, but a mapping from the logical to the physical attributes.
-                        tuple.v1().addAll(intermediateAttributes);
-                        tuple.v2().add(stat);
-                    }
-                }
-            }
-
-            return tuple;
-        }
-    }
-
-    public static boolean hasIdenticalDelegate(FieldAttribute attr, SearchStats stats) {
-        return stats.hasIdenticalDelegate(attr.name());
-    }
-
-    public static boolean isPushableFieldAttribute(Expression exp, Predicate<FieldAttribute> hasIdenticalDelegate) {
-        if (exp instanceof FieldAttribute fa && fa.getExactInfo().hasExact() && isAggregatable(fa)) {
-            return fa.dataType() != DataType.TEXT || hasIdenticalDelegate.test(fa);
-        }
-        return false;
-    }
-
-    private static class SpatialDocValuesExtraction extends OptimizerRule<AggregateExec> {
-        @Override
-        protected PhysicalPlan rule(AggregateExec aggregate) {
-            var foundAttributes = new HashSet<FieldAttribute>();
-
-            PhysicalPlan plan = aggregate.transformDown(UnaryExec.class, exec -> {
-                if (exec instanceof AggregateExec agg) {
-                    var orderedAggregates = new ArrayList<NamedExpression>();
-                    var changedAggregates = false;
-                    for (NamedExpression aggExpr : agg.aggregates()) {
-                        if (aggExpr instanceof Alias as && as.child() instanceof SpatialAggregateFunction af) {
-                            if (af.field() instanceof FieldAttribute fieldAttribute
-                                && allowedForDocValues(fieldAttribute, agg, foundAttributes)) {
-                                // We need to both mark the field to load differently, and change the spatial function to know to use it
-                                foundAttributes.add(fieldAttribute);
-                                changedAggregates = true;
-                                orderedAggregates.add(as.replaceChild(af.withDocValues()));
-                            } else {
-                                orderedAggregates.add(aggExpr);
-                            }
-                        } else {
-                            orderedAggregates.add(aggExpr);
-                        }
-                    }
-                    if (changedAggregates) {
-                        exec = new AggregateExec(
-                            agg.source(),
-                            agg.child(),
-                            agg.groupings(),
-                            orderedAggregates,
-                            agg.getMode(),
-                            agg.intermediateAttributes(),
-                            agg.estimatedRowSize()
-                        );
-                    }
-                }
-                if (exec instanceof EvalExec evalExec) {
-                    List<Alias> fields = evalExec.fields();
-                    List<Alias> changed = fields.stream()
-                        .map(
-                            f -> (Alias) f.transformDown(
-                                SpatialRelatesFunction.class,
-                                spatialRelatesFunction -> (spatialRelatesFunction.hasFieldAttribute(foundAttributes))
-                                    ? spatialRelatesFunction.withDocValues(foundAttributes)
-                                    : spatialRelatesFunction
-                            )
-                        )
-                        .toList();
-                    if (changed.equals(fields) == false) {
-                        exec = new EvalExec(exec.source(), exec.child(), changed);
-                    }
-                }
-                if (exec instanceof FilterExec filterExec) {
-                    // Note that ST_CENTROID does not support shapes, but SpatialRelatesFunction does, so when we extend the centroid
-                    // to support shapes, we need to consider loading shape doc-values for both centroid and relates (ST_INTERSECTS)
-                    var condition = filterExec.condition()
-                        .transformDown(
-                            SpatialRelatesFunction.class,
-                            spatialRelatesFunction -> (spatialRelatesFunction.hasFieldAttribute(foundAttributes))
-                                ? spatialRelatesFunction.withDocValues(foundAttributes)
-                                : spatialRelatesFunction
-                        );
-                    if (filterExec.condition().equals(condition) == false) {
-                        exec = new FilterExec(filterExec.source(), filterExec.child(), condition);
-                    }
-                }
-                if (exec instanceof FieldExtractExec fieldExtractExec) {
-                    // Tell the field extractor that it should extract the field from doc-values instead of source values
-                    var attributesToExtract = fieldExtractExec.attributesToExtract();
-                    Set<Attribute> docValuesAttributes = new HashSet<>();
-                    for (Attribute found : foundAttributes) {
-                        if (attributesToExtract.contains(found)) {
-                            docValuesAttributes.add(found);
-                        }
-                    }
-                    if (docValuesAttributes.size() > 0) {
-                        exec = new FieldExtractExec(exec.source(), exec.child(), attributesToExtract, docValuesAttributes);
-                    }
-                }
-                return exec;
-            });
-            return plan;
-        }
-
-        /**
-         * This function disallows the use of more than one field for doc-values extraction in the same spatial relation function.
-         * This is because comparing two doc-values fields is not supported in the current implementation.
-         */
-        private boolean allowedForDocValues(FieldAttribute fieldAttribute, AggregateExec agg, Set<FieldAttribute> foundAttributes) {
-            var candidateDocValuesAttributes = new HashSet<>(foundAttributes);
-            candidateDocValuesAttributes.add(fieldAttribute);
-            var spatialRelatesAttributes = new HashSet<FieldAttribute>();
-            agg.forEachExpressionDown(SpatialRelatesFunction.class, relatesFunction -> {
-                candidateDocValuesAttributes.forEach(candidate -> {
-                    if (relatesFunction.hasFieldAttribute(Set.of(candidate))) {
-                        spatialRelatesAttributes.add(candidate);
-                    }
-                });
-            });
-            // Disallow more than one spatial field to be extracted using doc-values (for now)
-            return spatialRelatesAttributes.size() < 2;
-        }
-    }
-
-    /**
-     * When a spatial distance predicate can be pushed down to lucene, this is done by capturing the distance within the same function.
-     * In principle this is like re-writing the predicate:
-     * <pre>WHERE ST_DISTANCE(field, TO_GEOPOINT("POINT(0 0)")) &lt;= 10000</pre>
-     * as:
-     * <pre>WHERE ST_INTERSECTS(field, TO_GEOSHAPE("CIRCLE(0,0,10000)"))</pre>
-     */
-    public static class EnableSpatialDistancePushdown extends PhysicalOptimizerRules.ParameterizedOptimizerRule<
-        FilterExec,
-        LocalPhysicalOptimizerContext> {
-
-        @Override
-        protected PhysicalPlan rule(FilterExec filterExec, LocalPhysicalOptimizerContext ctx) {
-            PhysicalPlan plan = filterExec;
-            if (filterExec.child() instanceof EsQueryExec) {
-                // Find and rewrite any binary comparisons that involve a distance function and a literal
-                var rewritten = filterExec.condition().transformDown(EsqlBinaryComparison.class, comparison -> {
-                    ComparisonType comparisonType = ComparisonType.from(comparison.getFunctionType());
-                    if (comparison.left() instanceof StDistance dist && comparison.right().foldable()) {
-                        return rewriteComparison(comparison, dist, comparison.right(), comparisonType);
-                    } else if (comparison.right() instanceof StDistance dist && comparison.left().foldable()) {
-                        return rewriteComparison(comparison, dist, comparison.left(), ComparisonType.invert(comparisonType));
-                    }
-                    return comparison;
-                });
-                if (rewritten.equals(filterExec.condition()) == false) {
-                    plan = new FilterExec(filterExec.source(), filterExec.child(), rewritten);
-                }
-            }
-
-            return plan;
-        }
-
-        private Expression rewriteComparison(
-            EsqlBinaryComparison comparison,
-            StDistance dist,
-            Expression literal,
-            ComparisonType comparisonType
-        ) {
-            Object value = literal.fold();
-            if (value instanceof Number number) {
-                if (dist.right().foldable()) {
-                    return rewriteDistanceFilter(comparison, dist.left(), dist.right(), number, comparisonType);
-                } else if (dist.left().foldable()) {
-                    return rewriteDistanceFilter(comparison, dist.right(), dist.left(), number, comparisonType);
-                }
-            }
-            return comparison;
-        }
-
-        private Expression rewriteDistanceFilter(
-            EsqlBinaryComparison comparison,
-            Expression spatialExp,
-            Expression literalExp,
-            Number number,
-            ComparisonType comparisonType
-        ) {
-            Geometry geometry = SpatialRelatesUtils.makeGeometryFromLiteral(literalExp);
-            if (geometry instanceof Point point) {
-                double distance = number.doubleValue();
-                Source source = comparison.source();
-                if (comparisonType.lt) {
-                    distance = comparisonType.eq ? distance : Math.nextDown(distance);
-                    return new SpatialIntersects(source, spatialExp, makeCircleLiteral(point, distance, literalExp));
-                } else if (comparisonType.gt) {
-                    distance = comparisonType.eq ? distance : Math.nextUp(distance);
-                    return new SpatialDisjoint(source, spatialExp, makeCircleLiteral(point, distance, literalExp));
-                } else if (comparisonType.eq) {
-                    return new And(
-                        source,
-                        new SpatialIntersects(source, spatialExp, makeCircleLiteral(point, distance, literalExp)),
-                        new SpatialDisjoint(source, spatialExp, makeCircleLiteral(point, Math.nextDown(distance), literalExp))
-                    );
-                }
-            }
-            return comparison;
-        }
-
-        private Literal makeCircleLiteral(Point point, double distance, Expression literalExpression) {
-            var circle = new Circle(point.getX(), point.getY(), distance);
-            var wkb = WellKnownBinary.toWKB(circle, ByteOrder.LITTLE_ENDIAN);
-            return new Literal(literalExpression.source(), new BytesRef(wkb), DataType.GEO_SHAPE);
-        }
-
-        /**
-         * This enum captures the key differences between various inequalities as perceived from the spatial distance function.
-         * In particular, we need to know which direction the inequality points, with lt=true meaning the left is expected to be smaller
-         * than the right. And eq=true meaning we expect euality as well. We currently don't support Equals and NotEquals, so the third
-         * field disables those.
-         */
-        enum ComparisonType {
-            LTE(true, false, true),
-            LT(true, false, false),
-            GTE(false, true, true),
-            GT(false, true, false),
-            EQ(false, false, true);
-
-            private final boolean lt;
-            private final boolean gt;
-            private final boolean eq;
-
-            ComparisonType(boolean lt, boolean gt, boolean eq) {
-                this.lt = lt;
-                this.gt = gt;
-                this.eq = eq;
-            }
-
-            static ComparisonType from(EsqlBinaryComparison.BinaryComparisonOperation op) {
-                return switch (op) {
-                    case LT -> LT;
-                    case LTE -> LTE;
-                    case GT -> GT;
-                    case GTE -> GTE;
-                    default -> EQ;
-                };
-            }
-
-            static ComparisonType invert(ComparisonType comparisonType) {
-                return switch (comparisonType) {
-                    case LT -> GT;
-                    case LTE -> GTE;
-                    case GT -> LT;
-                    case GTE -> LTE;
-                    default -> EQ;
-                };
-            }
-        }
-    }
 }

+ 54 - 298
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java

@@ -7,87 +7,61 @@
 
 package org.elasticsearch.xpack.esql.optimizer;
 
-import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
 import org.elasticsearch.xpack.esql.VerificationException;
 import org.elasticsearch.xpack.esql.common.Failures;
-import org.elasticsearch.xpack.esql.core.expression.Alias;
-import org.elasticsearch.xpack.esql.core.expression.Attribute;
-import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
-import org.elasticsearch.xpack.esql.core.expression.Expression;
-import org.elasticsearch.xpack.esql.core.expression.Expressions;
-import org.elasticsearch.xpack.esql.core.expression.NameId;
-import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
-import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
-import org.elasticsearch.xpack.esql.core.rule.ParameterizedRule;
 import org.elasticsearch.xpack.esql.core.rule.ParameterizedRuleExecutor;
 import org.elasticsearch.xpack.esql.core.type.DataType;
-import org.elasticsearch.xpack.esql.expression.Order;
-import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
-import org.elasticsearch.xpack.esql.optimizer.rules.AddDefaultTopN;
-import org.elasticsearch.xpack.esql.optimizer.rules.BooleanFunctionEqualsElimination;
-import org.elasticsearch.xpack.esql.optimizer.rules.BooleanSimplification;
-import org.elasticsearch.xpack.esql.optimizer.rules.CombineBinaryComparisons;
-import org.elasticsearch.xpack.esql.optimizer.rules.CombineDisjunctions;
-import org.elasticsearch.xpack.esql.optimizer.rules.CombineEvals;
-import org.elasticsearch.xpack.esql.optimizer.rules.CombineProjections;
-import org.elasticsearch.xpack.esql.optimizer.rules.ConstantFolding;
-import org.elasticsearch.xpack.esql.optimizer.rules.ConvertStringToByteRef;
-import org.elasticsearch.xpack.esql.optimizer.rules.DuplicateLimitAfterMvExpand;
-import org.elasticsearch.xpack.esql.optimizer.rules.FoldNull;
-import org.elasticsearch.xpack.esql.optimizer.rules.LiteralsOnTheRight;
-import org.elasticsearch.xpack.esql.optimizer.rules.PartiallyFoldCase;
-import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEmptyRelation;
-import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEquals;
-import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEvalFoldables;
-import org.elasticsearch.xpack.esql.optimizer.rules.PropagateNullable;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneColumns;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneEmptyPlans;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneFilters;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneLiteralsInOrderBy;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneOrderByBeforeStats;
-import org.elasticsearch.xpack.esql.optimizer.rules.PruneRedundantSortClauses;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownAndCombineFilters;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownAndCombineLimits;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownAndCombineOrderBy;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownEnrich;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownEval;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownRegexExtract;
-import org.elasticsearch.xpack.esql.optimizer.rules.RemoveStatsOverride;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceAliasingEvalWithProject;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceLimitAndSortAsTopN;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceLookupWithJoin;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceOrderByExpressionWithEval;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceRegexMatch;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceStatsAggExpressionWithEval;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceStatsNestedExpressionWithEval;
-import org.elasticsearch.xpack.esql.optimizer.rules.ReplaceTrivialTypeConversions;
-import org.elasticsearch.xpack.esql.optimizer.rules.SetAsOptimized;
-import org.elasticsearch.xpack.esql.optimizer.rules.SimplifyComparisonsArithmetics;
-import org.elasticsearch.xpack.esql.optimizer.rules.SkipQueryOnEmptyMappings;
-import org.elasticsearch.xpack.esql.optimizer.rules.SkipQueryOnLimitZero;
-import org.elasticsearch.xpack.esql.optimizer.rules.SplitInWithFoldableValue;
-import org.elasticsearch.xpack.esql.optimizer.rules.SubstituteSpatialSurrogates;
-import org.elasticsearch.xpack.esql.optimizer.rules.SubstituteSurrogates;
-import org.elasticsearch.xpack.esql.optimizer.rules.TranslateMetricsAggregate;
-import org.elasticsearch.xpack.esql.plan.GeneratingPlan;
-import org.elasticsearch.xpack.esql.plan.logical.Eval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.AddDefaultTopN;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.BooleanFunctionEqualsElimination;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.BooleanSimplification;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.CombineBinaryComparisons;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.CombineDisjunctions;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.CombineEvals;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.CombineProjections;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ConstantFolding;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ConvertStringToByteRef;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.DuplicateLimitAfterMvExpand;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.FoldNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.LiteralsOnTheRight;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PartiallyFoldCase;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEmptyRelation;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEquals;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEvalFoldables;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateNullable;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneColumns;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneEmptyPlans;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneFilters;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneLiteralsInOrderBy;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneOrderByBeforeStats;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneRedundantSortClauses;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownAndCombineFilters;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownAndCombineLimits;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownAndCombineOrderBy;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownEnrich;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownEval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownRegexExtract;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.RemoveStatsOverride;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceAliasingEvalWithProject;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceLimitAndSortAsTopN;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceLookupWithJoin;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceOrderByExpressionWithEval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceRegexMatch;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceStatsAggExpressionWithEval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceStatsNestedExpressionWithEval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceTrivialTypeConversions;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SetAsOptimized;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SimplifyComparisonsArithmetics;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SkipQueryOnEmptyMappings;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SkipQueryOnLimitZero;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SplitInWithFoldableValue;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SubstituteSpatialSurrogates;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SubstituteSurrogates;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.TranslateMetricsAggregate;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
-import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
-import org.elasticsearch.xpack.esql.plan.logical.Project;
-import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
-import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation;
-import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import static java.util.Arrays.asList;
-import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputExpressions;
 
 /**
  * <p>This class is part of the planner</p>
@@ -119,30 +93,6 @@ public class LogicalPlanOptimizer extends ParameterizedRuleExecutor<LogicalPlan,
         super(optimizerContext);
     }
 
-    public static String temporaryName(Expression inner, Expression outer, int suffix) {
-        String in = toString(inner);
-        String out = toString(outer);
-        return Attribute.rawTemporaryName(in, out, String.valueOf(suffix));
-    }
-
-    public static String locallyUniqueTemporaryName(String inner, String outer) {
-        return Attribute.rawTemporaryName(inner, outer, (new NameId()).toString());
-    }
-
-    static String toString(Expression ex) {
-        return ex instanceof AggregateFunction af ? af.functionName() : extractString(ex);
-    }
-
-    static String extractString(Expression ex) {
-        return ex instanceof NamedExpression ne ? ne.name() : limitToString(ex.sourceText()).replace(' ', '_');
-    }
-
-    static int TO_STRING_LIMIT = 16;
-
-    static String limitToString(String string) {
-        return string.length() > TO_STRING_LIMIT ? string.substring(0, TO_STRING_LIMIT - 1) + ">" : string;
-    }
-
     public LogicalPlan optimize(LogicalPlan verified) {
         var optimized = execute(verified);
 
@@ -159,6 +109,14 @@ public class LogicalPlanOptimizer extends ParameterizedRuleExecutor<LogicalPlan,
         return rules();
     }
 
+    protected static List<Batch<LogicalPlan>> rules() {
+        var skip = new Batch<>("Skip Compute", new SkipQueryOnLimitZero());
+        var defaultTopN = new Batch<>("Add default TopN", new AddDefaultTopN());
+        var label = new Batch<>("Set as Optimized", Limiter.ONCE, new SetAsOptimized());
+
+        return asList(substitutions(), operators(), skip, cleanup(), defaultTopN, label);
+    }
+
     protected static Batch<LogicalPlan> substitutions() {
         return new Batch<>(
             "Substitutions",
@@ -226,206 +184,4 @@ public class LogicalPlanOptimizer extends ParameterizedRuleExecutor<LogicalPlan,
     protected static Batch<LogicalPlan> cleanup() {
         return new Batch<>("Clean Up", new ReplaceLimitAndSortAsTopN());
     }
-
-    protected static List<Batch<LogicalPlan>> rules() {
-        var skip = new Batch<>("Skip Compute", new SkipQueryOnLimitZero());
-        var defaultTopN = new Batch<>("Add default TopN", new AddDefaultTopN());
-        var label = new Batch<>("Set as Optimized", Limiter.ONCE, new SetAsOptimized());
-
-        return asList(substitutions(), operators(), skip, cleanup(), defaultTopN, label);
-    }
-
-    public static LogicalPlan skipPlan(UnaryPlan plan) {
-        return new LocalRelation(plan.source(), plan.output(), LocalSupplier.EMPTY);
-    }
-
-    public static LogicalPlan skipPlan(UnaryPlan plan, LocalSupplier supplier) {
-        return new LocalRelation(plan.source(), plan.output(), supplier);
-    }
-
-    /**
-     * Pushes LogicalPlans which generate new attributes (Eval, Grok/Dissect, Enrich), past OrderBys and Projections.
-     * Although it seems arbitrary whether the OrderBy or the generating plan is executed first, this transformation ensures that OrderBys
-     * only separated by e.g. an Eval can be combined by {@link PushDownAndCombineOrderBy}.
-     * <p>
-     * E.g. {@code ... | sort a | eval x = b + 1 | sort x} becomes {@code ... | eval x = b + 1 | sort a | sort x}
-     * <p>
-     * Ordering the generating plans before the OrderBys has the advantage that it's always possible to order the plans like this.
-     * E.g., in the example above it would not be possible to put the eval after the two orderBys.
-     * <p>
-     * In case one of the generating plan's attributes would shadow the OrderBy's attributes, we alias the generated attribute first.
-     * <p>
-     * E.g. {@code ... | sort a | eval a = b + 1 | ...} becomes {@code ... | eval $$a = a | eval a = b + 1 | sort $$a | drop $$a ...}
-     * <p>
-     * In case the generating plan's attributes would shadow the Project's attributes, we rename the generated attributes in place.
-     * <p>
-     * E.g. {@code ... | rename a as z | eval a = b + 1 | ...} becomes {@code ... eval $$a = b + 1 | rename a as z, $$a as a ...}
-     */
-    public static <Plan extends UnaryPlan & GeneratingPlan<Plan>> LogicalPlan pushGeneratingPlanPastProjectAndOrderBy(Plan generatingPlan) {
-        LogicalPlan child = generatingPlan.child();
-        if (child instanceof OrderBy orderBy) {
-            Set<String> evalFieldNames = new LinkedHashSet<>(Expressions.names(generatingPlan.generatedAttributes()));
-
-            // Look for attributes in the OrderBy's expressions and create aliases with temporary names for them.
-            AttributeReplacement nonShadowedOrders = renameAttributesInExpressions(evalFieldNames, orderBy.order());
-
-            AttributeMap<Alias> aliasesForShadowedOrderByAttrs = nonShadowedOrders.replacedAttributes;
-            @SuppressWarnings("unchecked")
-            List<Order> newOrder = (List<Order>) (List<?>) nonShadowedOrders.rewrittenExpressions;
-
-            if (aliasesForShadowedOrderByAttrs.isEmpty() == false) {
-                List<Alias> newAliases = new ArrayList<>(aliasesForShadowedOrderByAttrs.values());
-
-                LogicalPlan plan = new Eval(orderBy.source(), orderBy.child(), newAliases);
-                plan = generatingPlan.replaceChild(plan);
-                plan = new OrderBy(orderBy.source(), plan, newOrder);
-                plan = new Project(generatingPlan.source(), plan, generatingPlan.output());
-
-                return plan;
-            }
-
-            return orderBy.replaceChild(generatingPlan.replaceChild(orderBy.child()));
-        } else if (child instanceof Project project) {
-            // We need to account for attribute shadowing: a rename might rely on a name generated in an Eval/Grok/Dissect/Enrich.
-            // E.g. in:
-            //
-            // Eval[[2 * x{f}#1 AS y]]
-            // \_Project[[x{f}#1, y{f}#2, y{f}#2 AS z]]
-            //
-            // Just moving the Eval down breaks z because we shadow y{f}#2.
-            // Instead, we use a different alias in the Eval, eventually renaming back to y:
-            //
-            // Project[[x{f}#1, y{f}#2 as z, $$y{r}#3 as y]]
-            // \_Eval[[2 * x{f}#1 as $$y]]
-
-            List<Attribute> generatedAttributes = generatingPlan.generatedAttributes();
-
-            @SuppressWarnings("unchecked")
-            Plan generatingPlanWithResolvedExpressions = (Plan) resolveRenamesFromProject(generatingPlan, project);
-
-            Set<String> namesReferencedInRenames = new HashSet<>();
-            for (NamedExpression ne : project.projections()) {
-                if (ne instanceof Alias as) {
-                    namesReferencedInRenames.addAll(as.child().references().names());
-                }
-            }
-            Map<String, String> renameGeneratedAttributeTo = newNamesForConflictingAttributes(
-                generatingPlan.generatedAttributes(),
-                namesReferencedInRenames
-            );
-            List<String> newNames = generatedAttributes.stream()
-                .map(attr -> renameGeneratedAttributeTo.getOrDefault(attr.name(), attr.name()))
-                .toList();
-            Plan generatingPlanWithRenamedAttributes = generatingPlanWithResolvedExpressions.withGeneratedNames(newNames);
-
-            // Put the project at the top, but include the generated attributes.
-            // Any generated attributes that had to be renamed need to be re-renamed to their original names.
-            List<NamedExpression> generatedAttributesRenamedToOriginal = new ArrayList<>(generatedAttributes.size());
-            List<Attribute> renamedGeneratedAttributes = generatingPlanWithRenamedAttributes.generatedAttributes();
-            for (int i = 0; i < generatedAttributes.size(); i++) {
-                Attribute originalAttribute = generatedAttributes.get(i);
-                Attribute renamedAttribute = renamedGeneratedAttributes.get(i);
-                if (originalAttribute.name().equals(renamedAttribute.name())) {
-                    generatedAttributesRenamedToOriginal.add(renamedAttribute);
-                } else {
-                    generatedAttributesRenamedToOriginal.add(
-                        new Alias(
-                            originalAttribute.source(),
-                            originalAttribute.name(),
-                            renamedAttribute,
-                            originalAttribute.id(),
-                            originalAttribute.synthetic()
-                        )
-                    );
-                }
-            }
-
-            Project projectWithGeneratingChild = project.replaceChild(generatingPlanWithRenamedAttributes.replaceChild(project.child()));
-            return projectWithGeneratingChild.withProjections(
-                mergeOutputExpressions(generatedAttributesRenamedToOriginal, projectWithGeneratingChild.projections())
-            );
-        }
-
-        return generatingPlan;
-    }
-
-    private record AttributeReplacement(List<Expression> rewrittenExpressions, AttributeMap<Alias> replacedAttributes) {};
-
-    /**
-     * Replace attributes in the given expressions by assigning them temporary names.
-     * Returns the rewritten expressions and a map with an alias for each replaced attribute; the rewritten expressions reference
-     * these aliases.
-     */
-    private static AttributeReplacement renameAttributesInExpressions(
-        Set<String> attributeNamesToRename,
-        List<? extends Expression> expressions
-    ) {
-        AttributeMap<Alias> aliasesForReplacedAttributes = new AttributeMap<>();
-        List<Expression> rewrittenExpressions = new ArrayList<>();
-
-        for (Expression expr : expressions) {
-            rewrittenExpressions.add(expr.transformUp(Attribute.class, attr -> {
-                if (attributeNamesToRename.contains(attr.name())) {
-                    Alias renamedAttribute = aliasesForReplacedAttributes.computeIfAbsent(attr, a -> {
-                        String tempName = locallyUniqueTemporaryName(a.name(), "temp_name");
-                        return new Alias(a.source(), tempName, a, null, true);
-                    });
-                    return renamedAttribute.toAttribute();
-                }
-
-                return attr;
-            }));
-        }
-
-        return new AttributeReplacement(rewrittenExpressions, aliasesForReplacedAttributes);
-    }
-
-    private static Map<String, String> newNamesForConflictingAttributes(
-        List<Attribute> potentiallyConflictingAttributes,
-        Set<String> reservedNames
-    ) {
-        if (reservedNames.isEmpty()) {
-            return Map.of();
-        }
-
-        Map<String, String> renameAttributeTo = new HashMap<>();
-        for (Attribute attr : potentiallyConflictingAttributes) {
-            String name = attr.name();
-            if (reservedNames.contains(name)) {
-                renameAttributeTo.putIfAbsent(name, locallyUniqueTemporaryName(name, "temp_name"));
-            }
-        }
-
-        return renameAttributeTo;
-    }
-
-    public static Project pushDownPastProject(UnaryPlan parent) {
-        if (parent.child() instanceof Project project) {
-            UnaryPlan expressionsWithResolvedAliases = resolveRenamesFromProject(parent, project);
-
-            return project.replaceChild(expressionsWithResolvedAliases.replaceChild(project.child()));
-        } else {
-            throw new EsqlIllegalArgumentException("Expected child to be instance of Project");
-        }
-    }
-
-    private static UnaryPlan resolveRenamesFromProject(UnaryPlan plan, Project project) {
-        AttributeMap.Builder<Expression> aliasBuilder = AttributeMap.builder();
-        project.forEachExpression(Alias.class, a -> aliasBuilder.put(a.toAttribute(), a.child()));
-        var aliases = aliasBuilder.build();
-
-        return (UnaryPlan) plan.transformExpressionsOnly(ReferenceAttribute.class, r -> aliases.resolve(r, r));
-    }
-
-    public abstract static class ParameterizedOptimizerRule<SubPlan extends LogicalPlan, P> extends ParameterizedRule<
-        SubPlan,
-        LogicalPlan,
-        P> {
-
-        public final LogicalPlan apply(LogicalPlan plan, P context) {
-            return plan.transformDown(typeToken(), t -> rule(t, context));
-        }
-
-        protected abstract LogicalPlan rule(SubPlan plan, P context);
-    }
 }

+ 2 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalVerifier.java

@@ -9,12 +9,12 @@ package org.elasticsearch.xpack.esql.optimizer;
 
 import org.elasticsearch.xpack.esql.capabilities.Validatable;
 import org.elasticsearch.xpack.esql.common.Failures;
-import org.elasticsearch.xpack.esql.optimizer.OptimizerRules.DependencyConsistency;
+import org.elasticsearch.xpack.esql.optimizer.rules.PlanConsistencyChecker;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 
 public final class LogicalVerifier {
 
-    private static final DependencyConsistency<LogicalPlan> DEPENDENCY_CHECK = new DependencyConsistency<>();
+    private static final PlanConsistencyChecker<LogicalPlan> DEPENDENCY_CHECK = new PlanConsistencyChecker<>();
     public static final LogicalVerifier INSTANCE = new LogicalVerifier();
 
     private LogicalVerifier() {}

+ 0 - 53
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java

@@ -1,53 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.esql.optimizer;
-
-import org.elasticsearch.xpack.esql.common.Failures;
-import org.elasticsearch.xpack.esql.core.expression.Attribute;
-import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
-import org.elasticsearch.xpack.esql.core.expression.NameId;
-import org.elasticsearch.xpack.esql.plan.QueryPlan;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.elasticsearch.xpack.esql.common.Failure.fail;
-
-class OptimizerRules {
-
-    private OptimizerRules() {}
-
-    static class DependencyConsistency<P extends QueryPlan<P>> {
-
-        void checkPlan(P p, Failures failures) {
-            AttributeSet refs = p.references();
-            AttributeSet input = p.inputSet();
-            AttributeSet missing = refs.subtract(input);
-            // TODO: for Joins, we should probably check if the required fields from the left child are actually in the left child, not
-            // just any child (and analogously for the right child).
-            if (missing.isEmpty() == false) {
-                failures.add(fail(p, "Plan [{}] optimized incorrectly due to missing references {}", p.nodeString(), missing));
-            }
-
-            Set<String> outputAttributeNames = new HashSet<>();
-            Set<NameId> outputAttributeIds = new HashSet<>();
-            for (Attribute outputAttr : p.output()) {
-                if (outputAttributeNames.add(outputAttr.name()) == false || outputAttributeIds.add(outputAttr.id()) == false) {
-                    failures.add(
-                        fail(
-                            p,
-                            "Plan [{}] optimized incorrectly due to duplicate output attribute {}",
-                            p.nodeString(),
-                            outputAttr.toString()
-                        )
-                    );
-                }
-            }
-        }
-    }
-}

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java

@@ -11,7 +11,7 @@ import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.rule.ParameterizedRule;
 import org.elasticsearch.xpack.esql.core.rule.Rule;
 import org.elasticsearch.xpack.esql.core.util.ReflectionUtils;
-import org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules.TransformDirection;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection;
 import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
 
 public class PhysicalOptimizerRules {

+ 1 - 77
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java

@@ -9,32 +9,16 @@ package org.elasticsearch.xpack.esql.optimizer;
 
 import org.elasticsearch.xpack.esql.VerificationException;
 import org.elasticsearch.xpack.esql.common.Failure;
-import org.elasticsearch.xpack.esql.core.expression.Alias;
-import org.elasticsearch.xpack.esql.core.expression.Attribute;
-import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
-import org.elasticsearch.xpack.esql.core.expression.Expressions;
-import org.elasticsearch.xpack.esql.core.expression.Literal;
 import org.elasticsearch.xpack.esql.core.rule.ParameterizedRuleExecutor;
-import org.elasticsearch.xpack.esql.core.rule.Rule;
 import org.elasticsearch.xpack.esql.core.rule.RuleExecutor;
-import org.elasticsearch.xpack.esql.core.tree.Source;
-import org.elasticsearch.xpack.esql.core.util.Holder;
-import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
-import org.elasticsearch.xpack.esql.plan.logical.Eval;
-import org.elasticsearch.xpack.esql.plan.logical.Project;
-import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.ProjectAwayColumns;
 import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
 import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
-import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import static java.lang.Boolean.FALSE;
-import static java.lang.Boolean.TRUE;
 import static java.util.Arrays.asList;
-import static java.util.Collections.singletonList;
 
 /**
  * This class is part of the planner. Performs global (coordinator) optimization of the physical plan. Local (data-node) optimizations
@@ -70,64 +54,4 @@ public class PhysicalPlanOptimizer extends ParameterizedRuleExecutor<PhysicalPla
     protected Iterable<RuleExecutor.Batch<PhysicalPlan>> batches() {
         return rules;
     }
-
-    /**
-     * Adds an explicit project to minimize the amount of attributes sent from the local plan to the coordinator.
-     * This is done here to localize the project close to the data source and simplify the upcoming field
-     * extraction.
-     */
-    static class ProjectAwayColumns extends Rule<PhysicalPlan, PhysicalPlan> {
-
-        @Override
-        public PhysicalPlan apply(PhysicalPlan plan) {
-            Holder<Boolean> keepTraversing = new Holder<>(TRUE);
-            // Invariant: if we add a projection with these attributes after the current plan node, the plan remains valid
-            // and the overall output will not change.
-            Holder<AttributeSet> requiredAttributes = new Holder<>(plan.outputSet());
-
-            // This will require updating should we choose to have non-unary execution plans in the future.
-            return plan.transformDown(UnaryExec.class, currentPlanNode -> {
-                if (keepTraversing.get() == false) {
-                    return currentPlanNode;
-                }
-                if (currentPlanNode instanceof ExchangeExec exec) {
-                    keepTraversing.set(FALSE);
-                    var child = exec.child();
-                    // otherwise expect a Fragment
-                    if (child instanceof FragmentExec fragmentExec) {
-                        var logicalFragment = fragmentExec.fragment();
-
-                        // no need for projection when dealing with aggs
-                        if (logicalFragment instanceof Aggregate == false) {
-                            List<Attribute> output = new ArrayList<>(requiredAttributes.get());
-                            // if all the fields are filtered out, it's only the count that matters
-                            // however until a proper fix (see https://github.com/elastic/elasticsearch/issues/98703)
-                            // add a synthetic field (so it doesn't clash with the user defined one) to return a constant
-                            // to avoid the block from being trimmed
-                            if (output.isEmpty()) {
-                                var alias = new Alias(logicalFragment.source(), "<all-fields-projected>", Literal.NULL, null, true);
-                                List<Alias> fields = singletonList(alias);
-                                logicalFragment = new Eval(logicalFragment.source(), logicalFragment, fields);
-                                output = Expressions.asAttributes(fields);
-                            }
-                            // add a logical projection (let the local replanning remove it if needed)
-                            FragmentExec newChild = new FragmentExec(
-                                Source.EMPTY,
-                                new Project(logicalFragment.source(), logicalFragment, output),
-                                fragmentExec.esFilter(),
-                                fragmentExec.estimatedRowSize(),
-                                fragmentExec.reducer()
-                            );
-                            return new ExchangeExec(exec.source(), output, exec.inBetweenAggs(), newChild);
-                        }
-                    }
-                } else {
-                    AttributeSet childOutput = currentPlanNode.inputSet();
-                    AttributeSet addedAttributes = currentPlanNode.outputSet().subtract(childOutput);
-                    requiredAttributes.set(requiredAttributes.get().subtract(addedAttributes).combine(currentPlanNode.references()));
-                }
-                return currentPlanNode;
-            });
-        }
-    }
 }

+ 2 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalVerifier.java

@@ -10,7 +10,7 @@ package org.elasticsearch.xpack.esql.optimizer;
 import org.elasticsearch.xpack.esql.common.Failure;
 import org.elasticsearch.xpack.esql.core.expression.Attribute;
 import org.elasticsearch.xpack.esql.core.expression.Expressions;
-import org.elasticsearch.xpack.esql.optimizer.OptimizerRules.DependencyConsistency;
+import org.elasticsearch.xpack.esql.optimizer.rules.PlanConsistencyChecker;
 import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
 import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
 
@@ -24,7 +24,7 @@ import static org.elasticsearch.xpack.esql.common.Failure.fail;
 public final class PhysicalVerifier {
 
     public static final PhysicalVerifier INSTANCE = new PhysicalVerifier();
-    private static final DependencyConsistency<PhysicalPlan> DEPENDENCY_CHECK = new DependencyConsistency<>();
+    private static final PlanConsistencyChecker<PhysicalPlan> DEPENDENCY_CHECK = new PlanConsistencyChecker<>();
 
     private PhysicalVerifier() {}
 

+ 48 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PlanConsistencyChecker.java

@@ -0,0 +1,48 @@
+/*
+ * 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.optimizer.rules;
+
+import org.elasticsearch.xpack.esql.common.Failures;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
+import org.elasticsearch.xpack.esql.core.expression.NameId;
+import org.elasticsearch.xpack.esql.plan.QueryPlan;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.elasticsearch.xpack.esql.common.Failure.fail;
+
+public class PlanConsistencyChecker<P extends QueryPlan<P>> {
+
+    /**
+     * Check whether a single {@link QueryPlan} produces no duplicate attributes and its children provide all of its required
+     * {@link QueryPlan#references() references}. Otherwise, add
+     * {@link org.elasticsearch.xpack.esql.common.Failure Failure}s to the {@link Failures} object.
+     */
+    public void checkPlan(P p, Failures failures) {
+        AttributeSet refs = p.references();
+        AttributeSet input = p.inputSet();
+        AttributeSet missing = refs.subtract(input);
+        // TODO: for Joins, we should probably check if the required fields from the left child are actually in the left child, not
+        // just any child (and analogously for the right child).
+        if (missing.isEmpty() == false) {
+            failures.add(fail(p, "Plan [{}] optimized incorrectly due to missing references {}", p.nodeString(), missing));
+        }
+
+        Set<String> outputAttributeNames = new HashSet<>();
+        Set<NameId> outputAttributeIds = new HashSet<>();
+        for (Attribute outputAttr : p.output()) {
+            if (outputAttributeNames.add(outputAttr.name()) == false || outputAttributeIds.add(outputAttr.id()) == false) {
+                failures.add(
+                    fail(p, "Plan [{}] optimized incorrectly due to duplicate output attribute {}", p.nodeString(), outputAttr.toString())
+                );
+            }
+        }
+    }
+}

+ 5 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/AddDefaultTopN.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/AddDefaultTopN.java

@@ -5,12 +5,11 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Literal;
 import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
@@ -39,7 +38,10 @@ import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
  * OR if there is no sort between "limit" and "mv_expand".
  * But, since this type of query has such a filter, the "sort emp_no" will have no limit when it reaches the current rule.
  */
-public final class AddDefaultTopN extends LogicalPlanOptimizer.ParameterizedOptimizerRule<LogicalPlan, LogicalOptimizerContext> {
+public final class AddDefaultTopN extends OptimizerRules.ParameterizedOptimizerRule<LogicalPlan, LogicalOptimizerContext> {
+    public AddDefaultTopN() {
+        super(OptimizerRules.TransformDirection.DOWN);
+    }
 
     @Override
     protected LogicalPlan rule(LogicalPlan plan, LogicalOptimizerContext context) {

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsElimination.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsElimination.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.function.Function;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanSimplification.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplification.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Literal;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineBinaryComparisons.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisons.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctions.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctions.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineEvals.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineEvals.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineProjections.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
 import org.elasticsearch.xpack.esql.core.expression.Alias;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ConstantFolding.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFolding.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Literal;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ConvertStringToByteRef.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConvertStringToByteRef.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/DuplicateLimitAfterMvExpand.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/DuplicateLimitAfterMvExpand.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
 import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/FoldNull.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/LiteralsOnTheRight.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRight.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Literal;
 import org.elasticsearch.xpack.esql.core.expression.predicate.BinaryOperator;

+ 22 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/OptimizerRules.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/OptimizerRules.java

@@ -4,9 +4,10 @@
  * 2.0; you may not use this file except in compliance with the Elastic License
  * 2.0.
  */
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.rule.ParameterizedRule;
 import org.elasticsearch.xpack.esql.core.rule.Rule;
 import org.elasticsearch.xpack.esql.core.util.ReflectionUtils;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
@@ -69,4 +70,24 @@ public final class OptimizerRules {
         UP,
         DOWN
     }
+
+    public abstract static class ParameterizedOptimizerRule<SubPlan extends LogicalPlan, P> extends ParameterizedRule<
+        SubPlan,
+        LogicalPlan,
+        P> {
+
+        private final TransformDirection direction;
+
+        protected ParameterizedOptimizerRule(TransformDirection direction) {
+            this.direction = direction;
+        }
+
+        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);
+    }
 }

+ 2 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PartiallyFoldCase.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PartiallyFoldCase.java

@@ -5,12 +5,12 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.expression.function.scalar.conditional.Case;
 
-import static org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules.TransformDirection.DOWN;
+import static org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection.DOWN;
 
 /**
  * Fold the arms of {@code CASE} statements.

+ 7 - 4
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEmptyRelation.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.BlockFactory;
@@ -15,7 +15,6 @@ import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
@@ -36,9 +35,9 @@ public class PropagateEmptyRelation extends OptimizerRules.OptimizerRule<UnaryPl
             // only care about non-grouped aggs might return something (count)
             if (plan instanceof Aggregate agg && agg.groupings().isEmpty()) {
                 List<Block> emptyBlocks = aggsFromEmpty(agg.aggregates());
-                p = LogicalPlanOptimizer.skipPlan(plan, LocalSupplier.of(emptyBlocks.toArray(Block[]::new)));
+                p = replacePlanByRelation(plan, LocalSupplier.of(emptyBlocks.toArray(Block[]::new)));
             } else {
-                p = LogicalPlanOptimizer.skipPlan(plan);
+                p = PruneEmptyPlans.skipPlan(plan);
             }
         }
         return p;
@@ -69,4 +68,8 @@ public class PropagateEmptyRelation extends OptimizerRules.OptimizerRule<UnaryPl
         wrapper.accept(value);
         blocks.add(wrapper.builder().build());
     }
+
+    private static LogicalPlan replacePlanByRelation(UnaryPlan plan, LocalSupplier supplier) {
+        return new LocalRelation(plan.source(), plan.output(), supplier);
+    }
 }

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEquals.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEquals.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Literal;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEvalFoldables.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEvalFoldables.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.AttributeMap;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateNullable.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullable.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneColumns.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.BlockUtils;

+ 8 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneEmptyPlans.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyPlans.java

@@ -5,16 +5,21 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
+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<UnaryPlan> {
 
+    public static LogicalPlan skipPlan(UnaryPlan plan) {
+        return new LocalRelation(plan.source(), plan.output(), LocalSupplier.EMPTY);
+    }
+
     @Override
     protected LogicalPlan rule(UnaryPlan plan) {
-        return plan.output().isEmpty() ? LogicalPlanOptimizer.skipPlan(plan) : plan;
+        return plan.output().isEmpty() ? skipPlan(plan) : plan;
     }
 }

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneFilters.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Expressions;
@@ -14,7 +14,6 @@ import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And;
 import org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic;
 import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or;
 import org.elasticsearch.xpack.esql.core.type.DataType;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Filter;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 
@@ -31,7 +30,7 @@ public final class PruneFilters extends OptimizerRules.OptimizerRule<Filter> {
                 return filter.child();
             }
             if (FALSE.equals(condition) || Expressions.isNull(condition)) {
-                return LogicalPlanOptimizer.skipPlan(filter);
+                return PruneEmptyPlans.skipPlan(filter);
             }
         }
 

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneLiteralsInOrderBy.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneLiteralsInOrderBy.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.expression.Order;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneOrderByBeforeStats.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneOrderByBeforeStats.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.esql.plan.logical.Enrich;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PruneRedundantSortClauses.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneRedundantSortClauses.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.ExpressionSet;
 import org.elasticsearch.xpack.esql.expression.Order;

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownAndCombineFilters.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Attribute;
 import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
@@ -13,7 +13,6 @@ import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Expressions;
 import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.esql.plan.logical.Enrich;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
@@ -59,7 +58,7 @@ public final class PushDownAndCombineFilters extends OptimizerRules.OptimizerRul
             var attributes = new AttributeSet(Expressions.asAttributes(enrich.enrichFields()));
             plan = maybePushDownPastUnary(filter, enrich, attributes::contains);
         } else if (child instanceof Project) {
-            return LogicalPlanOptimizer.pushDownPastProject(filter);
+            return PushDownUtils.pushDownPastProject(filter);
         } else if (child instanceof OrderBy orderBy) {
             // swap the filter with its child
             plan = orderBy.replaceChild(filter.with(orderBy.child(), condition));

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownAndCombineLimits.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimits.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Literal;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownAndCombineOrderBy.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineOrderBy.java

@@ -5,9 +5,8 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
 import org.elasticsearch.xpack.esql.plan.logical.Project;
@@ -21,7 +20,7 @@ public final class PushDownAndCombineOrderBy extends OptimizerRules.OptimizerRul
             // combine orders
             return new OrderBy(orderBy.source(), childOrder.child(), orderBy.order());
         } else if (child instanceof Project) {
-            return LogicalPlanOptimizer.pushDownPastProject(orderBy);
+            return PushDownUtils.pushDownPastProject(orderBy);
         }
 
         return orderBy;

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownEnrich.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEnrich.java

@@ -5,15 +5,14 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Enrich;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 
 public final class PushDownEnrich extends OptimizerRules.OptimizerRule<Enrich> {
     @Override
     protected LogicalPlan rule(Enrich en) {
-        return LogicalPlanOptimizer.pushGeneratingPlanPastProjectAndOrderBy(en);
+        return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(en);
     }
 }

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownEval.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownEval.java

@@ -5,15 +5,14 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 
 public final class PushDownEval extends OptimizerRules.OptimizerRule<Eval> {
     @Override
     protected LogicalPlan rule(Eval eval) {
-        return LogicalPlanOptimizer.pushGeneratingPlanPastProjectAndOrderBy(eval);
+        return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(eval);
     }
 }

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PushDownRegexExtract.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownRegexExtract.java

@@ -5,15 +5,14 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.RegexExtract;
 
 public final class PushDownRegexExtract extends OptimizerRules.OptimizerRule<RegexExtract> {
     @Override
     protected LogicalPlan rule(RegexExtract re) {
-        return LogicalPlanOptimizer.pushGeneratingPlanPastProjectAndOrderBy(re);
+        return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(re);
     }
 }

+ 210 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownUtils.java

@@ -0,0 +1,210 @@
+/*
+ * 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.optimizer.rules.logical;
+
+import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.Expressions;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
+import org.elasticsearch.xpack.esql.expression.Order;
+import org.elasticsearch.xpack.esql.plan.GeneratingPlan;
+import org.elasticsearch.xpack.esql.plan.logical.Eval;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
+import org.elasticsearch.xpack.esql.plan.logical.Project;
+import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputExpressions;
+
+class PushDownUtils {
+    /**
+     * Pushes LogicalPlans which generate new attributes (Eval, Grok/Dissect, Enrich), past OrderBys and Projections.
+     * Although it seems arbitrary whether the OrderBy or the generating plan is executed first, this transformation ensures that OrderBys
+     * only separated by e.g. an Eval can be combined by {@link PushDownAndCombineOrderBy}.
+     * <p>
+     * E.g. {@code ... | sort a | eval x = b + 1 | sort x} becomes {@code ... | eval x = b + 1 | sort a | sort x}
+     * <p>
+     * Ordering the generating plans before the OrderBys has the advantage that it's always possible to order the plans like this.
+     * E.g., in the example above it would not be possible to put the eval after the two orderBys.
+     * <p>
+     * In case one of the generating plan's attributes would shadow the OrderBy's attributes, we alias the generated attribute first.
+     * <p>
+     * E.g. {@code ... | sort a | eval a = b + 1 | ...} becomes {@code ... | eval $$a = a | eval a = b + 1 | sort $$a | drop $$a ...}
+     * <p>
+     * In case the generating plan's attributes would shadow the Project's attributes, we rename the generated attributes in place.
+     * <p>
+     * E.g. {@code ... | rename a as z | eval a = b + 1 | ...} becomes {@code ... eval $$a = b + 1 | rename a as z, $$a as a ...}
+     */
+    public static <Plan extends UnaryPlan & GeneratingPlan<Plan>> LogicalPlan pushGeneratingPlanPastProjectAndOrderBy(Plan generatingPlan) {
+        LogicalPlan child = generatingPlan.child();
+        if (child instanceof OrderBy orderBy) {
+            Set<String> evalFieldNames = new LinkedHashSet<>(Expressions.names(generatingPlan.generatedAttributes()));
+
+            // Look for attributes in the OrderBy's expressions and create aliases with temporary names for them.
+            AttributeReplacement nonShadowedOrders = renameAttributesInExpressions(evalFieldNames, orderBy.order());
+
+            AttributeMap<Alias> aliasesForShadowedOrderByAttrs = nonShadowedOrders.replacedAttributes;
+            @SuppressWarnings("unchecked")
+            List<Order> newOrder = (List<Order>) (List<?>) nonShadowedOrders.rewrittenExpressions;
+
+            if (aliasesForShadowedOrderByAttrs.isEmpty() == false) {
+                List<Alias> newAliases = new ArrayList<>(aliasesForShadowedOrderByAttrs.values());
+
+                LogicalPlan plan = new Eval(orderBy.source(), orderBy.child(), newAliases);
+                plan = generatingPlan.replaceChild(plan);
+                plan = new OrderBy(orderBy.source(), plan, newOrder);
+                plan = new Project(generatingPlan.source(), plan, generatingPlan.output());
+
+                return plan;
+            }
+
+            return orderBy.replaceChild(generatingPlan.replaceChild(orderBy.child()));
+        } else if (child instanceof Project project) {
+            // We need to account for attribute shadowing: a rename might rely on a name generated in an Eval/Grok/Dissect/Enrich.
+            // E.g. in:
+            //
+            // Eval[[2 * x{f}#1 AS y]]
+            // \_Project[[x{f}#1, y{f}#2, y{f}#2 AS z]]
+            //
+            // Just moving the Eval down breaks z because we shadow y{f}#2.
+            // Instead, we use a different alias in the Eval, eventually renaming back to y:
+            //
+            // Project[[x{f}#1, y{f}#2 as z, $$y{r}#3 as y]]
+            // \_Eval[[2 * x{f}#1 as $$y]]
+
+            List<Attribute> generatedAttributes = generatingPlan.generatedAttributes();
+
+            @SuppressWarnings("unchecked")
+            Plan generatingPlanWithResolvedExpressions = (Plan) resolveRenamesFromProject(generatingPlan, project);
+
+            Set<String> namesReferencedInRenames = new HashSet<>();
+            for (NamedExpression ne : project.projections()) {
+                if (ne instanceof Alias as) {
+                    namesReferencedInRenames.addAll(as.child().references().names());
+                }
+            }
+            Map<String, String> renameGeneratedAttributeTo = newNamesForConflictingAttributes(
+                generatingPlan.generatedAttributes(),
+                namesReferencedInRenames
+            );
+            List<String> newNames = generatedAttributes.stream()
+                .map(attr -> renameGeneratedAttributeTo.getOrDefault(attr.name(), attr.name()))
+                .toList();
+            Plan generatingPlanWithRenamedAttributes = generatingPlanWithResolvedExpressions.withGeneratedNames(newNames);
+
+            // Put the project at the top, but include the generated attributes.
+            // Any generated attributes that had to be renamed need to be re-renamed to their original names.
+            List<NamedExpression> generatedAttributesRenamedToOriginal = new ArrayList<>(generatedAttributes.size());
+            List<Attribute> renamedGeneratedAttributes = generatingPlanWithRenamedAttributes.generatedAttributes();
+            for (int i = 0; i < generatedAttributes.size(); i++) {
+                Attribute originalAttribute = generatedAttributes.get(i);
+                Attribute renamedAttribute = renamedGeneratedAttributes.get(i);
+                if (originalAttribute.name().equals(renamedAttribute.name())) {
+                    generatedAttributesRenamedToOriginal.add(renamedAttribute);
+                } else {
+                    generatedAttributesRenamedToOriginal.add(
+                        new Alias(
+                            originalAttribute.source(),
+                            originalAttribute.name(),
+                            renamedAttribute,
+                            originalAttribute.id(),
+                            originalAttribute.synthetic()
+                        )
+                    );
+                }
+            }
+
+            Project projectWithGeneratingChild = project.replaceChild(generatingPlanWithRenamedAttributes.replaceChild(project.child()));
+            return projectWithGeneratingChild.withProjections(
+                mergeOutputExpressions(generatedAttributesRenamedToOriginal, projectWithGeneratingChild.projections())
+            );
+        }
+
+        return generatingPlan;
+    }
+
+    /**
+     * Replace attributes in the given expressions by assigning them temporary names.
+     * Returns the rewritten expressions and a map with an alias for each replaced attribute; the rewritten expressions reference
+     * these aliases.
+     */
+    private static AttributeReplacement renameAttributesInExpressions(
+        Set<String> attributeNamesToRename,
+        List<? extends Expression> expressions
+    ) {
+        AttributeMap<Alias> aliasesForReplacedAttributes = new AttributeMap<>();
+        List<Expression> rewrittenExpressions = new ArrayList<>();
+
+        for (Expression expr : expressions) {
+            rewrittenExpressions.add(expr.transformUp(Attribute.class, attr -> {
+                if (attributeNamesToRename.contains(attr.name())) {
+                    Alias renamedAttribute = aliasesForReplacedAttributes.computeIfAbsent(attr, a -> {
+                        String tempName = TemporaryNameUtils.locallyUniqueTemporaryName(a.name(), "temp_name");
+                        return new Alias(a.source(), tempName, a, null, true);
+                    });
+                    return renamedAttribute.toAttribute();
+                }
+
+                return attr;
+            }));
+        }
+
+        return new AttributeReplacement(rewrittenExpressions, aliasesForReplacedAttributes);
+    }
+
+    private static Map<String, String> newNamesForConflictingAttributes(
+        List<Attribute> potentiallyConflictingAttributes,
+        Set<String> reservedNames
+    ) {
+        if (reservedNames.isEmpty()) {
+            return Map.of();
+        }
+
+        Map<String, String> renameAttributeTo = new HashMap<>();
+        for (Attribute attr : potentiallyConflictingAttributes) {
+            String name = attr.name();
+            if (reservedNames.contains(name)) {
+                renameAttributeTo.putIfAbsent(name, TemporaryNameUtils.locallyUniqueTemporaryName(name, "temp_name"));
+            }
+        }
+
+        return renameAttributeTo;
+    }
+
+    public static Project pushDownPastProject(UnaryPlan parent) {
+        if (parent.child() instanceof Project project) {
+            UnaryPlan expressionsWithResolvedAliases = resolveRenamesFromProject(parent, project);
+
+            return project.replaceChild(expressionsWithResolvedAliases.replaceChild(project.child()));
+        } else {
+            throw new EsqlIllegalArgumentException("Expected child to be instance of Project");
+        }
+    }
+
+    private static UnaryPlan resolveRenamesFromProject(UnaryPlan plan, Project project) {
+        AttributeMap.Builder<Expression> aliasBuilder = AttributeMap.builder();
+        project.forEachExpression(Alias.class, a -> aliasBuilder.put(a.toAttribute(), a.child()));
+        var aliases = aliasBuilder.build();
+
+        return (UnaryPlan) plan.transformExpressionsOnly(ReferenceAttribute.class, r -> aliases.resolve(r, r));
+    }
+
+    private record AttributeReplacement(List<Expression> rewrittenExpressions, AttributeMap<Alias> replacedAttributes) {}
+}

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/RemoveStatsOverride.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/RemoveStatsOverride.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.xpack.esql.analysis.AnalyzerRules;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceAliasingEvalWithProject.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceAliasingEvalWithProject.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.Attribute;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceLimitAndSortAsTopN.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.plan.logical.Limit;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceLookupWithJoin.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLookupWithJoin.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.Lookup;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceOrderByExpressionWithEval.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceOrderByExpressionWithEval.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.Attribute;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceRegexMatch.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatch.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Literal;

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceStatsAggExpressionWithEval.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsAggExpressionWithEval.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.common.util.Maps;
 import org.elasticsearch.xpack.esql.core.expression.Alias;
@@ -16,7 +16,6 @@ import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
 import org.elasticsearch.xpack.esql.core.util.Holder;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
@@ -143,6 +142,6 @@ public final class ReplaceStatsAggExpressionWithEval extends OptimizerRules.Opti
     }
 
     static String syntheticName(Expression expression, Expression af, int counter) {
-        return LogicalPlanOptimizer.temporaryName(expression, af, counter);
+        return TemporaryNameUtils.temporaryName(expression, af, counter);
     }
 }

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceStatsNestedExpressionWithEval.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsNestedExpressionWithEval.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Alias;
 import org.elasticsearch.xpack.esql.core.expression.Attribute;
@@ -14,7 +14,6 @@ import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
 import org.elasticsearch.xpack.esql.core.util.Holder;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
 import org.elasticsearch.xpack.esql.expression.function.grouping.GroupingFunction;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.plan.logical.Stats;
@@ -152,6 +151,6 @@ public final class ReplaceStatsNestedExpressionWithEval extends OptimizerRules.O
     }
 
     static String syntheticName(Expression expression, AggregateFunction af, int counter) {
-        return LogicalPlanOptimizer.temporaryName(expression, af, counter);
+        return TemporaryNameUtils.temporaryName(expression, af, counter);
     }
 }

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceTrivialTypeConversions.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceTrivialTypeConversions.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
 import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SetAsOptimized.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SetAsOptimized.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.rule.Rule;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SimplifyComparisonsArithmetics.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SimplifyComparisonsArithmetics.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Literal;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SkipQueryOnEmptyMappings.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnEmptyMappings.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SkipQueryOnLimitZero.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SkipQueryOnLimitZero.java

@@ -5,9 +5,8 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Limit;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
 
@@ -16,7 +15,7 @@ public final class SkipQueryOnLimitZero extends OptimizerRules.OptimizerRule<Lim
     protected LogicalPlan rule(Limit limit) {
         if (limit.limit().foldable()) {
             if (Integer.valueOf(0).equals((limit.limit().fold()))) {
-                return LogicalPlanOptimizer.skipPlan(limit);
+                return PruneEmptyPlans.skipPlan(limit);
             }
         }
         return limit;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SplitInWithFoldableValue.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Expressions;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SubstituteSpatialSurrogates.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSpatialSurrogates.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction;
 

+ 2 - 3
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/SubstituteSurrogates.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogates.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.BlockUtils;
@@ -18,7 +18,6 @@ import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
 import org.elasticsearch.xpack.esql.expression.SurrogateExpression;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.Rate;
-import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
@@ -80,7 +79,7 @@ public final class SubstituteSurrogates extends OptimizerRules.OptimizerRule<Agg
                         var attr = aggFuncToAttr.get(af);
                         // the agg doesn't exist in the Aggregate, create an alias for it and save its attribute
                         if (attr == null) {
-                            var temporaryName = LogicalPlanOptimizer.temporaryName(af, agg, counter[0]++);
+                            var temporaryName = TemporaryNameUtils.temporaryName(af, agg, counter[0]++);
                             // create a synthetic alias (so it doesn't clash with a user defined name)
                             var newAlias = new Alias(agg.source(), temporaryName, af, null, true);
                             attr = newAlias.toAttribute();

+ 40 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TemporaryNameUtils.java

@@ -0,0 +1,40 @@
+/*
+ * 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.optimizer.rules.logical;
+
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.NameId;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
+
+class TemporaryNameUtils {
+    static int TO_STRING_LIMIT = 16;
+
+    public static String temporaryName(Expression inner, Expression outer, int suffix) {
+        String in = toString(inner);
+        String out = toString(outer);
+        return Attribute.rawTemporaryName(in, out, String.valueOf(suffix));
+    }
+
+    public static String locallyUniqueTemporaryName(String inner, String outer) {
+        return Attribute.rawTemporaryName(inner, outer, (new NameId()).toString());
+    }
+
+    static String toString(Expression ex) {
+        return ex instanceof AggregateFunction af ? af.functionName() : extractString(ex);
+    }
+
+    static String extractString(Expression ex) {
+        return ex instanceof NamedExpression ne ? ne.name() : limitToString(ex.sourceText()).replace(' ', '_');
+    }
+
+    static String limitToString(String string) {
+        return string.length() > TO_STRING_LIMIT ? string.substring(0, TO_STRING_LIMIT - 1) + ">" : string;
+    }
+}

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/TranslateMetricsAggregate.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/TranslateMetricsAggregate.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;

+ 111 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferIsNotNull.java

@@ -0,0 +1,111 @@
+/*
+ * 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.optimizer.rules.logical.local;
+
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull;
+import org.elasticsearch.xpack.esql.core.rule.Rule;
+import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
+import org.elasticsearch.xpack.esql.expression.function.scalar.nulls.Coalesce;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import static java.util.Collections.emptySet;
+
+/**
+ * Simplify IsNotNull targets by resolving the underlying expression to its root fields with unknown
+ * nullability.
+ * e.g.
+ * (x + 1) / 2 IS NOT NULL --> x IS NOT NULL AND (x+1) / 2 IS NOT NULL
+ * SUBSTRING(x, 3) > 4 IS NOT NULL --> x IS NOT NULL AND SUBSTRING(x, 3) > 4 IS NOT NULL
+ * When dealing with multiple fields, a conjunction/disjunction based on the predicate:
+ * (x + y) / 4 IS NOT NULL --> x IS NOT NULL AND y IS NOT NULL AND (x + y) / 4 IS NOT NULL
+ * This handles the case of fields nested inside functions or expressions in order to avoid:
+ * - having to evaluate the whole expression
+ * - not pushing down the filter due to expression evaluation
+ * IS NULL cannot be simplified since it leads to a disjunction which prevents the filter to be
+ * pushed down:
+ * (x + 1) IS NULL --> x IS NULL OR x + 1 IS NULL
+ * and x IS NULL cannot be pushed down
+ * <br/>
+ * 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<LogicalPlan, LogicalPlan> {
+
+    @Override
+    public LogicalPlan apply(LogicalPlan plan) {
+        // the alias map is shared across the whole plan
+        AttributeMap<Expression> aliases = new AttributeMap<>();
+        // traverse bottom-up to pick up the aliases as we go
+        plan = plan.transformUp(p -> inspectPlan(p, aliases));
+        return plan;
+    }
+
+    private LogicalPlan inspectPlan(LogicalPlan plan, AttributeMap<Expression> aliases) {
+        // inspect just this plan properties
+        plan.forEachExpression(Alias.class, a -> aliases.put(a.toAttribute(), a.child()));
+        // now go about finding isNull/isNotNull
+        LogicalPlan newPlan = plan.transformExpressionsOnlyUp(IsNotNull.class, inn -> inferNotNullable(inn, aliases));
+        return newPlan;
+    }
+
+    private Expression inferNotNullable(IsNotNull inn, AttributeMap<Expression> aliases) {
+        Expression result = inn;
+        Set<Expression> refs = resolveExpressionAsRootAttributes(inn.field(), aliases);
+        // no refs found or could not detect - return the original function
+        if (refs.size() > 0) {
+            // add IsNull for the filters along with the initial inn
+            var innList = CollectionUtils.combine(refs.stream().map(r -> (Expression) new IsNotNull(inn.source(), r)).toList(), inn);
+            result = Predicates.combineAnd(innList);
+        }
+        return result;
+    }
+
+    /**
+     * Unroll the expression to its references to get to the root fields
+     * that really matter for filtering.
+     */
+    protected Set<Expression> resolveExpressionAsRootAttributes(Expression exp, AttributeMap<Expression> aliases) {
+        Set<Expression> resolvedExpressions = new LinkedHashSet<>();
+        boolean changed = doResolve(exp, aliases, resolvedExpressions);
+        return changed ? resolvedExpressions : emptySet();
+    }
+
+    private boolean doResolve(Expression exp, AttributeMap<Expression> aliases, Set<Expression> resolvedExpressions) {
+        boolean changed = false;
+        // check if the expression can be skipped or is not nullabe
+        if (skipExpression(exp)) {
+            resolvedExpressions.add(exp);
+        } else {
+            for (Expression e : exp.references()) {
+                Expression resolved = aliases.resolve(e, e);
+                // found a root attribute, bail out
+                if (resolved instanceof Attribute a && resolved == e) {
+                    resolvedExpressions.add(a);
+                    // don't mark things as change if the original expression hasn't been broken down
+                    changed |= resolved != exp;
+                } else {
+                    // go further
+                    changed |= doResolve(resolved, aliases, resolvedExpressions);
+                }
+            }
+        }
+        return changed;
+    }
+
+    private static boolean skipExpression(Expression e) {
+        return e instanceof Coalesce;
+    }
+}

+ 75 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/InferNonNullAggConstraint.java

@@ -0,0 +1,75 @@
+/*
+ * 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.optimizer.rules.logical.local;
+
+import org.elasticsearch.common.util.set.Sets;
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
+import org.elasticsearch.xpack.esql.optimizer.LocalLogicalOptimizerContext;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules;
+import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
+import org.elasticsearch.xpack.esql.plan.logical.Filter;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.esql.stats.SearchStats;
+
+import java.util.Set;
+
+/**
+ * The vast majority of aggs ignore null entries - this rule adds a pushable filter, as it is cheap
+ * to execute, to filter this entries out to begin with.
+ * STATS x = min(a), y = sum(b)
+ * becomes
+ * | WHERE a IS NOT NULL OR b IS NOT NULL
+ * | STATS x = min(a), y = sum(b)
+ * <br>
+ * 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<Aggregate, LocalLogicalOptimizerContext> {
+    public InferNonNullAggConstraint() {
+        super(OptimizerRules.TransformDirection.UP);
+    }
+
+    @Override
+    protected LogicalPlan rule(Aggregate aggregate, LocalLogicalOptimizerContext context) {
+        // only look at aggregates with default grouping
+        if (aggregate.groupings().size() > 0) {
+            return aggregate;
+        }
+
+        SearchStats stats = context.searchStats();
+        LogicalPlan plan = aggregate;
+        var aggs = aggregate.aggregates();
+        Set<Expression> nonNullAggFields = Sets.newLinkedHashSetWithExpectedSize(aggs.size());
+        for (var agg : aggs) {
+            if (Alias.unwrap(agg) instanceof AggregateFunction af) {
+                Expression field = af.field();
+                // ignore literals (e.g. COUNT(1))
+                // make sure the field exists at the source and is indexed (not runtime)
+                if (field.foldable() == false && field instanceof FieldAttribute fa && stats.isIndexed(fa.name())) {
+                    nonNullAggFields.add(field);
+                } else {
+                    // otherwise bail out since unless disjunction needs to cover _all_ fields, things get filtered out
+                    return plan;
+                }
+            }
+        }
+
+        if (nonNullAggFields.size() > 0) {
+            Expression condition = Predicates.combineOr(
+                nonNullAggFields.stream().map(f -> (Expression) new IsNotNull(aggregate.source(), f)).toList()
+            );
+            plan = aggregate.replaceChild(new Filter(aggregate.source(), aggregate.child(), condition));
+        }
+        return plan;
+    }
+}

+ 48 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/LocalPropagateEmptyRelation.java

@@ -0,0 +1,48 @@
+/*
+ * 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.optimizer.rules.logical.local;
+
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.data.BlockFactory;
+import org.elasticsearch.compute.data.BlockUtils;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateEmptyRelation;
+import org.elasticsearch.xpack.esql.planner.AbstractPhysicalOperationProviders;
+import org.elasticsearch.xpack.esql.planner.PlannerUtils;
+
+import java.util.List;
+
+/**
+ * Local aggregation can only produce intermediate state that get wired into the global agg.
+ */
+public class LocalPropagateEmptyRelation extends PropagateEmptyRelation {
+
+    /**
+     * Local variant of the aggregation that returns the intermediate value.
+     */
+    @Override
+    protected void aggOutput(NamedExpression agg, AggregateFunction aggFunc, BlockFactory blockFactory, List<Block> blocks) {
+        List<Attribute> output = AbstractPhysicalOperationProviders.intermediateAttributes(List.of(agg), List.of());
+        for (Attribute o : output) {
+            DataType dataType = o.dataType();
+            // boolean right now is used for the internal #seen so always return true
+            var value = dataType == DataType.BOOLEAN ? true
+                // look for count(literal) with literal != null
+                : aggFunc instanceof Count count && (count.foldable() == false || count.fold() != null) ? 0L
+                // otherwise nullify
+                : null;
+            var wrapper = BlockUtils.wrapperFor(blockFactory, PlannerUtils.toElementType(dataType), 1);
+            wrapper.accept(value);
+            blocks.add(wrapper.builder().build());
+        }
+    }
+}

+ 99 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java

@@ -0,0 +1,99 @@
+/*
+ * 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.optimizer.rules.logical.local;
+
+import org.elasticsearch.common.util.Maps;
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.core.rule.ParameterizedRule;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.optimizer.LocalLogicalOptimizerContext;
+import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
+import org.elasticsearch.xpack.esql.plan.logical.Eval;
+import org.elasticsearch.xpack.esql.plan.logical.Filter;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
+import org.elasticsearch.xpack.esql.plan.logical.Project;
+import org.elasticsearch.xpack.esql.plan.logical.RegexExtract;
+import org.elasticsearch.xpack.esql.plan.logical.TopN;
+import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation;
+import org.elasticsearch.xpack.esql.stats.SearchStats;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Look for any fields used in the plan that are missing locally and replace them with null.
+ * This should minimize the plan execution, in the best scenario skipping its execution all together.
+ */
+public class ReplaceMissingFieldWithNull extends ParameterizedRule<LogicalPlan, LogicalPlan, LocalLogicalOptimizerContext> {
+
+    @Override
+    public LogicalPlan apply(LogicalPlan plan, LocalLogicalOptimizerContext localLogicalOptimizerContext) {
+        return plan.transformUp(p -> missingToNull(p, localLogicalOptimizerContext.searchStats()));
+    }
+
+    private LogicalPlan missingToNull(LogicalPlan plan, SearchStats stats) {
+        if (plan instanceof EsRelation || plan instanceof LocalRelation) {
+            return plan;
+        }
+
+        if (plan instanceof Aggregate a) {
+            // don't do anything (for now)
+            return a;
+        }
+        // keep the aliased name
+        else if (plan instanceof Project project) {
+            var projections = project.projections();
+            List<NamedExpression> newProjections = new ArrayList<>(projections.size());
+            Map<DataType, Alias> nullLiteral = Maps.newLinkedHashMapWithExpectedSize(DataType.types().size());
+
+            for (NamedExpression projection : projections) {
+                // Do not use the attribute name, this can deviate from the field name for union types.
+                if (projection instanceof FieldAttribute f && stats.exists(f.fieldName()) == false) {
+                    DataType dt = f.dataType();
+                    Alias nullAlias = nullLiteral.get(f.dataType());
+                    // save the first field as null (per datatype)
+                    if (nullAlias == null) {
+                        Alias alias = new Alias(f.source(), f.name(), Literal.of(f, null), f.id());
+                        nullLiteral.put(dt, alias);
+                        projection = alias.toAttribute();
+                    }
+                    // otherwise point to it
+                    else {
+                        // since avoids creating field copies
+                        projection = new Alias(f.source(), f.name(), nullAlias.toAttribute(), f.id());
+                    }
+                }
+
+                newProjections.add(projection);
+            }
+            // add the first found field as null
+            if (nullLiteral.size() > 0) {
+                plan = new Eval(project.source(), project.child(), new ArrayList<>(nullLiteral.values()));
+                plan = new Project(project.source(), plan, newProjections);
+            }
+        } else if (plan instanceof Eval
+            || plan instanceof Filter
+            || plan instanceof OrderBy
+            || plan instanceof RegexExtract
+            || plan instanceof TopN) {
+                plan = plan.transformExpressionsOnlyUp(
+                    FieldAttribute.class,
+                    // Do not use the attribute name, this can deviate from the field name for union types.
+                    f -> stats.exists(f.fieldName()) ? f : Literal.of(f, null)
+                );
+            }
+
+        return plan;
+    }
+}

+ 30 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceTopNWithLimitAndSort.java

@@ -0,0 +1,30 @@
+/*
+ * 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.optimizer.rules.logical.local;
+
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules;
+import org.elasticsearch.xpack.esql.plan.logical.Limit;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.esql.plan.logical.OrderBy;
+import org.elasticsearch.xpack.esql.plan.logical.TopN;
+
+import static org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection.UP;
+
+/**
+ * Break TopN back into Limit + OrderBy to allow the order rules to kick in.
+ */
+public class ReplaceTopNWithLimitAndSort extends OptimizerRules.OptimizerRule<TopN> {
+    public ReplaceTopNWithLimitAndSort() {
+        super(UP);
+    }
+
+    @Override
+    protected LogicalPlan rule(TopN plan) {
+        return new Limit(plan.source(), plan.limit(), new OrderBy(plan.source(), plan.child(), plan.order()));
+    }
+}

+ 91 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/ProjectAwayColumns.java

@@ -0,0 +1,91 @@
+/*
+ * 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.optimizer.rules.physical;
+
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
+import org.elasticsearch.xpack.esql.core.expression.Expressions;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.rule.Rule;
+import org.elasticsearch.xpack.esql.core.tree.Source;
+import org.elasticsearch.xpack.esql.core.util.Holder;
+import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
+import org.elasticsearch.xpack.esql.plan.logical.Eval;
+import org.elasticsearch.xpack.esql.plan.logical.Project;
+import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
+import static java.util.Collections.singletonList;
+
+/**
+ * Adds an explicit project to minimize the amount of attributes sent from the local plan to the coordinator.
+ * This is done here to localize the project close to the data source and simplify the upcoming field
+ * extraction.
+ */
+public class ProjectAwayColumns extends Rule<PhysicalPlan, PhysicalPlan> {
+
+    @Override
+    public PhysicalPlan apply(PhysicalPlan plan) {
+        Holder<Boolean> keepTraversing = new Holder<>(TRUE);
+        // Invariant: if we add a projection with these attributes after the current plan node, the plan remains valid
+        // and the overall output will not change.
+        Holder<AttributeSet> requiredAttributes = new Holder<>(plan.outputSet());
+
+        // This will require updating should we choose to have non-unary execution plans in the future.
+        return plan.transformDown(UnaryExec.class, currentPlanNode -> {
+            if (keepTraversing.get() == false) {
+                return currentPlanNode;
+            }
+            if (currentPlanNode instanceof ExchangeExec exec) {
+                keepTraversing.set(FALSE);
+                var child = exec.child();
+                // otherwise expect a Fragment
+                if (child instanceof FragmentExec fragmentExec) {
+                    var logicalFragment = fragmentExec.fragment();
+
+                    // no need for projection when dealing with aggs
+                    if (logicalFragment instanceof Aggregate == false) {
+                        List<Attribute> output = new ArrayList<>(requiredAttributes.get());
+                        // if all the fields are filtered out, it's only the count that matters
+                        // however until a proper fix (see https://github.com/elastic/elasticsearch/issues/98703)
+                        // add a synthetic field (so it doesn't clash with the user defined one) to return a constant
+                        // to avoid the block from being trimmed
+                        if (output.isEmpty()) {
+                            var alias = new Alias(logicalFragment.source(), "<all-fields-projected>", Literal.NULL, null, true);
+                            List<Alias> fields = singletonList(alias);
+                            logicalFragment = new Eval(logicalFragment.source(), logicalFragment, fields);
+                            output = Expressions.asAttributes(fields);
+                        }
+                        // add a logical projection (let the local replanning remove it if needed)
+                        FragmentExec newChild = new FragmentExec(
+                            Source.EMPTY,
+                            new Project(logicalFragment.source(), logicalFragment, output),
+                            fragmentExec.esFilter(),
+                            fragmentExec.estimatedRowSize(),
+                            fragmentExec.reducer()
+                        );
+                        return new ExchangeExec(exec.source(), output, exec.inBetweenAggs(), newChild);
+                    }
+                }
+            } else {
+                AttributeSet childOutput = currentPlanNode.inputSet();
+                AttributeSet addedAttributes = currentPlanNode.outputSet().subtract(childOutput);
+                requiredAttributes.set(requiredAttributes.get().subtract(addedAttributes).combine(currentPlanNode.references()));
+            }
+            return currentPlanNode;
+        });
+    }
+}

+ 160 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/EnableSpatialDistancePushdown.java

@@ -0,0 +1,160 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.geometry.Circle;
+import org.elasticsearch.geometry.Geometry;
+import org.elasticsearch.geometry.Point;
+import org.elasticsearch.geometry.utils.WellKnownBinary;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And;
+import org.elasticsearch.xpack.esql.core.tree.Source;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialDisjoint;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialIntersects;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesUtils;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.StDistance;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
+import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+
+import java.nio.ByteOrder;
+
+/**
+ * When a spatial distance predicate can be pushed down to lucene, this is done by capturing the distance within the same function.
+ * In principle this is like re-writing the predicate:
+ * <pre>WHERE ST_DISTANCE(field, TO_GEOPOINT("POINT(0 0)")) &lt;= 10000</pre>
+ * as:
+ * <pre>WHERE ST_INTERSECTS(field, TO_GEOSHAPE("CIRCLE(0,0,10000)"))</pre>
+ */
+public class EnableSpatialDistancePushdown extends PhysicalOptimizerRules.ParameterizedOptimizerRule<
+    FilterExec,
+    LocalPhysicalOptimizerContext> {
+
+    @Override
+    protected PhysicalPlan rule(FilterExec filterExec, LocalPhysicalOptimizerContext ctx) {
+        PhysicalPlan plan = filterExec;
+        if (filterExec.child() instanceof EsQueryExec) {
+            // Find and rewrite any binary comparisons that involve a distance function and a literal
+            var rewritten = filterExec.condition().transformDown(EsqlBinaryComparison.class, comparison -> {
+                ComparisonType comparisonType = ComparisonType.from(comparison.getFunctionType());
+                if (comparison.left() instanceof StDistance dist && comparison.right().foldable()) {
+                    return rewriteComparison(comparison, dist, comparison.right(), comparisonType);
+                } else if (comparison.right() instanceof StDistance dist && comparison.left().foldable()) {
+                    return rewriteComparison(comparison, dist, comparison.left(), ComparisonType.invert(comparisonType));
+                }
+                return comparison;
+            });
+            if (rewritten.equals(filterExec.condition()) == false) {
+                plan = new FilterExec(filterExec.source(), filterExec.child(), rewritten);
+            }
+        }
+
+        return plan;
+    }
+
+    private Expression rewriteComparison(
+        EsqlBinaryComparison comparison,
+        StDistance dist,
+        Expression literal,
+        ComparisonType comparisonType
+    ) {
+        Object value = literal.fold();
+        if (value instanceof Number number) {
+            if (dist.right().foldable()) {
+                return rewriteDistanceFilter(comparison, dist.left(), dist.right(), number, comparisonType);
+            } else if (dist.left().foldable()) {
+                return rewriteDistanceFilter(comparison, dist.right(), dist.left(), number, comparisonType);
+            }
+        }
+        return comparison;
+    }
+
+    private Expression rewriteDistanceFilter(
+        EsqlBinaryComparison comparison,
+        Expression spatialExp,
+        Expression literalExp,
+        Number number,
+        ComparisonType comparisonType
+    ) {
+        Geometry geometry = SpatialRelatesUtils.makeGeometryFromLiteral(literalExp);
+        if (geometry instanceof Point point) {
+            double distance = number.doubleValue();
+            Source source = comparison.source();
+            if (comparisonType.lt) {
+                distance = comparisonType.eq ? distance : Math.nextDown(distance);
+                return new SpatialIntersects(source, spatialExp, makeCircleLiteral(point, distance, literalExp));
+            } else if (comparisonType.gt) {
+                distance = comparisonType.eq ? distance : Math.nextUp(distance);
+                return new SpatialDisjoint(source, spatialExp, makeCircleLiteral(point, distance, literalExp));
+            } else if (comparisonType.eq) {
+                return new And(
+                    source,
+                    new SpatialIntersects(source, spatialExp, makeCircleLiteral(point, distance, literalExp)),
+                    new SpatialDisjoint(source, spatialExp, makeCircleLiteral(point, Math.nextDown(distance), literalExp))
+                );
+            }
+        }
+        return comparison;
+    }
+
+    private Literal makeCircleLiteral(Point point, double distance, Expression literalExpression) {
+        var circle = new Circle(point.getX(), point.getY(), distance);
+        var wkb = WellKnownBinary.toWKB(circle, ByteOrder.LITTLE_ENDIAN);
+        return new Literal(literalExpression.source(), new BytesRef(wkb), DataType.GEO_SHAPE);
+    }
+
+    /**
+     * This enum captures the key differences between various inequalities as perceived from the spatial distance function.
+     * In particular, we need to know which direction the inequality points, with lt=true meaning the left is expected to be smaller
+     * than the right. And eq=true meaning we expect euality as well. We currently don't support Equals and NotEquals, so the third
+     * field disables those.
+     */
+    enum ComparisonType {
+        LTE(true, false, true),
+        LT(true, false, false),
+        GTE(false, true, true),
+        GT(false, true, false),
+        EQ(false, false, true);
+
+        private final boolean lt;
+        private final boolean gt;
+        private final boolean eq;
+
+        ComparisonType(boolean lt, boolean gt, boolean eq) {
+            this.lt = lt;
+            this.gt = gt;
+            this.eq = eq;
+        }
+
+        static ComparisonType from(EsqlBinaryComparison.BinaryComparisonOperation op) {
+            return switch (op) {
+                case LT -> LT;
+                case LTE -> LTE;
+                case GT -> GT;
+                case GTE -> GTE;
+                default -> EQ;
+            };
+        }
+
+        static ComparisonType invert(ComparisonType comparisonType) {
+            return switch (comparisonType) {
+                case LT -> GT;
+                case LTE -> GTE;
+                case GT -> LT;
+                case GTE -> LTE;
+                default -> EQ;
+            };
+        }
+    }
+}

+ 86 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java

@@ -0,0 +1,86 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.Expressions;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
+import org.elasticsearch.xpack.esql.core.expression.TypedAttribute;
+import org.elasticsearch.xpack.esql.core.rule.Rule;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.ProjectAwayColumns;
+import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
+import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
+
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+/**
+ *
+ * Materialize the concrete fields that need to be extracted from the storage until the last possible moment.
+ * Expects the local plan to already have a projection containing the fields needed upstream.
+ * <p>
+ * 1. add the materialization right before usage inside the local plan
+ * 2. materialize any missing fields needed further up the chain
+ *
+ * @see ProjectAwayColumns
+ */
+public class InsertFieldExtraction extends Rule<PhysicalPlan, PhysicalPlan> {
+
+    @Override
+    public PhysicalPlan apply(PhysicalPlan plan) {
+        // apply the plan locally, adding a field extractor right before data is loaded
+        // by going bottom-up
+        plan = plan.transformUp(UnaryExec.class, p -> {
+            var missing = missingAttributes(p);
+
+            /*
+             * If there is a single grouping then we'll try to use ords. Either way
+             * it loads the field lazily. If we have more than one field we need to
+             * make sure the fields are loaded for the standard hash aggregator.
+             */
+            if (p instanceof AggregateExec agg && agg.groupings().size() == 1) {
+                var leaves = new LinkedList<>();
+                // TODO: this seems out of place
+                agg.aggregates().stream().filter(a -> agg.groupings().contains(a) == false).forEach(a -> leaves.addAll(a.collectLeaves()));
+                var remove = agg.groupings().stream().filter(g -> leaves.contains(g) == false).toList();
+                missing.removeAll(Expressions.references(remove));
+            }
+
+            // add extractor
+            if (missing.isEmpty() == false) {
+                // collect source attributes and add the extractor
+                var extractor = new FieldExtractExec(p.source(), p.child(), List.copyOf(missing));
+                p = p.replaceChild(extractor);
+            }
+
+            return p;
+        });
+
+        return plan;
+    }
+
+    private static Set<Attribute> missingAttributes(PhysicalPlan p) {
+        var missing = new LinkedHashSet<Attribute>();
+        var input = p.inputSet();
+
+        // collect field attributes used inside expressions
+        p.forEachExpression(TypedAttribute.class, f -> {
+            if (f instanceof FieldAttribute || f instanceof MetadataAttribute) {
+                if (input.contains(f) == false) {
+                    missing.add(f);
+                }
+            }
+        });
+        return missing;
+    }
+}

+ 37 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushDownUtils.java

@@ -0,0 +1,37 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.stats.SearchStats;
+
+import java.util.function.Predicate;
+
+class LucenePushDownUtils {
+    /**
+     * this method is supposed to be used to define if a field can be used for exact push down (eg. sort or filter).
+     * "aggregatable" is the most accurate information we can have from field_caps as of now.
+     * Pushing down operations on fields that are not aggregatable would result in an error.
+     */
+    public static boolean isAggregatable(FieldAttribute f) {
+        return f.exactAttribute().field().isAggregatable();
+    }
+
+    public static boolean hasIdenticalDelegate(FieldAttribute attr, SearchStats stats) {
+        return stats.hasIdenticalDelegate(attr.name());
+    }
+
+    public static boolean isPushableFieldAttribute(Expression exp, Predicate<FieldAttribute> hasIdenticalDelegate) {
+        if (exp instanceof FieldAttribute fa && fa.getExactInfo().hasExact() && isAggregatable(fa)) {
+            return fa.dataType() != DataType.TEXT || hasIdenticalDelegate.test(fa);
+        }
+        return false;
+    }
+}

+ 218 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushFiltersToSource.java

@@ -0,0 +1,218 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.Expressions;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
+import org.elasticsearch.xpack.esql.core.expression.function.scalar.UnaryScalarFunction;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Range;
+import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate;
+import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.StringQueryPredicate;
+import org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic;
+import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not;
+import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull;
+import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNull;
+import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison;
+import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RegexMatch;
+import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardLike;
+import org.elasticsearch.xpack.esql.core.querydsl.query.Query;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
+import org.elasticsearch.xpack.esql.core.util.Queries;
+import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.InsensitiveBinaryComparison;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals;
+import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.planner.PlannerUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Predicate;
+
+import static java.util.Arrays.asList;
+import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.splitAnd;
+
+public class PushFiltersToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<FilterExec, LocalPhysicalOptimizerContext> {
+
+    @Override
+    protected PhysicalPlan rule(FilterExec filterExec, LocalPhysicalOptimizerContext ctx) {
+        PhysicalPlan plan = filterExec;
+        if (filterExec.child() instanceof EsQueryExec queryExec) {
+            List<Expression> pushable = new ArrayList<>();
+            List<Expression> nonPushable = new ArrayList<>();
+            for (Expression exp : splitAnd(filterExec.condition())) {
+                (canPushToSource(exp, x -> LucenePushDownUtils.hasIdenticalDelegate(x, ctx.searchStats())) ? pushable : nonPushable).add(
+                    exp
+                );
+            }
+            // Combine GT, GTE, LT and LTE in pushable to Range if possible
+            List<Expression> newPushable = combineEligiblePushableToRange(pushable);
+            if (newPushable.size() > 0) { // update the executable with pushable conditions
+                Query queryDSL = PlannerUtils.TRANSLATOR_HANDLER.asQuery(Predicates.combineAnd(newPushable));
+                QueryBuilder planQuery = queryDSL.asBuilder();
+                var query = Queries.combine(Queries.Clause.FILTER, asList(queryExec.query(), planQuery));
+                queryExec = new EsQueryExec(
+                    queryExec.source(),
+                    queryExec.index(),
+                    queryExec.indexMode(),
+                    queryExec.output(),
+                    query,
+                    queryExec.limit(),
+                    queryExec.sorts(),
+                    queryExec.estimatedRowSize()
+                );
+                if (nonPushable.size() > 0) { // update filter with remaining non-pushable conditions
+                    plan = new FilterExec(filterExec.source(), queryExec, Predicates.combineAnd(nonPushable));
+                } else { // prune Filter entirely
+                    plan = queryExec;
+                }
+            } // else: nothing changes
+        }
+
+        return plan;
+    }
+
+    private static List<Expression> combineEligiblePushableToRange(List<Expression> pushable) {
+        List<EsqlBinaryComparison> bcs = new ArrayList<>();
+        List<Range> ranges = new ArrayList<>();
+        List<Expression> others = new ArrayList<>();
+        boolean changed = false;
+
+        pushable.forEach(e -> {
+            if (e instanceof GreaterThan || e instanceof GreaterThanOrEqual || e instanceof LessThan || e instanceof LessThanOrEqual) {
+                if (((EsqlBinaryComparison) e).right().foldable()) {
+                    bcs.add((EsqlBinaryComparison) e);
+                } else {
+                    others.add(e);
+                }
+            } else {
+                others.add(e);
+            }
+        });
+
+        for (int i = 0, step = 1; i < bcs.size() - 1; i += step, step = 1) {
+            BinaryComparison main = bcs.get(i);
+            for (int j = i + 1; j < bcs.size(); j++) {
+                BinaryComparison other = bcs.get(j);
+                if (main.left().semanticEquals(other.left())) {
+                    // >/>= AND </<=
+                    if ((main instanceof GreaterThan || main instanceof GreaterThanOrEqual)
+                        && (other instanceof LessThan || other instanceof LessThanOrEqual)) {
+                        bcs.remove(j);
+                        bcs.remove(i);
+
+                        ranges.add(
+                            new Range(
+                                main.source(),
+                                main.left(),
+                                main.right(),
+                                main instanceof GreaterThanOrEqual,
+                                other.right(),
+                                other instanceof LessThanOrEqual,
+                                main.zoneId()
+                            )
+                        );
+
+                        changed = true;
+                        step = 0;
+                        break;
+                    }
+                    // </<= AND >/>=
+                    else if ((other instanceof GreaterThan || other instanceof GreaterThanOrEqual)
+                        && (main instanceof LessThan || main instanceof LessThanOrEqual)) {
+                            bcs.remove(j);
+                            bcs.remove(i);
+
+                            ranges.add(
+                                new Range(
+                                    main.source(),
+                                    main.left(),
+                                    other.right(),
+                                    other instanceof GreaterThanOrEqual,
+                                    main.right(),
+                                    main instanceof LessThanOrEqual,
+                                    main.zoneId()
+                                )
+                            );
+
+                            changed = true;
+                            step = 0;
+                            break;
+                        }
+                }
+            }
+        }
+        return changed ? CollectionUtils.combine(others, bcs, ranges) : pushable;
+    }
+
+    public static boolean canPushToSource(Expression exp, Predicate<FieldAttribute> hasIdenticalDelegate) {
+        if (exp instanceof BinaryComparison bc) {
+            return isAttributePushable(bc.left(), bc, hasIdenticalDelegate) && bc.right().foldable();
+        } else if (exp instanceof InsensitiveBinaryComparison bc) {
+            return isAttributePushable(bc.left(), bc, hasIdenticalDelegate) && bc.right().foldable();
+        } else if (exp instanceof BinaryLogic bl) {
+            return canPushToSource(bl.left(), hasIdenticalDelegate) && canPushToSource(bl.right(), hasIdenticalDelegate);
+        } else if (exp instanceof In in) {
+            return isAttributePushable(in.value(), null, hasIdenticalDelegate) && Expressions.foldable(in.list());
+        } else if (exp instanceof Not not) {
+            return canPushToSource(not.field(), hasIdenticalDelegate);
+        } else if (exp instanceof UnaryScalarFunction usf) {
+            if (usf instanceof RegexMatch<?> || usf instanceof IsNull || usf instanceof IsNotNull) {
+                if (usf instanceof IsNull || usf instanceof IsNotNull) {
+                    if (usf.field() instanceof FieldAttribute fa && fa.dataType().equals(DataType.TEXT)) {
+                        return true;
+                    }
+                }
+                return isAttributePushable(usf.field(), usf, hasIdenticalDelegate);
+            }
+        } else if (exp instanceof CIDRMatch cidrMatch) {
+            return isAttributePushable(cidrMatch.ipField(), cidrMatch, hasIdenticalDelegate) && Expressions.foldable(cidrMatch.matches());
+        } else if (exp instanceof SpatialRelatesFunction bc) {
+            return bc.canPushToSource(LucenePushDownUtils::isAggregatable);
+        } else if (exp instanceof MatchQueryPredicate mqp) {
+            return mqp.field() instanceof FieldAttribute && DataType.isString(mqp.field().dataType());
+        } else if (exp instanceof StringQueryPredicate) {
+            return true;
+        }
+        return false;
+    }
+
+    private static boolean isAttributePushable(
+        Expression expression,
+        Expression operation,
+        Predicate<FieldAttribute> hasIdenticalDelegate
+    ) {
+        if (LucenePushDownUtils.isPushableFieldAttribute(expression, hasIdenticalDelegate)) {
+            return true;
+        }
+        if (expression instanceof MetadataAttribute ma && ma.searchable()) {
+            return operation == null
+                // no range or regex queries supported with metadata fields
+                || operation instanceof Equals
+                || operation instanceof NotEquals
+                || operation instanceof WildcardLike;
+        }
+        return false;
+    }
+}

+ 28 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushLimitToSource.java

@@ -0,0 +1,28 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
+import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+
+public class PushLimitToSource extends PhysicalOptimizerRules.OptimizerRule<LimitExec> {
+    @Override
+    protected PhysicalPlan rule(LimitExec limitExec) {
+        PhysicalPlan plan = limitExec;
+        PhysicalPlan child = limitExec.child();
+        if (child instanceof EsQueryExec queryExec) { // add_task_parallelism_above_query: false
+            plan = queryExec.withLimit(limitExec.limit());
+        } else if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
+            plan = exchangeExec.replaceChild(queryExec.withLimit(limitExec.limit()));
+        }
+        return plan;
+    }
+}

+ 124 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java

@@ -0,0 +1,124 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.core.Tuple;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.AttributeMap;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.core.util.StringUtils;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
+import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.planner.AbstractPhysicalOperationProviders;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+import static org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec.StatsType.COUNT;
+
+/**
+ * Looks for the case where certain stats exist right before the query and thus can be pushed down.
+ */
+public class PushStatsToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<AggregateExec, LocalPhysicalOptimizerContext> {
+
+    @Override
+    protected PhysicalPlan rule(AggregateExec aggregateExec, LocalPhysicalOptimizerContext context) {
+        PhysicalPlan plan = aggregateExec;
+        if (aggregateExec.child() instanceof EsQueryExec queryExec) {
+            var tuple = pushableStats(aggregateExec, context);
+
+            // for the moment support pushing count just for one field
+            List<EsStatsQueryExec.Stat> stats = tuple.v2();
+            if (stats.size() > 1) {
+                return aggregateExec;
+            }
+
+            // TODO: handle case where some aggs cannot be pushed down by breaking the aggs into two sources (regular + stats) + union
+            // use the stats since the attributes are larger in size (due to seen)
+            if (tuple.v2().size() == aggregateExec.aggregates().size()) {
+                plan = new EsStatsQueryExec(
+                    aggregateExec.source(),
+                    queryExec.index(),
+                    queryExec.query(),
+                    queryExec.limit(),
+                    tuple.v1(),
+                    tuple.v2()
+                );
+            }
+        }
+        return plan;
+    }
+
+    private Tuple<List<Attribute>, List<EsStatsQueryExec.Stat>> pushableStats(
+        AggregateExec aggregate,
+        LocalPhysicalOptimizerContext context
+    ) {
+        AttributeMap<EsStatsQueryExec.Stat> stats = new AttributeMap<>();
+        Tuple<List<Attribute>, List<EsStatsQueryExec.Stat>> tuple = new Tuple<>(new ArrayList<>(), new ArrayList<>());
+
+        if (aggregate.groupings().isEmpty()) {
+            for (NamedExpression agg : aggregate.aggregates()) {
+                var attribute = agg.toAttribute();
+                EsStatsQueryExec.Stat stat = stats.computeIfAbsent(attribute, a -> {
+                    if (agg instanceof Alias as) {
+                        Expression child = as.child();
+                        if (child instanceof Count count) {
+                            var target = count.field();
+                            String fieldName = null;
+                            QueryBuilder query = null;
+                            // TODO: add count over field (has to be field attribute)
+                            if (target.foldable()) {
+                                fieldName = StringUtils.WILDCARD;
+                            }
+                            // check if regular field
+                            else {
+                                if (target instanceof FieldAttribute fa) {
+                                    var fName = fa.name();
+                                    if (context.searchStats().isSingleValue(fName)) {
+                                        fieldName = fa.name();
+                                        query = QueryBuilders.existsQuery(fieldName);
+                                    }
+                                }
+                            }
+                            if (fieldName != null) {
+                                return new EsStatsQueryExec.Stat(fieldName, COUNT, query);
+                            }
+                        }
+                    }
+                    return null;
+                });
+                if (stat != null) {
+                    List<Attribute> intermediateAttributes = AbstractPhysicalOperationProviders.intermediateAttributes(
+                        singletonList(agg),
+                        emptyList()
+                    );
+                    // TODO: the attributes have been recreated here; they will have wrong name ids, and the dependency check will
+                    // probably fail when we fix https://github.com/elastic/elasticsearch/issues/105436.
+                    // We may need to refactor AbstractPhysicalOperationProviders.intermediateAttributes so it doesn't return just
+                    // a list of attributes, but a mapping from the logical to the physical attributes.
+                    tuple.v1().addAll(intermediateAttributes);
+                    tuple.v2().add(stat);
+                }
+            }
+        }
+
+        return tuple;
+    }
+}

+ 64 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java

@@ -0,0 +1,64 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.expression.Order;
+import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.plan.physical.TopNExec;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Predicate;
+
+public class PushTopNToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule<TopNExec, LocalPhysicalOptimizerContext> {
+    @Override
+    protected PhysicalPlan rule(TopNExec topNExec, LocalPhysicalOptimizerContext ctx) {
+        PhysicalPlan plan = topNExec;
+        PhysicalPlan child = topNExec.child();
+        if (canPushSorts(child)
+            && canPushDownOrders(topNExec.order(), x -> LucenePushDownUtils.hasIdenticalDelegate(x, ctx.searchStats()))) {
+            var sorts = buildFieldSorts(topNExec.order());
+            var limit = topNExec.limit();
+
+            if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
+                plan = exchangeExec.replaceChild(queryExec.withSorts(sorts).withLimit(limit));
+            } else {
+                plan = ((EsQueryExec) child).withSorts(sorts).withLimit(limit);
+            }
+        }
+        return plan;
+    }
+
+    private static boolean canPushSorts(PhysicalPlan plan) {
+        if (plan instanceof EsQueryExec queryExec) {
+            return queryExec.canPushSorts();
+        }
+        if (plan instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) {
+            return queryExec.canPushSorts();
+        }
+        return false;
+    }
+
+    private boolean canPushDownOrders(List<Order> orders, Predicate<FieldAttribute> hasIdenticalDelegate) {
+        // allow only exact FieldAttributes (no expressions) for sorting
+        return orders.stream().allMatch(o -> LucenePushDownUtils.isPushableFieldAttribute(o.child(), hasIdenticalDelegate));
+    }
+
+    private List<EsQueryExec.FieldSort> buildFieldSorts(List<Order> orders) {
+        List<EsQueryExec.FieldSort> sorts = new ArrayList<>(orders.size());
+        for (Order o : orders) {
+            sorts.add(new EsQueryExec.FieldSort(((FieldAttribute) o.child()).exactAttribute(), o.direction(), o.nullsPosition()));
+        }
+        return sorts;
+    }
+}

+ 50 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java

@@ -0,0 +1,50 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.optimizer.rules.physical.local;
+
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+
+import java.util.List;
+
+import static org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection.UP;
+
+public class ReplaceSourceAttributes extends PhysicalOptimizerRules.OptimizerRule<EsSourceExec> {
+
+    public ReplaceSourceAttributes() {
+        super(UP);
+    }
+
+    @Override
+    protected PhysicalPlan rule(EsSourceExec plan) {
+        var docId = new FieldAttribute(plan.source(), EsQueryExec.DOC_ID_FIELD.getName(), EsQueryExec.DOC_ID_FIELD);
+        if (plan.indexMode() == IndexMode.TIME_SERIES) {
+            Attribute tsid = null, timestamp = null;
+            for (Attribute attr : plan.output()) {
+                String name = attr.name();
+                if (name.equals(MetadataAttribute.TSID_FIELD)) {
+                    tsid = attr;
+                } else if (name.equals(MetadataAttribute.TIMESTAMP_FIELD)) {
+                    timestamp = attr;
+                }
+            }
+            if (tsid == null || timestamp == null) {
+                throw new IllegalStateException("_tsid or @timestamp are missing from the time-series source");
+            }
+            return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId, tsid, timestamp), plan.query());
+        } else {
+            return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId), plan.query());
+        }
+    }
+}

+ 131 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/SpatialDocValuesExtraction.java

@@ -0,0 +1,131 @@
+/*
+ * 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.optimizer.rules.physical.local;
+
+import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.SpatialAggregateFunction;
+import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction;
+import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules;
+import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
+import org.elasticsearch.xpack.esql.plan.physical.EvalExec;
+import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class SpatialDocValuesExtraction extends PhysicalOptimizerRules.OptimizerRule<AggregateExec> {
+    @Override
+    protected PhysicalPlan rule(AggregateExec aggregate) {
+        var foundAttributes = new HashSet<FieldAttribute>();
+
+        PhysicalPlan plan = aggregate.transformDown(UnaryExec.class, exec -> {
+            if (exec instanceof AggregateExec agg) {
+                var orderedAggregates = new ArrayList<NamedExpression>();
+                var changedAggregates = false;
+                for (NamedExpression aggExpr : agg.aggregates()) {
+                    if (aggExpr instanceof Alias as && as.child() instanceof SpatialAggregateFunction af) {
+                        if (af.field() instanceof FieldAttribute fieldAttribute
+                            && allowedForDocValues(fieldAttribute, agg, foundAttributes)) {
+                            // We need to both mark the field to load differently, and change the spatial function to know to use it
+                            foundAttributes.add(fieldAttribute);
+                            changedAggregates = true;
+                            orderedAggregates.add(as.replaceChild(af.withDocValues()));
+                        } else {
+                            orderedAggregates.add(aggExpr);
+                        }
+                    } else {
+                        orderedAggregates.add(aggExpr);
+                    }
+                }
+                if (changedAggregates) {
+                    exec = new AggregateExec(
+                        agg.source(),
+                        agg.child(),
+                        agg.groupings(),
+                        orderedAggregates,
+                        agg.getMode(),
+                        agg.intermediateAttributes(),
+                        agg.estimatedRowSize()
+                    );
+                }
+            }
+            if (exec instanceof EvalExec evalExec) {
+                List<Alias> fields = evalExec.fields();
+                List<Alias> changed = fields.stream()
+                    .map(
+                        f -> (Alias) f.transformDown(
+                            SpatialRelatesFunction.class,
+                            spatialRelatesFunction -> (spatialRelatesFunction.hasFieldAttribute(foundAttributes))
+                                ? spatialRelatesFunction.withDocValues(foundAttributes)
+                                : spatialRelatesFunction
+                        )
+                    )
+                    .toList();
+                if (changed.equals(fields) == false) {
+                    exec = new EvalExec(exec.source(), exec.child(), changed);
+                }
+            }
+            if (exec instanceof FilterExec filterExec) {
+                // Note that ST_CENTROID does not support shapes, but SpatialRelatesFunction does, so when we extend the centroid
+                // to support shapes, we need to consider loading shape doc-values for both centroid and relates (ST_INTERSECTS)
+                var condition = filterExec.condition()
+                    .transformDown(
+                        SpatialRelatesFunction.class,
+                        spatialRelatesFunction -> (spatialRelatesFunction.hasFieldAttribute(foundAttributes))
+                            ? spatialRelatesFunction.withDocValues(foundAttributes)
+                            : spatialRelatesFunction
+                    );
+                if (filterExec.condition().equals(condition) == false) {
+                    exec = new FilterExec(filterExec.source(), filterExec.child(), condition);
+                }
+            }
+            if (exec instanceof FieldExtractExec fieldExtractExec) {
+                // Tell the field extractor that it should extract the field from doc-values instead of source values
+                var attributesToExtract = fieldExtractExec.attributesToExtract();
+                Set<Attribute> docValuesAttributes = new HashSet<>();
+                for (Attribute found : foundAttributes) {
+                    if (attributesToExtract.contains(found)) {
+                        docValuesAttributes.add(found);
+                    }
+                }
+                if (docValuesAttributes.size() > 0) {
+                    exec = new FieldExtractExec(exec.source(), exec.child(), attributesToExtract, docValuesAttributes);
+                }
+            }
+            return exec;
+        });
+        return plan;
+    }
+
+    /**
+     * This function disallows the use of more than one field for doc-values extraction in the same spatial relation function.
+     * This is because comparing two doc-values fields is not supported in the current implementation.
+     */
+    private boolean allowedForDocValues(FieldAttribute fieldAttribute, AggregateExec agg, Set<FieldAttribute> foundAttributes) {
+        var candidateDocValuesAttributes = new HashSet<>(foundAttributes);
+        candidateDocValuesAttributes.add(fieldAttribute);
+        var spatialRelatesAttributes = new HashSet<FieldAttribute>();
+        agg.forEachExpressionDown(SpatialRelatesFunction.class, relatesFunction -> {
+            candidateDocValuesAttributes.forEach(candidate -> {
+                if (relatesFunction.hasFieldAttribute(Set.of(candidate))) {
+                    spatialRelatesAttributes.add(candidate);
+                }
+            });
+        });
+        // Disallow more than one spatial field to be extracted using doc-values (for now)
+        return spatialRelatesAttributes.size() < 2;
+    }
+}

+ 3 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java

@@ -62,11 +62,12 @@ import static java.util.Arrays.asList;
 import static org.elasticsearch.index.mapper.MappedFieldType.FieldExtractPreference.DOC_VALUES;
 import static org.elasticsearch.index.mapper.MappedFieldType.FieldExtractPreference.NONE;
 import static org.elasticsearch.xpack.esql.core.util.Queries.Clause.FILTER;
-import static org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer.PushFiltersToSource.canPushToSource;
-import static org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer.TRANSLATOR_HANDLER;
+import static org.elasticsearch.xpack.esql.optimizer.rules.physical.local.PushFiltersToSource.canPushToSource;
 
 public class PlannerUtils {
 
+    public static final EsqlTranslatorHandler TRANSLATOR_HANDLER = new EsqlTranslatorHandler();
+
     public static Tuple<PhysicalPlan, PhysicalPlan> breakPlanBetweenCoordinatorAndDataNode(PhysicalPlan plan, Configuration config) {
         var dataNodePlan = new Holder<PhysicalPlan>();
 

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java

@@ -25,7 +25,7 @@ import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.util.NumericUtils;
 import org.elasticsearch.xpack.esql.expression.SurrogateExpression;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
-import org.elasticsearch.xpack.esql.optimizer.FoldNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.FoldNull;
 import org.elasticsearch.xpack.esql.planner.PlannerUtils;
 import org.elasticsearch.xpack.esql.planner.ToAggregator;
 

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java

@@ -61,7 +61,7 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals;
-import org.elasticsearch.xpack.esql.optimizer.FoldNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.FoldNull;
 import org.elasticsearch.xpack.esql.parser.ExpressionBuilder;
 import org.elasticsearch.xpack.esql.planner.Layout;
 import org.elasticsearch.xpack.esql.planner.PlannerUtils;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java

@@ -22,7 +22,7 @@ import org.elasticsearch.indices.CrankyCircuitBreakerService;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.util.NumericUtils;
-import org.elasticsearch.xpack.esql.optimizer.FoldNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.FoldNull;
 import org.elasticsearch.xpack.esql.planner.PlannerUtils;
 import org.hamcrest.Matcher;
 

+ 0 - 17
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/FoldNull.java

@@ -1,17 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.esql.optimizer;
-
-import org.elasticsearch.xpack.esql.core.expression.Expression;
-
-public class FoldNull extends org.elasticsearch.xpack.esql.optimizer.rules.FoldNull {
-    @Override
-    public Expression rule(Expression e) {
-        return super.rule(e);
-    }
-}

+ 6 - 5
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java

@@ -34,6 +34,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.string.StartsWith
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 import org.elasticsearch.xpack.esql.index.EsIndex;
 import org.elasticsearch.xpack.esql.index.IndexResolution;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.InferIsNotNull;
 import org.elasticsearch.xpack.esql.parser.EsqlParser;
 import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
 import org.elasticsearch.xpack.esql.plan.logical.Eval;
@@ -467,7 +468,7 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
         Expression inn = isNotNull(fieldA);
         Filter f = new Filter(EMPTY, relation, inn);
 
-        assertEquals(f, new LocalLogicalPlanOptimizer.InferIsNotNull().apply(f));
+        assertEquals(f, new InferIsNotNull().apply(f));
     }
 
     public void testIsNotNullOnOperatorWithOneField() {
@@ -477,7 +478,7 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
         Filter f = new Filter(EMPTY, relation, inn);
         Filter expected = new Filter(EMPTY, relation, new And(EMPTY, isNotNull(fieldA), inn));
 
-        assertEquals(expected, new LocalLogicalPlanOptimizer.InferIsNotNull().apply(f));
+        assertEquals(expected, new InferIsNotNull().apply(f));
     }
 
     public void testIsNotNullOnOperatorWithTwoFields() {
@@ -488,7 +489,7 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
         Filter f = new Filter(EMPTY, relation, inn);
         Filter expected = new Filter(EMPTY, relation, new And(EMPTY, new And(EMPTY, isNotNull(fieldA), isNotNull(fieldB)), inn));
 
-        assertEquals(expected, new LocalLogicalPlanOptimizer.InferIsNotNull().apply(f));
+        assertEquals(expected, new InferIsNotNull().apply(f));
     }
 
     public void testIsNotNullOnFunctionWithOneField() {
@@ -500,7 +501,7 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
         Filter f = new Filter(EMPTY, relation, inn);
         Filter expected = new Filter(EMPTY, relation, new And(EMPTY, isNotNull(fieldA), inn));
 
-        assertEquals(expected, new LocalLogicalPlanOptimizer.InferIsNotNull().apply(f));
+        assertEquals(expected, new InferIsNotNull().apply(f));
     }
 
     public void testIsNotNullOnFunctionWithTwoFields() {
@@ -512,7 +513,7 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
         Filter f = new Filter(EMPTY, relation, inn);
         Filter expected = new Filter(EMPTY, relation, new And(EMPTY, new And(EMPTY, isNotNull(fieldA), isNotNull(fieldB)), inn));
 
-        assertEquals(expected, new LocalLogicalPlanOptimizer.InferIsNotNull().apply(f));
+        assertEquals(expected, new InferIsNotNull().apply(f));
     }
 
     private IsNotNull isNotNull(Expression field) {

+ 10 - 8
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java

@@ -109,14 +109,16 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Les
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals;
 import org.elasticsearch.xpack.esql.index.EsIndex;
 import org.elasticsearch.xpack.esql.index.IndexResolution;
-import org.elasticsearch.xpack.esql.optimizer.rules.LiteralsOnTheRight;
-import org.elasticsearch.xpack.esql.optimizer.rules.OptimizerRules;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownAndCombineFilters;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownAndCombineLimits;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownEnrich;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownEval;
-import org.elasticsearch.xpack.esql.optimizer.rules.PushDownRegexExtract;
-import org.elasticsearch.xpack.esql.optimizer.rules.SplitInWithFoldableValue;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.FoldNull;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.LiteralsOnTheRight;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateNullable;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownAndCombineFilters;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownAndCombineLimits;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownEnrich;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownEval;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.PushDownRegexExtract;
+import org.elasticsearch.xpack.esql.optimizer.rules.logical.SplitInWithFoldableValue;
 import org.elasticsearch.xpack.esql.parser.EsqlParser;
 import org.elasticsearch.xpack.esql.parser.ParsingException;
 import org.elasticsearch.xpack.esql.plan.GeneratingPlan;

+ 2 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java

@@ -73,6 +73,7 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Les
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual;
 import org.elasticsearch.xpack.esql.index.EsIndex;
 import org.elasticsearch.xpack.esql.index.IndexResolution;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.ProjectAwayColumns;
 import org.elasticsearch.xpack.esql.parser.EsqlParser;
 import org.elasticsearch.xpack.esql.parser.ParsingException;
 import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
@@ -2099,7 +2100,7 @@ public class PhysicalPlanOptimizerTests extends ESTestCase {
     }
 
     public void testProjectAwayColumns() {
-        var rule = new PhysicalPlanOptimizer.ProjectAwayColumns();
+        var rule = new ProjectAwayColumns();
 
         // FROM test | limit 10000
         //

+ 0 - 18
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PropagateNullable.java

@@ -1,18 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.esql.optimizer;
-
-import org.elasticsearch.xpack.esql.core.expression.Expression;
-import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And;
-
-public class PropagateNullable extends org.elasticsearch.xpack.esql.optimizer.rules.PropagateNullable {
-    @Override
-    public Expression rule(And and) {
-        return super.rule(and);
-    }
-}

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsEliminationTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanFunctionEqualsEliminationTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanSimplificationTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/BooleanSimplificationTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineBinaryComparisonsTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineBinaryComparisonsTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.EsqlTestUtils;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctionsTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineDisjunctionsTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/ConstantFoldingTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Alias;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/FoldNullTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNullTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/LiteralsOnTheRightTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/LiteralsOnTheRightTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Alias;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEqualsTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEqualsTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateNullableTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateNullableTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/ReplaceRegexMatchTests.java → x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java

@@ -5,7 +5,7 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.optimizer.rules;
+package org.elasticsearch.xpack.esql.optimizer.rules.logical;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.esql.core.expression.Expression;