From a0de6cef8328a29529f71851efe719ab7b08a869 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Thu, 19 Nov 2020 13:46:10 +0200 Subject: [PATCH 1/3] QL: Introduce common analyzer base class Extract the analyzer base classes into a common project similar to the Optimizer ones. Extract AddMissingEqualsOnBoolFields a common rule to be used across Close #63695 --- .../xpack/eql/analysis/Analyzer.java | 8 +- .../xpack/eql/optimizer/Optimizer.java | 34 +------- .../xpack/ql/analyzer/AnalyzerRules.java | 84 +++++++++++++++++++ .../xpack/sql/analysis/analyzer/Analyzer.java | 82 ++++++------------ 4 files changed, 116 insertions(+), 92 deletions(-) create mode 100644 x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java index 159bc06fbb4d6..0a0db21f01f52 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java @@ -21,8 +21,9 @@ import java.util.Collection; import java.util.LinkedHashSet; -import static java.util.Collections.singletonList; +import static java.util.Arrays.asList; import static org.elasticsearch.xpack.eql.analysis.AnalysisUtils.resolveAgainstList; +import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AddMissingEqualsToBoolField; public class Analyzer extends RuleExecutor { @@ -42,7 +43,10 @@ protected Iterable.Batch> batches() { new ResolveRefs(), new ResolveFunctions()); - return singletonList(resolution); + Batch cleanup = new Batch("Finish Analysis", Limiter.ONCE, + new AddMissingEqualsToBoolField()); + + return asList(resolution, cleanup); } public LogicalPlan analyze(LogicalPlan plan) { 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 4bbeccf9d5722..7006a2b20ce1e 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 @@ -17,7 +17,6 @@ import org.elasticsearch.xpack.eql.util.MathUtils; import org.elasticsearch.xpack.eql.util.StringUtils; import org.elasticsearch.xpack.ql.expression.Expression; -import org.elasticsearch.xpack.ql.expression.FieldAttribute; import org.elasticsearch.xpack.ql.expression.Literal; import org.elasticsearch.xpack.ql.expression.NamedExpression; import org.elasticsearch.xpack.ql.expression.Order; @@ -25,7 +24,6 @@ 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.BinaryLogic; 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; @@ -75,9 +73,6 @@ protected Iterable.Batch> batches() { new ReplaceSurrogateFunction(), new ReplaceRegexMatch()); - Batch syntactic = new Batch("Rewrite Syntactic Sugar", Limiter.ONCE, - new AddMissingEquals()); - Batch operators = new Batch("Operator Optimization", new ConstantFolding(), // boolean @@ -109,7 +104,7 @@ protected Iterable.Batch> batches() { Batch label = new Batch("Set as Optimized", Limiter.ONCE, new SetAsOptimized()); - return asList(substitutions, syntactic, operators, constraints, operators, ordering, local, label); + return asList(substitutions, operators, constraints, operators, ordering, local, label); } private static class ReplaceWildcards extends OptimizerRule { @@ -153,33 +148,6 @@ private static boolean isWildcard(Expression expr) { } } - private static class AddMissingEquals extends OptimizerRule { - - @Override - protected LogicalPlan rule(Filter filter) { - // check the condition itself - Expression condition = replaceRawBoolFieldWithEquals(filter.condition()); - // otherwise look for binary logic - if (condition == filter.condition()) { - condition = condition.transformUp(b -> - b.replaceChildren(asList(replaceRawBoolFieldWithEquals(b.left()), replaceRawBoolFieldWithEquals(b.right()))) - , BinaryLogic.class); - } - - if (condition != filter.condition()) { - filter = new Filter(filter.source(), filter.child(), condition); - } - return filter; - } - - private Expression replaceRawBoolFieldWithEquals(Expression e) { - if (e instanceof FieldAttribute) { - e = new Equals(e.source(), e, Literal.of(e, Boolean.TRUE)); - } - return e; - } - } - private static class ReplaceNullChecks extends OptimizerRule { @Override diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java new file mode 100644 index 0000000000000..a837343bc29b1 --- /dev/null +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java @@ -0,0 +1,84 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ + +package org.elasticsearch.xpack.ql.analyzer; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.rule.Rule; + +import static java.util.Arrays.asList; + +public final class AnalyzerRules { + + public static class AddMissingEqualsToBoolField extends AnalyzerRule { + + @Override + protected LogicalPlan rule(Filter filter) { + // check the condition itself + Expression condition = replaceRawBoolFieldWithEquals(filter.condition()); + // otherwise look for binary logic + if (condition == filter.condition()) { + condition = condition.transformUp(b -> + b.replaceChildren(asList(replaceRawBoolFieldWithEquals(b.left()), replaceRawBoolFieldWithEquals(b.right()))) + , BinaryLogic.class); + } + + if (condition != filter.condition()) { + filter = new Filter(filter.source(), filter.child(), condition); + } + return filter; + } + + private Expression replaceRawBoolFieldWithEquals(Expression e) { + if (e instanceof FieldAttribute) { + e = new Equals(e.source(), e, Literal.of(e, Boolean.TRUE)); + } + return e; + } + + @Override + protected boolean skipResolved() { + return false; + } + } + + + public static abstract class AnalyzerRule extends Rule { + + // transformUp (post-order) - that is first children and then the node + // but with a twist; only if the tree is not resolved or analyzed + @Override + public final LogicalPlan apply(LogicalPlan plan) { + return plan.transformUp(t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t), typeToken()); + } + + @Override + protected abstract LogicalPlan rule(SubPlan plan); + + protected boolean skipResolved() { + return true; + } + } + + public abstract static class BaseAnalyzerRule extends AnalyzerRule { + + @Override + protected LogicalPlan rule(LogicalPlan plan) { + if (plan.childrenResolved() == false) { + return plan; + } + return doRule(plan); + } + + protected abstract LogicalPlan doRule(LogicalPlan plan); + } +} diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java index 2cf91b6687f44..296eb01cde21f 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java @@ -7,6 +7,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.LoggerMessageFormat; +import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AddMissingEqualsToBoolField; import org.elasticsearch.xpack.ql.capabilities.Resolvables; import org.elasticsearch.xpack.ql.common.Failure; import org.elasticsearch.xpack.ql.expression.Alias; @@ -38,7 +39,6 @@ import org.elasticsearch.xpack.ql.plan.logical.Project; import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; import org.elasticsearch.xpack.ql.plan.logical.UnresolvedRelation; -import org.elasticsearch.xpack.ql.rule.Rule; import org.elasticsearch.xpack.ql.rule.RuleExecutor; import org.elasticsearch.xpack.ql.session.Configuration; import org.elasticsearch.xpack.ql.type.DataType; @@ -70,6 +70,8 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.stream.Collectors.toList; +import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AnalyzerRule; +import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.BaseAnalyzerRule; import static org.elasticsearch.xpack.ql.util.CollectionUtils.combine; public class Analyzer extends RuleExecutor { @@ -257,7 +259,7 @@ private static boolean containsAggregate(Expression exp) { return containsAggregate(singletonList(exp)); } - private static class CTESubstitution extends AnalyzeRule { + private static class CTESubstitution extends AnalyzerRule { @Override protected LogicalPlan rule(With plan) { @@ -296,7 +298,7 @@ protected boolean skipResolved() { } } - private class ResolveTable extends AnalyzeRule { + private class ResolveTable extends AnalyzerRule { @Override protected LogicalPlan rule(UnresolvedRelation plan) { TableIdentifier table = plan.table(); @@ -316,7 +318,7 @@ protected LogicalPlan rule(UnresolvedRelation plan) { } } - private static class ResolveRefs extends BaseAnalyzeRule { + private static class ResolveRefs extends BaseAnalyzerRule { @Override protected LogicalPlan doRule(LogicalPlan plan) { @@ -498,7 +500,7 @@ private LogicalPlan dedupRight(LogicalPlan left, LogicalPlan right) { // Allow ordinal positioning in order/sort by (quite useful when dealing with aggs) // Note that ordering starts at 1 - private static class ResolveOrdinalInOrderByAndGroupBy extends BaseAnalyzeRule { + private static class ResolveOrdinalInOrderByAndGroupBy extends BaseAnalyzerRule { @Override protected boolean skipResolved() { @@ -592,7 +594,7 @@ private Integer findOrdinal(Expression expression) { if (expression.dataType().isInteger()) { Object v = Foldables.valueOf(expression); if (v instanceof Number) { - return Integer.valueOf(((Number) v).intValue()); + return ((Number) v).intValue(); } } } @@ -603,7 +605,7 @@ private Integer findOrdinal(Expression expression) { // It is valid to filter (including HAVING) or sort by attributes not present in the SELECT clause. // This rule pushed down the attributes for them to be resolved then projects them away. // As such this rule is an extended version of ResolveRefs - private static class ResolveMissingRefs extends BaseAnalyzeRule { + private static class ResolveMissingRefs extends BaseAnalyzerRule { private static class AggGroupingFailure { final List expectedGrouping; @@ -771,11 +773,9 @@ private static LogicalPlan propagateMissing(LogicalPlan plan, AttributeSet missi for (Attribute m : missing) { // but we can't add an agg if the group is missing if (!Expressions.match(a.groupings(), m::semanticEquals)) { - if (m instanceof Attribute) { - // pass failure information to help the verifier - m = new UnresolvedAttribute(m.source(), m.name(), m.qualifier(), null, null, - new AggGroupingFailure(Expressions.names(a.groupings()))); - } + // pass failure information to help the verifier + m = new UnresolvedAttribute(m.source(), m.name(), m.qualifier(), null, null, + new AggGroupingFailure(Expressions.names(a.groupings()))); failed.add(m); } } @@ -819,7 +819,7 @@ private static UnresolvedAttribute resolveMetadataToMessage(UnresolvedAttribute // // As such, identify all project and aggregates that have a Filter child // and look at any resolved aliases that match and replace them. - private class ResolveFilterRefs extends AnalyzeRule { + private static class ResolveFilterRefs extends AnalyzerRule { @Override protected LogicalPlan rule(LogicalPlan plan) { @@ -879,7 +879,7 @@ private Expression replaceAliases(Expression condition, List { + private class ResolveFunctions extends AnalyzerRule { @Override protected LogicalPlan rule(LogicalPlan plan) { @@ -918,7 +918,7 @@ protected LogicalPlan rule(LogicalPlan plan) { } } - private static class ResolveAliases extends BaseAnalyzeRule { + private static class ResolveAliases extends BaseAnalyzerRule { @Override protected LogicalPlan doRule(LogicalPlan plan) { @@ -951,13 +951,12 @@ protected LogicalPlan doRule(LogicalPlan plan) { } private boolean hasUnresolvedAliases(List expressions) { - return expressions != null && Expressions.anyMatch(expressions, e -> e instanceof UnresolvedAlias); + return expressions != null && Expressions.anyMatch(expressions, UnresolvedAlias.class::isInstance); } private List assignAliases(List exprs) { List newExpr = new ArrayList<>(exprs.size()); - for (int i = 0; i < exprs.size(); i++) { - NamedExpression expr = exprs.get(i); + for (NamedExpression expr : exprs) { NamedExpression transformed = (NamedExpression) expr.transformUp(ua -> { Expression child = ua.child(); if (child instanceof NamedExpression) { @@ -984,7 +983,7 @@ private List assignAliases(List expr // // Replace a project with aggregation into an aggregation // - private static class ProjectedAggregations extends AnalyzeRule { + private static class ProjectedAggregations extends AnalyzerRule { @Override protected LogicalPlan rule(Project p) { @@ -1001,7 +1000,7 @@ protected LogicalPlan rule(Project p) { // is a filter followed by projection and fails as the engine does not // understand it is an implicit grouping. // - private static class HavingOverProject extends AnalyzeRule { + private static class HavingOverProject extends AnalyzerRule { @Override protected LogicalPlan rule(Filter f) { @@ -1016,7 +1015,7 @@ protected LogicalPlan rule(Filter f) { if (n.foldable() == false && Functions.isAggregate(n) == false // folding might not work (it might wait for the optimizer) // so check whether any column is referenced - && n.anyMatch(e -> e instanceof FieldAttribute)) { + && n.anyMatch(FieldAttribute.class::isInstance)) { return f; } } @@ -1038,7 +1037,7 @@ protected boolean skipResolved() { // Handle aggs in HAVING. To help folding any aggs not found in Aggregation // will be pushed down to the Aggregate and then projected. This also simplifies the Verifier's job. // - private class ResolveAggsInHaving extends AnalyzeRule { + private class ResolveAggsInHaving extends AnalyzerRule { @Override protected boolean skipResolved() { @@ -1116,7 +1115,7 @@ private Set findMissingAggregate(Aggregate target, Expression f // Similar to Having however using a different matching pattern since HAVING is always Filter with Agg, // while an OrderBy can have multiple intermediate nodes (Filter,Project, etc...) // - private static class ResolveAggsInOrderBy extends AnalyzeRule { + private static class ResolveAggsInOrderBy extends AnalyzerRule { @Override protected boolean skipResolved() { @@ -1174,7 +1173,7 @@ protected LogicalPlan rule(OrderBy ob) { } } - private class ImplicitCasting extends AnalyzeRule { + private static class ImplicitCasting extends AnalyzerRule { @Override protected boolean skipResolved() { @@ -1221,7 +1220,7 @@ private Expression implicitCast(Expression e) { } - public static class PruneSubqueryAliases extends AnalyzeRule { + public static class PruneSubqueryAliases extends AnalyzerRule { @Override protected LogicalPlan rule(SubQueryAlias alias) { @@ -1234,7 +1233,7 @@ protected boolean skipResolved() { } } - public static class CleanAliases extends AnalyzeRule { + public static class CleanAliases extends AnalyzerRule { public static final CleanAliases INSTANCE = new CleanAliases(); @@ -1299,35 +1298,4 @@ protected boolean skipResolved() { return false; } } - - - abstract static class AnalyzeRule extends Rule { - - // transformUp (post-order) - that is first children and then the node - // but with a twist; only if the tree is not resolved or analyzed - @Override - public final LogicalPlan apply(LogicalPlan plan) { - return plan.transformUp(t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t), typeToken()); - } - - @Override - protected abstract LogicalPlan rule(SubPlan plan); - - protected boolean skipResolved() { - return true; - } - } - - abstract static class BaseAnalyzeRule extends AnalyzeRule { - - @Override - protected LogicalPlan rule(LogicalPlan plan) { - if (plan.childrenResolved() == false) { - return plan; - } - return doRule(plan); - } - - protected abstract LogicalPlan doRule(LogicalPlan plan); - } } From d5baaee79c4ba303bb7c526e95db2bf1daa4fd94 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Sat, 21 Nov 2020 16:01:13 +0200 Subject: [PATCH 2/3] Fix checkstyle --- .../java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java | 2 +- .../org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java index a837343bc29b1..ca7f1da4a42aa 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java @@ -52,7 +52,7 @@ protected boolean skipResolved() { } - public static abstract class AnalyzerRule extends Rule { + public abstract static class AnalyzerRule extends Rule { // transformUp (post-order) - that is first children and then the node // but with a twist; only if the tree is not resolved or analyzed diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java index 296eb01cde21f..ef43a4fb8cb2e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java @@ -7,7 +7,6 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.LoggerMessageFormat; -import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AddMissingEqualsToBoolField; import org.elasticsearch.xpack.ql.capabilities.Resolvables; import org.elasticsearch.xpack.ql.common.Failure; import org.elasticsearch.xpack.ql.expression.Alias; From b08cf5a580004b101305d981cf4e6ae39f3a8d2f Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Sat, 21 Nov 2020 17:03:38 +0200 Subject: [PATCH 3/3] Update failing test --- x-pack/plugin/sql/qa/server/src/main/resources/date.csv-spec | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/sql/qa/server/src/main/resources/date.csv-spec b/x-pack/plugin/sql/qa/server/src/main/resources/date.csv-spec index f7c13b9ce72a9..741da638824c3 100644 --- a/x-pack/plugin/sql/qa/server/src/main/resources/date.csv-spec +++ b/x-pack/plugin/sql/qa/server/src/main/resources/date.csv-spec @@ -38,7 +38,7 @@ SELECT TRUNCATE(YEAR(TODAY() - INTERVAL 50 YEARS) / 1000) AS result; currentDateFilter -SELECT first_name FROM test_emp WHERE hire_date > CURRENT_DATE() - INTERVAL 35 YEARS ORDER BY first_name ASC LIMIT 10; +SELECT first_name FROM test_emp WHERE hire_date > CURRENT_DATE() - INTERVAL 44 YEARS ORDER BY first_name ASC LIMIT 10; first_name ----------------- @@ -265,4 +265,4 @@ HAVING DATE_PARSE(DATETIME_FORMAT(MAX(birth_date), 'dd/MM/uuuu'), 'dd/MM/uuuu') 1963-03-21 00:00:00.000Z | 03 1962-12-29 00:00:00.000Z | 12 null | null -; \ No newline at end of file +;