From f29cf83e1d69eeee92f9c4acda6e1b5b4eaa0c03 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Sun, 1 Oct 2023 13:05:51 -0700 Subject: [PATCH] ESQL: Pushdown count(field) to Lucene Use the LuceneCountOperator also for ungrouped count(field) queries Fix #99840 --- .../src/main/resources/stats.csv-spec | 65 +++++ .../optimizer/LocalPhysicalPlanOptimizer.java | 17 +- .../esql/plan/physical/EsStatsQueryExec.java | 16 +- .../esql/plan/physical/FragmentExec.java | 6 +- .../esql/planner/LocalExecutionPlanner.java | 16 +- .../LocalPhysicalPlanOptimizerTests.java | 259 ++++++++++++++++++ .../optimizer/PhysicalPlanOptimizerTests.java | 3 +- .../TestLocalPhysicalPlanOptimizer.java | 9 +- .../esql/tree/EsqlNodeSubclassTests.java | 2 +- 9 files changed, 377 insertions(+), 16 deletions(-) create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec index 6405c082cf784..f1d93ca7391cf 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec @@ -578,3 +578,68 @@ F M null ; + +countFieldNoGrouping +from employees | where emp_no < 10050 | stats c = count(salary); + +c:l +49 +; + +countFieldWithRenamingNoGrouping +from employees | rename emp_no as e, salary as s | where e < 10050 | stats c = count(s); + +c:l +49 +; + + +countFieldWithAliasNoGrouping +from employees | eval s = salary | rename s as sr | eval hidden_s = sr | rename emp_no as e | where e < 10050 | stats c = count(hidden_s); + +c:l +49 +; + +countFieldWithGrouping +from employees | rename languages as l | where emp_no < 10050 | stats c = count(emp_no) by l | sort l; + +c:l | l:i +9 | 1 +7 | 2 +6 | 3 +9 | 4 +8 | 5 +10 | null +; + +countFieldWithAliasWithGrouping +from employees | rename languages as l | eval e = emp_no | where emp_no < 10050 | stats c = count(e) by l | sort l; + +c:l | l:i +9 | 1 +7 | 2 +6 | 3 +9 | 4 +8 | 5 +10 | null +; + +countEvalExpNoGrouping +from employees | eval e = case(emp_no < 10050, emp_no, null) | stats c = count(e); + +c:l +49 +; + +countEvalExpWithGrouping +from employees | rename languages as l | eval e = case(emp_no < 10050, emp_no, null) | stats c = count(e) by l | sort l; + +c:l | l:i +9 | 1 +7 | 2 +6 | 3 +9 | 4 +8 | 5 +10 | null +; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java index eb4b11f5e2e34..3e9b83749bada 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java @@ -9,6 +9,7 @@ import org.elasticsearch.core.Tuple; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Equals; import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.NotEquals; @@ -346,7 +347,7 @@ protected PhysicalPlan rule(AggregateExec aggregateExec) { private Tuple, List> pushableStats(AggregateExec aggregate) { AttributeMap stats = new AttributeMap<>(); - Tuple, List> tuple = new Tuple<>(new ArrayList(), new ArrayList()); + Tuple, List> tuple = new Tuple<>(new ArrayList<>(), new ArrayList<>()); if (aggregate.groupings().isEmpty()) { for (NamedExpression agg : aggregate.aggregates()) { @@ -356,9 +357,21 @@ private Tuple, List> pushableStats(AggregateExec aggregate 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()) { - return new Stat(StringUtils.WILDCARD, COUNT); + fieldName = StringUtils.WILDCARD; + } + // check if regular field + else { + if (target instanceof FieldAttribute fa) { + fieldName = fa.name(); + query = QueryBuilders.existsQuery(fieldName); + } + } + if (fieldName != null) { + return new Stat(fieldName, COUNT, query); } } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsStatsQueryExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsStatsQueryExec.java index 8e65e66e3045f..fb62191395a61 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsStatsQueryExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsStatsQueryExec.java @@ -15,10 +15,13 @@ import org.elasticsearch.xpack.ql.tree.NodeInfo; import org.elasticsearch.xpack.ql.tree.NodeUtils; import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.util.Queries; import java.util.List; import java.util.Objects; +import static java.util.Arrays.asList; + /** * Specialized query class for retrieving statistics about the underlying data and not the actual documents. * For that see {@link EsQueryExec} @@ -29,10 +32,15 @@ public enum StatsType { COUNT, MIN, MAX, - EXISTS; + EXISTS } - public record Stat(String name, StatsType type) {}; + public record Stat(String name, StatsType type, QueryBuilder query) { + + public QueryBuilder filter(QueryBuilder sourceQuery) { + return query == null ? sourceQuery : Queries.combine(Queries.Clause.FILTER, asList(sourceQuery, query)); + } + } private final EsIndex index; private final QueryBuilder query; @@ -69,6 +77,10 @@ public QueryBuilder query() { return query; } + public List stats() { + return stats; + } + @Override public List output() { return attrs; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java index addc66c106abd..43fccf4cf62da 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java @@ -25,13 +25,13 @@ public class FragmentExec extends LeafExec implements EstimatesRowSize { * Estimate of the number of bytes that'll be loaded per position before * the stream of pages is consumed. */ - private final Integer estimatedRowSize; + private final int estimatedRowSize; public FragmentExec(LogicalPlan fragment) { - this(fragment.source(), fragment, null, null); + this(fragment.source(), fragment, null, 0); } - public FragmentExec(Source source, LogicalPlan fragment, QueryBuilder esFilter, Integer estimatedRowSize) { + public FragmentExec(Source source, LogicalPlan fragment, QueryBuilder esFilter, int estimatedRowSize) { super(source); this.fragment = fragment; this.esFilter = esFilter; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java index 156b93e1551c4..f3601873543e3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java @@ -88,7 +88,6 @@ import org.elasticsearch.xpack.ql.util.Holder; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -99,6 +98,7 @@ import java.util.function.Function; import java.util.stream.Stream; +import static java.util.Arrays.asList; import static java.util.stream.Collectors.joining; import static org.elasticsearch.compute.lucene.LuceneOperator.NO_LIMIT; import static org.elasticsearch.compute.operator.LimitOperator.Factory; @@ -239,9 +239,15 @@ private PhysicalOperation planEsStats(EsStatsQueryExec statsQuery, LocalExecutio if (physicalOperationProviders instanceof EsPhysicalOperationProviders == false) { throw new EsqlIllegalArgumentException("EsStatsQuery should only occur against a Lucene backend"); } - EsPhysicalOperationProviders esProvider = (EsPhysicalOperationProviders) physicalOperationProviders; + if (statsQuery.stats().size() > 1) { + throw new EsqlIllegalArgumentException("EsStatsQuery currently supports only one field statistic"); + } - Function querySupplier = EsPhysicalOperationProviders.querySupplier(statsQuery.query()); + // for now only one stat is supported + EsStatsQueryExec.Stat stat = statsQuery.stats().get(0); + + EsPhysicalOperationProviders esProvider = (EsPhysicalOperationProviders) physicalOperationProviders; + Function querySupplier = EsPhysicalOperationProviders.querySupplier(stat.filter(statsQuery.query())); Expression limitExp = statsQuery.limit(); int limit = limitExp != null ? (Integer) limitExp.fold() : NO_LIMIT; @@ -411,8 +417,8 @@ private PhysicalOperation planTopN(TopNExec topNExec, LocalExecutionPlannerConte return source.with( new TopNOperatorFactory( limit, - Arrays.asList(elementTypes), - Arrays.asList(encoders), + asList(elementTypes), + asList(encoders), orders, context.pageSize(2000 + topNExec.estimatedRowSize()) ), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java new file mode 100644 index 0000000000000..19c45a71f5cec --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java @@ -0,0 +1,259 @@ +/* + * 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 com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec.Stat; +import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.esql.planner.PlannerUtils; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.DisabledSearchStats; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.junit.Before; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.util.Arrays.asList; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.configuration; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; +import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; +import static org.elasticsearch.xpack.esql.plan.physical.EsStatsQueryExec.StatsType; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; + +public class LocalPhysicalPlanOptimizerTests extends ESTestCase { + + private static final String PARAM_FORMATTING = "%1$s"; + + /** + * Estimated size of a keyword field in bytes. + */ + private static final int KEYWORD_EST = EstimatesRowSize.estimateSize(DataTypes.KEYWORD); + + private EsqlParser parser; + private Analyzer analyzer; + private LogicalPlanOptimizer logicalOptimizer; + private PhysicalPlanOptimizer physicalPlanOptimizer; + private Mapper mapper; + private Map mapping; + private int allFieldRowSize; + + private final EsqlConfiguration config; + + @ParametersFactory(argumentFormatting = PARAM_FORMATTING) + public static List readScriptSpec() { + return settings().stream().map(t -> { + var settings = Settings.builder().loadFromMap(t.v2()).build(); + return new Object[] { t.v1(), configuration(new QueryPragmas(settings)) }; + }).toList(); + } + + private static List>> settings() { + return asList(new Tuple<>("default", Map.of())); + } + + public LocalPhysicalPlanOptimizerTests(String name, EsqlConfiguration config) { + this.config = config; + } + + @Before + public void init() { + parser = new EsqlParser(); + + mapping = loadMapping("mapping-basic.json"); + allFieldRowSize = mapping.values() + .stream() + .mapToInt( + f -> (EstimatesRowSize.estimateSize(EsqlDataTypes.widenSmallNumericTypes(f.getDataType())) + f.getProperties() + .values() + .stream() + // check one more level since the mapping contains TEXT fields with KEYWORD multi-fields + .mapToInt(x -> EstimatesRowSize.estimateSize(EsqlDataTypes.widenSmallNumericTypes(x.getDataType()))) + .sum()) + ) + .sum(); + EsIndex test = new EsIndex("test", mapping); + IndexResolution getIndexResult = IndexResolution.valid(test); + logicalOptimizer = new LogicalPlanOptimizer(); + physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config)); + FunctionRegistry functionRegistry = new EsqlFunctionRegistry(); + mapper = new Mapper(functionRegistry); + var enrichResolution = new EnrichResolution( + Set.of( + new EnrichPolicyResolution( + "foo", + new EnrichPolicy(EnrichPolicy.MATCH_TYPE, null, List.of("idx"), "fld", List.of("a", "b")), + IndexResolution.valid( + new EsIndex( + "idx", + Map.ofEntries( + Map.entry("a", new EsField("a", DataTypes.INTEGER, Map.of(), true)), + Map.entry("b", new EsField("b", DataTypes.LONG, Map.of(), true)) + ) + ) + ) + ) + ), + Set.of("foo") + ); + + analyzer = new Analyzer( + new AnalyzerContext(config, functionRegistry, getIndexResult, enrichResolution), + new Verifier(new Metrics()) + ); + } + + /** + * Expects + * LimitExec[500[INTEGER]] + * \_AggregateExec[[],[COUNT([2a][KEYWORD]) AS c],FINAL,null] + * \_ExchangeExec[[count{r}#24, seen{r}#25],true] + * \_EsStatsQueryExec[test], stats[Stat[name=*, type=COUNT, query=null]]], query[{"esql_single_value":{"field":"emp_no","next":{"range":{"emp_no":{"lt":10050,"boost":1.0}}}}}][count{r}#40, seen{r}#41], limit[], + */ + public void testCountAllWithEval() { + var plan = plan(""" + from test | eval s = salary | rename s as sr | eval hidden_s = sr | rename emp_no as e | where e < 10050 + | stats c = count(*) + """); + var stat = queryStatsFor(plan); + assertThat(stat.type(), is(StatsType.COUNT)); + assertThat(stat.query(), is(nullValue())); + } + + /** + * Expects + * LimitExec[500[INTEGER]] + * \_AggregateExec[[],[COUNT([2a][KEYWORD]) AS c],FINAL,null] + * \_ExchangeExec[[count{r}#14, seen{r}#15],true] + * \_EsStatsQueryExec[test], stats[Stat[name=*, type=COUNT, query=null]]], + * query[{"esql_single_value":{"field":"emp_no","next":{"range":{"emp_no":{"gt":10040,"boost":1.0}}}}}][count{r}#30, seen{r}#31], limit[], + */ + public void testCountAllWithFilter() { + var plan = plan("from test | where emp_no > 10040 | stats c = count(*)"); + var stat = queryStatsFor(plan); + assertThat(stat.type(), is(StatsType.COUNT)); + assertThat(stat.query(), is(nullValue())); + } + + /** + * Expects + * LimitExec[500[INTEGER]] + * \_AggregateExec[[],[COUNT(emp_no{f}#5) AS c],FINAL,null] + * \_ExchangeExec[[count{r}#15, seen{r}#16],true] + * \_EsStatsQueryExec[test], stats[Stat[name=emp_no, type=COUNT, query={ + * "exists" : { + * "field" : "emp_no", + * "boost" : 1.0 + * } + * }]]], query[{"esql_single_value":{"field":"emp_no","next":{"range":{"emp_no":{"gt":10040,"boost":1.0}}}}}][count{r}#31, seen{r}#32], limit[], + */ + public void testCountFieldWithFilter() { + var plan = plan("from test | where emp_no > 10040 | stats c = count(emp_no)"); + var stat = queryStatsFor(plan); + assertThat(stat.type(), is(StatsType.COUNT)); + assertThat(stat.query(), is(QueryBuilders.existsQuery("emp_no"))); + } + + /** + * Expects - for now + * LimitExec[500[INTEGER]] + * \_AggregateExec[[],[COUNT(hidden_s{r}#8) AS c],FINAL,null] + * \_ExchangeExec[[count{r}#25, seen{r}#26],true] + * \_AggregateExec[[],[COUNT(hidden_s{r}#8) AS c],PARTIAL,8] + * \_EvalExec[[salary{f}#20 AS s, s{r}#3 AS hidden_s]] + * \_FieldExtractExec[salary{f}#20] + * \_EsQueryExec[test], query[{"esql_single_value":{"field":"emp_no","next":{"range":{"emp_no":{"lt":10050,"boost":1.0}}}}}][_doc{f}#42], limit[], sort[] estimatedRowSize[16] + */ + public void testCountFieldWithEval() { + var plan = plan(""" + from test | eval s = salary | rename s as sr | eval hidden_s = sr | rename emp_no as e | where e < 10050 + | stats c = count(hidden_s) + """); + var limit = as(plan, LimitExec.class); + var agg = as(limit.child(), AggregateExec.class); + var exg = as(agg.child(), ExchangeExec.class); + agg = as(exg.child(), AggregateExec.class); + var eval = as(agg.child(), EvalExec.class); + } + + private Stat queryStatsFor(PhysicalPlan plan) { + var limit = as(plan, LimitExec.class); + var agg = as(limit.child(), AggregateExec.class); + var exg = as(agg.child(), ExchangeExec.class); + var statSource = as(exg.child(), EsStatsQueryExec.class); + var stats = statSource.stats(); + assertThat(stats, hasSize(1)); + var stat = stats.get(0); + return stat; + } + + private PhysicalPlan plan(String query) { + return optimizedPlan(physicalPlan(query)); + } + + private PhysicalPlan optimizedPlan(PhysicalPlan plan) { + return optimizedPlan(plan, EsqlTestUtils.TEST_SEARCH_STATS); + } + + private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { + // System.out.println("* Physical Before\n" + plan); + var p = EstimatesRowSize.estimateRowSize(0, physicalPlanOptimizer.optimize(plan)); + // System.out.println("* Physical After\n" + p); + // the real execution breaks the plan at the exchange and then decouples the plan + // this is of no use in the unit tests, which checks the plan as a whole instead of each + // individually hence why here the plan is kept as is + + var logicalTestOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(config, new DisabledSearchStats())); + var physicalTestOptimizer = new TestLocalPhysicalPlanOptimizer(new LocalPhysicalOptimizerContext(config), true); + var l = PlannerUtils.localPlan(plan, logicalTestOptimizer, physicalTestOptimizer); + + // System.out.println("* Localized DataNode Plan\n" + l); + return l; + } + + private PhysicalPlan physicalPlan(String query) { + var logical = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement(query))); + // System.out.println("Logical\n" + logical); + var physical = mapper.map(logical); + assertSerialization(physical); + return physical; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index e20ba72b82e5c..5bdce6f3dfd08 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.index.query.TermsQueryBuilder; import org.elasticsearch.index.query.WildcardQueryBuilder; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.xpack.core.enrich.EnrichPolicy; import org.elasticsearch.xpack.esql.EsqlTestUtils; import org.elasticsearch.xpack.esql.analysis.Analyzer; @@ -108,7 +107,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; -@TestLogging(value = "org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer:TRACE", reason = "debug") +//@TestLogging(value = "org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer:TRACE", reason = "debug") public class PhysicalPlanOptimizerTests extends ESTestCase { private static final String PARAM_FORMATTING = "%1$s"; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java index a4e94254b94ae..0d1e85171bac3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java @@ -13,12 +13,19 @@ public class TestLocalPhysicalPlanOptimizer extends LocalPhysicalPlanOptimizer { + private final boolean esRules; + public TestLocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context) { + this(context, false); + } + + public TestLocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context, boolean esRules) { super(context); + this.esRules = esRules; } @Override protected List> batches() { - return rules(false); + return rules(esRules); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java index 640dd410d8573..c4350c8ec74d7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java @@ -102,7 +102,7 @@ protected Object pluggableMakeArg(Class> toBuildClass, Class