diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java index 103e4234d2cc6..684cab6d2663f 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java @@ -27,7 +27,6 @@ import org.elasticsearch.xpack.ql.expression.Order.NullsPosition; import org.elasticsearch.xpack.ql.expression.Order.OrderDirection; import org.elasticsearch.xpack.ql.expression.predicate.Predicates; -import org.elasticsearch.xpack.ql.expression.predicate.logical.And; import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; @@ -47,6 +46,7 @@ import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.OptimizerRule; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PropagateEquals; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PruneLiteralsInOrderBy; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PushDownAndCombineFilters; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ReplaceSurrogateFunction; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.SetAsOptimized; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.SimplifyComparisonsArithmetics; @@ -93,13 +93,14 @@ protected Iterable.Batch> batches() { new PropagateNullable(), new CombineBinaryComparisons(), new CombineDisjunctionsToIn(), - new PushDownAndCombineFilters(), new SimplifyComparisonsArithmetics(DataTypes::areCompatible), // prune/elimination new PruneFilters(), new PruneLiteralsInOrderBy(), new PruneCast(), - new CombineLimits()); + new CombineLimits(), + new PushDownAndCombineFilters() + ); Batch constraints = new Batch("Infer constraints", Limiter.ONCE, new PropagateJoinKeyConstraints()); @@ -189,25 +190,6 @@ protected LogicalPlan rule(Filter filter) { } } - static class PushDownAndCombineFilters extends OptimizerRule { - - @Override - protected LogicalPlan rule(Filter filter) { - LogicalPlan child = filter.child(); - LogicalPlan plan = filter; - - if (child instanceof Filter) { - Filter f = (Filter) child; - plan = new Filter(f.source(), f.child(), new And(f.source(), f.condition(), filter.condition())); - } else if (child instanceof UnaryPlan) { - UnaryPlan up = (UnaryPlan) child; - plan = child.replaceChildrenSameSize(singletonList(new Filter(filter.source(), up.child(), filter.condition()))); - } - - return plan; - } - } - static class ReplaceRegexMatch extends org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ReplaceRegexMatch { @Override protected Expression regexToEquals(RegexMatch regexMatch, Literal literal) { diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java index d6f8466f308e3..73d641831400c 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java @@ -67,6 +67,7 @@ import static org.elasticsearch.xpack.eql.EqlTestUtils.TEST_CFG; import static org.elasticsearch.xpack.ql.TestUtils.UTC; import static org.elasticsearch.xpack.ql.expression.Literal.TRUE; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PushDownAndCombineFilters; import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; @@ -334,7 +335,7 @@ public void testCombineFilters() { Filter filterChild = basicFilter(left); Filter filterParent = new Filter(EMPTY, filterChild, right); - LogicalPlan result = new Optimizer.PushDownAndCombineFilters().apply(filterParent); + LogicalPlan result = new PushDownAndCombineFilters().apply(filterParent); assertEquals(Filter.class, result.getClass()); Expression condition = ((Filter) result).condition(); @@ -359,7 +360,7 @@ public void testPushDownFilterUnary() { OrderBy order = new OrderBy(EMPTY, rel(), emptyList()); Filter filter = new Filter(EMPTY, order, left); - LogicalPlan result = new Optimizer.PushDownAndCombineFilters().apply(filter); + LogicalPlan result = new PushDownAndCombineFilters().apply(filter); assertEquals(OrderBy.class, result.getClass()); OrderBy o = (OrderBy) result; @@ -386,7 +387,7 @@ public void testPushDownFilterDoesNotApplyOnNonUnary() { Sequence s = sequence(rule1, rule2); Filter filter = new Filter(EMPTY, s, left); - LogicalPlan result = new Optimizer.PushDownAndCombineFilters().apply(filter); + LogicalPlan result = new PushDownAndCombineFilters().apply(filter); assertEquals(Filter.class, result.getClass()); Filter f = (Filter) result; diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java index 516df40d3f3d8..28112ca87e510 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java @@ -13,6 +13,7 @@ import org.elasticsearch.xpack.ql.expression.Nullability; import org.elasticsearch.xpack.ql.expression.Order; import org.elasticsearch.xpack.ql.expression.function.Function; +import org.elasticsearch.xpack.ql.expression.function.Functions; import org.elasticsearch.xpack.ql.expression.function.scalar.SurrogateFunction; import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; import org.elasticsearch.xpack.ql.expression.predicate.BinaryPredicate; @@ -40,10 +41,12 @@ import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NullEquals; import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch; import org.elasticsearch.xpack.ql.expression.predicate.regex.StringPattern; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; import org.elasticsearch.xpack.ql.plan.logical.Filter; import org.elasticsearch.xpack.ql.plan.logical.Limit; import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; import org.elasticsearch.xpack.ql.rule.Rule; import org.elasticsearch.xpack.ql.type.DataType; import org.elasticsearch.xpack.ql.type.DataTypes; @@ -63,6 +66,7 @@ import static java.lang.Math.signum; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.elasticsearch.xpack.ql.expression.Literal.FALSE; import static org.elasticsearch.xpack.ql.expression.Literal.TRUE; import static org.elasticsearch.xpack.ql.expression.predicate.Predicates.combineAnd; @@ -1139,6 +1143,51 @@ protected In createIn(Expression key, List values, ZoneId zoneId) { } } + public static class PushDownAndCombineFilters extends OptimizerRule { + + @Override + protected LogicalPlan rule(Filter filter) { + LogicalPlan plan = filter; + LogicalPlan child = filter.child(); + Expression condition = filter.condition(); + + if (child instanceof Filter) { + Filter f = (Filter) child; + plan = f.with(new And(f.source(), f.condition(), condition)); + } + // as it stands, all other unary plans should allow filters to be pushed down + else if (child instanceof UnaryPlan) { + UnaryPlan unary = (UnaryPlan) child; + // in case of aggregates, worry about filters that contain aggregations + if (unary instanceof Aggregate && condition.anyMatch(Functions::isAggregate)) { + Aggregate agg = (Aggregate) unary; + List conjunctions = new ArrayList<>(splitAnd(condition)); + List inPlace = new ArrayList<>(); + // extract all conjunctions containing aggregates + for (Iterator iterator = conjunctions.iterator(); iterator.hasNext();) { + Expression conjunction = iterator.next(); + if (conjunction.anyMatch(Functions::isAggregate)) { + inPlace.add(conjunction); + iterator.remove(); + } + } + // if at least one expression can be pushed down, update the tree + if (conjunctions.size() > 0) { + child = child.replaceChildrenSameSize( + singletonList(filter.with(unary.child(), Predicates.combineAnd(conjunctions))) + ); + plan = filter.with(child, Predicates.combineAnd(inPlace)); + } + } else { + // push down filter + plan = child.replaceChildrenSameSize(singletonList(filter.with(unary.child(), condition))); + } + } + + return plan; + } + } + public static class ReplaceSurrogateFunction extends OptimizerExpressionRule { public ReplaceSurrogateFunction() { diff --git a/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java b/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java index d39aad7f28055..bdea29b2d10eb 100644 --- a/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java +++ b/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.xpack.ql.expression.FieldAttribute; import org.elasticsearch.xpack.ql.expression.Literal; import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.function.aggregate.Count; import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; import org.elasticsearch.xpack.ql.expression.predicate.Predicates; import org.elasticsearch.xpack.ql.expression.predicate.Range; @@ -45,6 +46,10 @@ import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.CombineBinaryComparisons; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ConstantFolding; import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PropagateEquals; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Project; import org.elasticsearch.xpack.ql.tree.NodeInfo; import org.elasticsearch.xpack.ql.tree.Source; import org.elasticsearch.xpack.ql.type.DataType; @@ -56,6 +61,7 @@ import java.util.List; import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; import static org.elasticsearch.xpack.ql.TestUtils.equalsOf; @@ -68,11 +74,13 @@ import static org.elasticsearch.xpack.ql.TestUtils.nullEqualsOf; import static org.elasticsearch.xpack.ql.TestUtils.of; import static org.elasticsearch.xpack.ql.TestUtils.rangeOf; +import static org.elasticsearch.xpack.ql.TestUtils.relation; import static org.elasticsearch.xpack.ql.expression.Literal.FALSE; import static org.elasticsearch.xpack.ql.expression.Literal.NULL; import static org.elasticsearch.xpack.ql.expression.Literal.TRUE; import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.CombineDisjunctionsToIn; import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PropagateNullable; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PushDownAndCombineFilters; import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ReplaceRegexMatch; import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; @@ -1609,7 +1617,48 @@ public void testIsNullDisjunction() throws Exception { assertEquals(and, new PropagateNullable().rule(and)); } - public void testSkipNull() throws Exception { + public void testCombineFilters() throws Exception { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + + Filter fa = new Filter(EMPTY, relation, conditionA); + Filter fb = new Filter(EMPTY, fa, conditionB); + + assertEquals(new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), new PushDownAndCombineFilters().apply(fb)); + } + + public void testPushDownFilter() throws Exception { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + + Filter fa = new Filter(EMPTY, relation, conditionA); + List projections = singletonList(getFieldAttribute("b")); + Project project = new Project(EMPTY, fa, projections); + Filter fb = new Filter(EMPTY, project, conditionB); + + Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)); + assertEquals(new Project(EMPTY, combinedFilter, projections), new PushDownAndCombineFilters().apply(fb)); + } + + public void testPushDownFilterThroughAgg() throws Exception { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + GreaterThanOrEqual aggregateCondition = greaterThanOrEqualOf(new Count(EMPTY, ONE, false), THREE); + + Filter fa = new Filter(EMPTY, relation, conditionA); + List projections = singletonList(getFieldAttribute("b")); + // invalid aggregate but that's fine cause its properties are not used by this rule + Aggregate aggregate = new Aggregate(EMPTY, fa, emptyList(), emptyList()); + Filter fb = new Filter(EMPTY, aggregate, new And(EMPTY, aggregateCondition, conditionB)); + + Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)); + + // expected + Filter expected = new Filter(EMPTY, new Aggregate(EMPTY, combinedFilter, emptyList(), emptyList()), aggregateCondition); + assertEquals(expected, new PushDownAndCombineFilters().apply(fb)); } } diff --git a/x-pack/plugin/sql/qa/server/src/main/resources/subselect.sql-spec b/x-pack/plugin/sql/qa/server/src/main/resources/subselect.sql-spec index 53cd6cc38d7ef..30af0dc35d089 100644 --- a/x-pack/plugin/sql/qa/server/src/main/resources/subselect.sql-spec +++ b/x-pack/plugin/sql/qa/server/src/main/resources/subselect.sql-spec @@ -28,9 +28,9 @@ SELECT g AS h FROM (SELECT first_name AS f, last_name, gender AS g FROM test_emp groupAndOrderByRealiasedExpression SELECT emp_group AS e, min_high_salary AS s FROM ( - SELECT emp_no % 2 AS emp_group, MIN(salary) AS min_high_salary + SELECT emp_no % 2 AS emp_group, MIN(salary) AS min_high_salary FROM test_emp - WHERE salary > 50000 + WHERE salary > 50000 GROUP BY emp_group ) ORDER BY e DESC; @@ -41,10 +41,10 @@ SELECT * FROM (SELECT * FROM ( SELECT * FROM test_emp )); multiLevelSelectStarWithAlias SELECT * FROM (SELECT * FROM ( SELECT * FROM test_emp ) b) c; -// AwaitsFix: https://github.com/elastic/elasticsearch/issues/69758 -filterAfterGroupBy-Ignore +filterAfterGroupBy SELECT s2 AS s3 FROM (SELECT s AS s2 FROM ( SELECT salary AS s FROM test_emp) GROUP BY s2) WHERE s2 < 5 ORDER BY s3 DESC NULLS last; - +filterAfterGroupByWithWhere +SELECT salary FROM test_emp WHERE salary < 5 GROUP BY salary ORDER BY salary DESC NULLS last; countAndComplexCondition SELECT COUNT(*) as c FROM (SELECT * FROM test_emp WHERE gender IS NOT NULL) WHERE ABS(salary) > 0 GROUP BY gender ORDER BY gender; countAndGroupBy @@ -90,3 +90,113 @@ SELECT GREATEST(10096, ABS(emp_no + 1)) AS gt FROM (SELECT * FROM test_emp) ORDE inAndIsNotNull SELECT NOT((languages = 2) IS NULL) AS col1, NOT((languages = 2) IS NOT NULL) AS col2 FROM (SELECT * FROM test_emp WHERE emp_no IN (10019, 10020, 10021)) WHERE emp_no IN (10018, 10019, 10020) ORDER BY emp_no; +filterOnColumn +SELECT * FROM ( + SELECT * FROM test_emp +) +WHERE languages > 1 +; +filterOnColumnFunction +SELECT * FROM ( + SELECT (languages + 1) % 5 AS l FROM test_emp +) +WHERE l > 1 +; +filterOnGroupBy +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM test_emp WHERE languages > 1 GROUP BY gender ORDER BY gender +) +WHERE gender IS NOT NULL +; +filterOnAggregate +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM test_emp WHERE languages > 1 GROUP BY gender ORDER BY gender +) +WHERE max % 2 = 0 +; + +filterOnAggregateAndGroupBy +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM test_emp WHERE languages > 1 GROUP BY gender ORDER BY gender +) +WHERE max % 2 = 0 AND gender IS NOT NULL +; + +// see https://github.com/elastic/elasticsearch/issues/71350 +filterOnAggregateAndGroupByWithDisjunction-Ignore +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM test_emp WHERE languages > 1 GROUP BY gender ORDER BY gender +) +WHERE max % 2 = 0 OR gender IS NOT NULL +; + +filterOnColumnWithOutsideFilter +SELECT * FROM ( + SELECT languages FROM test_emp WHERE languages > 1 +) +WHERE languages < 5 +; +filterOnColumnFunctionWithOutsideFilter +SELECT * FROM ( + SELECT * FROM test_emp WHERE languages + 1 > 1 +) +WHERE languages + 1 < 10 +; +filterOnOutsideAggregate +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM ( + SELECT * FROM test_emp WHERE languages > 1 + ) + GROUP BY gender ORDER BY gender +) +WHERE max % 2 = 0 +; +multiLevelFilterOnOutsideAggregate +SELECT m AS mx FROM ( + SELECT max AS m FROM ( + SELECT gender, MAX(salary) AS max FROM ( + SELECT * FROM test_emp WHERE languages > 1 + ) + GROUP BY gender + HAVING max > 0 + ORDER BY gender + ) + WHERE max / 10 > 1 +) +WHERE m % 10 > 1 +; +pushDownHavingFilter +SELECT * FROM ( + SELECT gender, MAX(salary) AS max FROM ( + SELECT * FROM test_emp WHERE languages > 1 + ) +GROUP BY gender ORDER BY gender +) +WHERE max % 2 = 0 +; + +filterOnGroupingInFunction +SELECT * FROM ( + SELECT languages + 1 AS lan, MAX(salary) AS max FROM test_emp WHERE languages > 1 GROUP BY lan ORDER BY 1 +) +WHERE (lan - 1) > 2 +; + +filterOnAggregateInFunction +SELECT * FROM ( + SELECT gender, MAX(salary) / 100 AS max FROM test_emp WHERE languages > 1 GROUP BY gender +) +WHERE max > 746 AND gender IS NOT NULL ORDER BY gender +; + +// AwaitsFix : https://github.com/elastic/elasticsearch/issues/71394 +filterOnGroupingAndAggregateByFunction-Ignore +SELECT * FROM ( + SELECT lan - 1 AS lan, max / 2 AS max FROM ( + SELECT languages + 1 AS lan, MAX(salary) / 10 AS max FROM test_emp WHERE languages > 1 GROUP BY lan ORDER BY 1 + ) +) +WHERE (lan - 1) > 1 AND max - 1000 > 2500 +; +countAndComplexCondition +SELECT COUNT(*) as c FROM (SELECT * FROM test_emp WHERE gender IS NOT NULL) WHERE ABS(salary) > 0 GROUP BY gender ORDER BY gender; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Verifier.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Verifier.java index cd2d0f60082ba..ec81f241fbca8 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Verifier.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Verifier.java @@ -671,11 +671,10 @@ private static void checkGroupingFunctionTarget(GroupingFunction f, Set private static void checkFilterOnAggs(LogicalPlan p, Set localFailures, AttributeMap attributeRefs) { if (p instanceof Filter) { Filter filter = (Filter) p; - LogicalPlan filterChild = filter.child(); - if (filterChild instanceof Aggregate == false) { + if (filter.anyMatch(Aggregate.class::isInstance) == false) { filter.condition().forEachDown(Expression.class, e -> { if (Functions.isAggregate(attributeRefs.resolve(e, e))) { - if (filterChild instanceof Project) { + if (filter.child() instanceof Project) { filter.condition().forEachDown(FieldAttribute.class, f -> localFailures.add(fail(e, "[{}] field must appear in the GROUP BY clause or in an aggregate function", Expressions.name(f))) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java index 4530010a970bf..3edaec13d0f69 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java @@ -111,6 +111,7 @@ import static java.util.Collections.singletonList; import static org.elasticsearch.xpack.ql.expression.Expressions.equalsAsAttribute; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PushDownAndCombineFilters; import static org.elasticsearch.xpack.ql.util.CollectionUtils.combine; @@ -165,7 +166,8 @@ protected Iterable.Batch> batches() { new PruneOrderByNestedFields(), new PruneCast(), // order by alignment of the aggs - new SortAggregateOnOrderBy() + new SortAggregateOnOrderBy(), + new PushDownAndCombineFilters() ); Batch aggregate = new Batch("Aggregation Rewrite", diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java index 703663a2160c0..4c186f000e19f 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java @@ -521,11 +521,6 @@ public void testGroupByHavingNonGrouped() { accept("SELECT AVG(int) FROM test GROUP BY bool HAVING AVG(int) > 2"); } - public void testGroupByWhereSubselect() { - accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool) WHERE b = false"); - accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool HAVING AVG(int) > 2) WHERE b = false"); - } - public void testGroupByAggregate() { assertEquals("1:36: Cannot use an aggregate [AVG] for grouping", error("SELECT AVG(int) FROM test GROUP BY AVG(int)")); @@ -1318,4 +1313,18 @@ public void testShapeInOrderBy() { public void testShapeInSelect() { accept("SELECT ST_X(shape) FROM test"); } + + public void testSubselectWhereOnGroupBy() { + accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool) WHERE b = false"); + accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool HAVING AVG(int) > 2) WHERE b = false"); + } + + public void testSubselectWhereOnAggregate() { + accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool) WHERE a > 10"); + accept("SELECT b, a FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool) WHERE a > 10 AND b = FALSE"); + } + + public void testSubselectWithOrderWhereOnAggregate() { + accept("SELECT * FROM (SELECT bool as b, AVG(int) as a FROM test GROUP BY bool ORDER BY bool) WHERE a > 10"); + } }