From 60f81a95258faa6c6b06777bb327aaa6457140ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Wed, 23 Feb 2022 20:11:20 +0100 Subject: [PATCH] Add rule for unwrapping TIMESTAMP to DATE cast when comparing with DATE literal --- .../io/trino/sql/planner/PlanOptimizers.java | 2 + ...UnwrapTimestampToDateCastInComparison.java | 186 ++++++++++++++++++ .../planner/TestUnwrapCastInComparison.java | 132 +++++++++++++ .../sql/query/TestUnwrapCastInComparison.java | 50 +++++ 4 files changed, 370 insertions(+) create mode 100644 core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/UnwrapTimestampToDateCastInComparison.java diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java index 2d8baeceefae..748226680aa0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java @@ -230,6 +230,7 @@ import io.trino.sql.planner.iterative.rule.UnwrapCastInComparison; import io.trino.sql.planner.iterative.rule.UnwrapRowSubscript; import io.trino.sql.planner.iterative.rule.UnwrapSingleColumnRowInApply; +import io.trino.sql.planner.iterative.rule.UnwrapTimestampToDateCastInComparison; import io.trino.sql.planner.optimizations.AddExchanges; import io.trino.sql.planner.optimizations.AddLocalExchanges; import io.trino.sql.planner.optimizations.BeginTableWrite; @@ -356,6 +357,7 @@ public PlanOptimizers( .addAll(new SimplifyExpressions(plannerContext, typeAnalyzer).rules()) .addAll(new UnwrapRowSubscript().rules()) .addAll(new PushCastIntoRow().rules()) + .addAll(new UnwrapTimestampToDateCastInComparison(plannerContext, typeAnalyzer).rules()) .addAll(new UnwrapCastInComparison(plannerContext, typeAnalyzer).rules()) .addAll(new RemoveDuplicateConditions(metadata).rules()) .addAll(new CanonicalizeExpressions(plannerContext, typeAnalyzer).rules()) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/UnwrapTimestampToDateCastInComparison.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/UnwrapTimestampToDateCastInComparison.java new file mode 100644 index 000000000000..3562168f421c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/UnwrapTimestampToDateCastInComparison.java @@ -0,0 +1,186 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.sql.planner.iterative.rule; + +import io.trino.Session; +import io.trino.metadata.OperatorNotFoundException; +import io.trino.metadata.ResolvedFunction; +import io.trino.spi.TrinoException; +import io.trino.spi.type.DateType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.sql.InterpretedFunctionInvoker; +import io.trino.sql.PlannerContext; +import io.trino.sql.planner.ExpressionInterpreter; +import io.trino.sql.planner.LiteralEncoder; +import io.trino.sql.planner.NoOpSymbolResolver; +import io.trino.sql.planner.TypeAnalyzer; +import io.trino.sql.planner.TypeProvider; +import io.trino.sql.tree.Cast; +import io.trino.sql.tree.ComparisonExpression; +import io.trino.sql.tree.Expression; +import io.trino.sql.tree.ExpressionTreeRewriter; +import io.trino.sql.tree.IsNullPredicate; +import io.trino.sql.tree.NullLiteral; + +import java.util.Optional; + +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.sql.ExpressionUtils.and; +import static io.trino.sql.ExpressionUtils.or; +import static io.trino.sql.tree.ComparisonExpression.Operator.GREATER_THAN_OR_EQUAL; +import static io.trino.sql.tree.ComparisonExpression.Operator.LESS_THAN; +import static java.util.Objects.requireNonNull; + +/** + * Rewrites CAST(ts_column as DATE) OP date_literal to range expression on ts_column. Dropping cast + * allows for further optimizations, such as pushdown into connectors. + *

+ * TODO: replace with more general mechanism supporting broader range of types + * + * @see io.trino.sql.planner.iterative.rule.UnwrapCastInComparison + */ +public class UnwrapTimestampToDateCastInComparison + extends ExpressionRewriteRuleSet +{ + public UnwrapTimestampToDateCastInComparison(PlannerContext plannerContext, TypeAnalyzer typeAnalyzer) + { + super(createRewrite(plannerContext, typeAnalyzer)); + } + + private static ExpressionRewriter createRewrite(PlannerContext plannerContext, TypeAnalyzer typeAnalyzer) + { + requireNonNull(plannerContext, "plannerContext is null"); + requireNonNull(typeAnalyzer, "typeAnalyzer is null"); + + return (expression, context) -> unwrapCasts(context.getSession(), plannerContext, typeAnalyzer, context.getSymbolAllocator().getTypes(), expression); + } + + public static Expression unwrapCasts(Session session, + PlannerContext plannerContext, + TypeAnalyzer typeAnalyzer, + TypeProvider types, + Expression expression) + { + return ExpressionTreeRewriter.rewriteWith(new Visitor(plannerContext, typeAnalyzer, session, types), expression); + } + + private static class Visitor + extends io.trino.sql.tree.ExpressionRewriter + { + private final PlannerContext plannerContext; + private final TypeAnalyzer typeAnalyzer; + private final Session session; + private final TypeProvider types; + private final InterpretedFunctionInvoker functionInvoker; + private final LiteralEncoder literalEncoder; + + public Visitor(PlannerContext plannerContext, TypeAnalyzer typeAnalyzer, Session session, TypeProvider types) + { + this.plannerContext = requireNonNull(plannerContext, "plannerContext is null"); + this.typeAnalyzer = requireNonNull(typeAnalyzer, "typeAnalyzer is null"); + this.session = requireNonNull(session, "session is null"); + this.types = requireNonNull(types, "types is null"); + this.functionInvoker = new InterpretedFunctionInvoker(plannerContext.getMetadata()); + this.literalEncoder = new LiteralEncoder(plannerContext); + } + + @Override + public Expression rewriteComparisonExpression(ComparisonExpression node, Void context, ExpressionTreeRewriter treeRewriter) + { + ComparisonExpression expression = (ComparisonExpression) treeRewriter.defaultRewrite((Expression) node, null); + return unwrapCast(expression); + } + + private Expression unwrapCast(ComparisonExpression expression) + { + // Canonicalization is handled by CanonicalizeExpressionRewriter + if (!(expression.getLeft() instanceof Cast)) { + return expression; + } + + Object right = new ExpressionInterpreter(expression.getRight(), plannerContext, session, typeAnalyzer.getTypes(session, types, expression.getRight())) + .optimize(NoOpSymbolResolver.INSTANCE); + + Cast cast = (Cast) expression.getLeft(); + ComparisonExpression.Operator operator = expression.getOperator(); + + if (right == null || right instanceof NullLiteral) { + // handled by general UnwrapCastInComparison + return expression; + } + + if (right instanceof Expression) { + return expression; + } + + Type sourceType = typeAnalyzer.getType(session, types, cast.getExpression()); + Type targetType = typeAnalyzer.getType(session, types, expression.getRight()); + + if (sourceType instanceof TimestampType && targetType == DATE) { + return unwrapTimestampToDateCast(session, (TimestampType) sourceType, (DateType) targetType, operator, cast.getExpression(), (long) right).orElse(expression); + } + + return expression; + } + + private Optional unwrapTimestampToDateCast(Session session, TimestampType sourceType, DateType targetType, ComparisonExpression.Operator operator, Expression timestampExpression, long date) + { + ResolvedFunction targetToSource; + try { + targetToSource = plannerContext.getMetadata().getCoercion(session, targetType, sourceType); + } + catch (OperatorNotFoundException e) { + throw new TrinoException(GENERIC_INTERNAL_ERROR, e); + } + + Expression dateTimestamp = literalEncoder.toExpression(session, coerce(date, targetToSource), sourceType); + Expression nextDateTimestamp = literalEncoder.toExpression(session, coerce(date + 1, targetToSource), sourceType); + + switch (operator) { + case EQUAL: + return Optional.of( + and( + new ComparisonExpression(GREATER_THAN_OR_EQUAL, timestampExpression, dateTimestamp), + new ComparisonExpression(LESS_THAN, timestampExpression, nextDateTimestamp))); + case NOT_EQUAL: + return Optional.of( + or( + new ComparisonExpression(LESS_THAN, timestampExpression, dateTimestamp), + new ComparisonExpression(GREATER_THAN_OR_EQUAL, timestampExpression, nextDateTimestamp))); + case LESS_THAN: + return Optional.of(new ComparisonExpression(LESS_THAN, timestampExpression, dateTimestamp)); + case LESS_THAN_OR_EQUAL: + return Optional.of(new ComparisonExpression(LESS_THAN, timestampExpression, nextDateTimestamp)); + case GREATER_THAN: + return Optional.of(new ComparisonExpression(GREATER_THAN_OR_EQUAL, timestampExpression, nextDateTimestamp)); + case GREATER_THAN_OR_EQUAL: + return Optional.of(new ComparisonExpression(GREATER_THAN_OR_EQUAL, timestampExpression, dateTimestamp)); + case IS_DISTINCT_FROM: + return Optional.of( + or( + new IsNullPredicate(timestampExpression), + new ComparisonExpression(LESS_THAN, timestampExpression, dateTimestamp), + new ComparisonExpression(GREATER_THAN_OR_EQUAL, timestampExpression, nextDateTimestamp))); + } + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unsupported operator: " + operator); + } + + private Object coerce(Object value, ResolvedFunction coercion) + { + return functionInvoker.invoke(coercion, session.toConnectorSession(), value); + } + } +} diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestUnwrapCastInComparison.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestUnwrapCastInComparison.java index 2cc38c8a734a..b13f6e9ddfd0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestUnwrapCastInComparison.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestUnwrapCastInComparison.java @@ -554,6 +554,138 @@ public void testNoEffect() testUnwrap("double", "CAST(a AS INTEGER) = INTEGER '1'", "CAST(a AS INTEGER) = 1"); } + @Test + public void testUnwrapCastTimestampAsDate() + { + // equal + testUnwrap("timestamp(3)", "CAST(a AS DATE) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // not equal + testUnwrap("timestamp(3)", "CAST(a AS DATE) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // less than + testUnwrap("timestamp(3)", "CAST(a AS DATE) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000000'"); + + // less than or equal + testUnwrap("timestamp(3)", "CAST(a AS DATE) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // greater than + testUnwrap("timestamp(3)", "CAST(a AS DATE) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // greater than or equal + testUnwrap("timestamp(3)", "CAST(a AS DATE) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000000'"); + + // is distinct + testUnwrap("timestamp(3)", "CAST(a AS DATE) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // is not distinct + testUnwrap("timestamp(3)", "CAST(a AS DATE) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "CAST(a AS DATE) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "CAST(a AS DATE) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "CAST(a AS DATE) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // null date literal + testUnwrap("timestamp(3)", "CAST(a AS DATE) = NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "CAST(a AS DATE) < NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "CAST(a AS DATE) <= NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "CAST(a AS DATE) > NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "CAST(a AS DATE) >= NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "CAST(a AS DATE) IS DISTINCT FROM NULL", "NOT(CAST(a AS DATE) IS NULL)"); + + // non-optimized expression on the right + testUnwrap("timestamp(3)", "CAST(a AS DATE) = DATE '1981-06-22' + INTERVAL '2' DAY", "a >= TIMESTAMP '1981-06-24 00:00:00.000' AND a < TIMESTAMP '1981-06-25 00:00:00.000'"); + + // cast on the right + testUnwrap("timestamp(3)", "DATE '1981-06-22' = CAST(a AS DATE)", "a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + } + + @Test + public void testUnwrapConvertTimestatmpToDate() + { + // equal + testUnwrap("timestamp(3)", "date(a) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) = DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // not equal + testUnwrap("timestamp(3)", "date(a) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) <> DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // less than + testUnwrap("timestamp(3)", "date(a) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) < DATE '1981-06-22'", "a < TIMESTAMP '1981-06-22 00:00:00.000000000000'"); + + // less than or equal + testUnwrap("timestamp(3)", "date(a) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) <= DATE '1981-06-22'", "a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // greater than + testUnwrap("timestamp(3)", "date(a) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) > DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // greater than or equal + testUnwrap("timestamp(3)", "date(a) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) >= DATE '1981-06-22'", "a >= TIMESTAMP '1981-06-22 00:00:00.000000000000'"); + + // is distinct + testUnwrap("timestamp(3)", "date(a) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) IS DISTINCT FROM DATE '1981-06-22'", "a IS NULL OR a < TIMESTAMP '1981-06-22 00:00:00.000000000000' OR a >= TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // is not distinct + testUnwrap("timestamp(3)", "date(a) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + testUnwrap("timestamp(6)", "date(a) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000'"); + testUnwrap("timestamp(9)", "date(a) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000'"); + testUnwrap("timestamp(12)", "date(a) IS NOT DISTINCT FROM DATE '1981-06-22'", "(NOT a IS NULL) AND a >= TIMESTAMP '1981-06-22 00:00:00.000000000000' AND a < TIMESTAMP '1981-06-23 00:00:00.000000000000'"); + + // null date literal + testUnwrap("timestamp(3)", "date(a) = NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "date(a) < NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "date(a) <= NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "date(a) > NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "date(a) >= NULL", "CAST(NULL AS BOOLEAN)"); + testUnwrap("timestamp(3)", "date(a) IS DISTINCT FROM NULL", "NOT(CAST(a AS DATE) IS NULL)"); + + // non-optimized expression on the right + testUnwrap("timestamp(3)", "date(a) = DATE '1981-06-22' + INTERVAL '2' DAY", "a >= TIMESTAMP '1981-06-24 00:00:00.000' AND a < TIMESTAMP '1981-06-25 00:00:00.000'"); + + // cast on the right + testUnwrap("timestamp(3)", "DATE '1981-06-22' = date(a)", "a >= TIMESTAMP '1981-06-22 00:00:00.000' AND a < TIMESTAMP '1981-06-23 00:00:00.000'"); + } + private void testNoUnwrap(String inputType, String inputPredicate, String expectedCastType) { testNoUnwrap(getQueryRunner().getDefaultSession(), inputType, inputPredicate, expectedCastType); diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestUnwrapCastInComparison.java b/core/trino-main/src/test/java/io/trino/sql/query/TestUnwrapCastInComparison.java index b00c89d0dcb9..84bc3382f35e 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestUnwrapCastInComparison.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestUnwrapCastInComparison.java @@ -350,6 +350,56 @@ private void validate(Session session, String operator, String fromType, Object assertTrue(result, "Query evaluated to false: " + query); } + @Test + public void testUnwrapTimestampToDate() + { + for (String from : asList( + null, + "1981-06-21 23:59:59.999", + "1981-06-22 00:00:00.000", + "1981-06-22 00:00:00.001", + "1981-06-22 23:59:59.999", + "1981-06-23 00:00:00.000", + "1981-06-23 00:00:00.001")) { + for (String operator : COMPARISON_OPERATORS) { + for (String to : asList( + null, + "1981-06-21", + "1981-06-22", + "1981-06-23")) { + String fromLiteral = from == null ? "NULL" : format("TIMESTAMP '%s'", from); + String toLiteral = to == null ? "NULL" : format("DATE '%s'", to); + validate(operator, "timestamp(3)", fromLiteral, "date", toLiteral); + validateWithDateFunction(operator, "timestamp(3)", fromLiteral, toLiteral); + } + } + } + } + + private void validateWithDateFunction(String operator, String fromType, Object fromValue, Object toValue) + { + validateWithDateFunction(assertions.getDefaultSession(), operator, fromType, fromValue, toValue); + } + + private void validateWithDateFunction(Session session, String operator, String fromType, Object fromValue, Object toValue) + { + String query = format( + "SELECT (date(v) %s CAST(%s AS date)) " + + "IS NOT DISTINCT FROM " + + "(CAST(%s AS date) %s CAST(%s AS date)) " + + "FROM (VALUES CAST(%s AS %s)) t(v)", + operator, toValue, + fromValue, operator, toValue, + fromValue, fromType); + + boolean result = (boolean) assertions.execute(session, query) + .getMaterializedRows() + .get(0) + .getField(0); + + assertTrue(result, "Query evaluated to false: " + query); + } + private static List toLiteral(String type, List values) { return values.stream()