From ca6930eb110903709fc4c2e1cbec19a95e55ee18 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 4 Jul 2023 17:38:53 +0200 Subject: [PATCH 1/4] Revert "Revert "Merge pull request #50951 from ZhiguoZh/20230607-toyear-fix"" --- src/Functions/DateTimeTransforms.h | 72 +++++++ .../FunctionDateOrDateTimeToSomething.h | 13 ++ src/Functions/IFunction.h | 29 ++- src/Functions/IFunctionAdaptors.h | 7 + ...OrDateTimeConverterWithPreimageVisitor.cpp | 199 ++++++++++++++++++ ...teOrDateTimeConverterWithPreimageVisitor.h | 37 ++++ src/Interpreters/TreeOptimizer.cpp | 19 ++ ...783_date_predicate_optimizations.reference | 52 +++++ .../02783_date_predicate_optimizations.sql | 76 +++++++ ...dicate_optimizations_ast_rewrite.reference | 87 ++++++++ ...te_predicate_optimizations_ast_rewrite.sql | 47 +++++ 11 files changed, 632 insertions(+), 6 deletions(-) create mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp create mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h create mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference create mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 019e0c42cde..84c71c89b11 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -322,6 +322,7 @@ struct ToTimeImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -393,6 +394,7 @@ struct ToStartOfSecondImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -440,6 +442,7 @@ struct ToStartOfMillisecondImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -483,6 +486,7 @@ struct ToStartOfMicrosecondImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -520,6 +524,7 @@ struct ToStartOfNanosecondImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -718,6 +723,28 @@ struct ToYearImpl return time_zone.toYear(DayNum(d)); } + static inline constexpr bool hasPreimage() { return true; } + + static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) + { + if (point.getType() != Field::Types::UInt64) return std::nullopt; + + auto year = point.get(); + if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; + + const DateLUTImpl & date_lut = DateLUT::instance(); + + auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); + auto end_time = date_lut.addYears(start_time, 1); + + if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) + return {std::make_pair(Field(start_time), Field(end_time))}; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", + type.getName(), name); + } + using FactorTransform = ZeroTransform; }; @@ -791,6 +818,7 @@ struct ToQuarterImpl { return time_zone.toQuarter(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -815,6 +843,7 @@ struct ToMonthImpl { return time_zone.toMonth(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -840,6 +869,7 @@ struct ToDayOfMonthImpl return time_zone.toDayOfMonth(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMonthImpl; }; @@ -887,6 +917,7 @@ struct ToDayOfYearImpl { return time_zone.toDayOfYear(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -911,6 +942,7 @@ struct ToHourImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -939,6 +971,7 @@ struct TimezoneOffsetImpl throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToTimeImpl; }; @@ -962,6 +995,7 @@ struct ToMinuteImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfHourImpl; }; @@ -986,6 +1020,7 @@ struct ToSecondImpl { throwDateTimeIsNotSupported(name); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMinuteImpl; }; @@ -1010,6 +1045,7 @@ struct ToISOYearImpl { return time_zone.toISOYear(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1066,6 +1102,7 @@ struct ToISOWeekImpl { return time_zone.toISOWeek(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToISOYearImpl; }; @@ -1108,6 +1145,7 @@ struct ToRelativeYearNumImpl { return time_zone.toYear(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1139,6 +1177,7 @@ struct ToRelativeQuarterNumImpl { return time_zone.toRelativeQuarterNum(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1170,6 +1209,7 @@ struct ToRelativeMonthNumImpl { return time_zone.toRelativeMonthNum(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1201,6 +1241,7 @@ struct ToRelativeWeekNumImpl { return time_zone.toRelativeWeekNum(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1232,6 +1273,7 @@ struct ToRelativeDayNumImpl { return static_cast(d); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1269,6 +1311,7 @@ struct ToRelativeHourNumImpl else return static_cast(time_zone.toRelativeHourNum(DayNum(d))); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1300,6 +1343,7 @@ struct ToRelativeMinuteNumImpl { return static_cast(time_zone.toRelativeMinuteNum(DayNum(d))); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1328,6 +1372,7 @@ struct ToRelativeSecondNumImpl { return static_cast(time_zone.fromDayNum(DayNum(d))); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1352,6 +1397,31 @@ struct ToYYYYMMImpl { return time_zone.toNumYYYYMM(DayNum(d)); } + static inline constexpr bool hasPreimage() { return true; } + + static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) + { + if (point.getType() != Field::Types::UInt64) return std::nullopt; + + auto year_month = point.get(); + auto year = year_month / 100; + auto month = year_month % 100; + + if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12)) + return std::nullopt; + + const DateLUTImpl & date_lut = DateLUT::instance(); + + auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); + auto end_time = date_lut.addMonths(start_time, 1); + + if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) + return {std::make_pair(Field(start_time), Field(end_time))}; + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", + type.getName(), name); + } using FactorTransform = ZeroTransform; }; @@ -1376,6 +1446,7 @@ struct ToYYYYMMDDImpl { return time_zone.toNumYYYYMMDD(DayNum(d)); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1400,6 +1471,7 @@ struct ToYYYYMMDDhhmmssImpl { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d))); } + static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index 82818cc3d2b..d98b788c7d7 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -7,6 +7,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NOT_IMPLEMENTED; } /// See DateTimeTransforms.h @@ -83,6 +84,18 @@ public: arguments[0].type->getName(), this->getName()); } + bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); } + + RangeOrNull getPreimage(const IDataType & type, const Field & point) const override + { + if constexpr (Transform::hasPreimage()) + return Transform::getPreimage(type, point); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Function {} has no information about its preimage", + Transform::name); + } + }; } diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index c5b9a78015d..433cb61d04e 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -11,11 +13,6 @@ #include -#if USE_EMBEDDED_COMPILER -# include -#endif - - /// This file contains user interface for functions. namespace llvm @@ -35,7 +32,8 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -class Field; +/// A left-closed and right-open interval representing the preimage of a function. +using RangeOrNull = std::optional>; /// The simplest executable object. /// Motivation: @@ -233,6 +231,12 @@ public: */ virtual bool hasInformationAboutMonotonicity() const { return false; } + /** Lets you know if the function has its definition of preimage. + * This is used to work with predicate optimizations, where the comparison between + * f(x) and a constant c could be converted to the comparison between x and f's preimage [b, e). + */ + virtual bool hasInformationAboutPreimage() const { return false; } + struct ShortCircuitSettings { /// Should we enable lazy execution for the first argument of short-circuit function? @@ -286,6 +290,14 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } + /** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage. + * std::nullopt might be returned if the point (a single value) is invalid for this function. + */ + virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); + } + }; using FunctionBasePtr = std::shared_ptr; @@ -475,12 +487,17 @@ public: virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0; virtual bool hasInformationAboutMonotonicity() const { return false; } + virtual bool hasInformationAboutPreimage() const { return false; } using Monotonicity = IFunctionBase::Monotonicity; virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } + virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); + } /// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored). virtual size_t getNumberOfArguments() const = 0; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 23725b1a8b1..123fdbc2f50 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -90,10 +90,17 @@ public: bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } + bool hasInformationAboutPreimage() const override { return function->hasInformationAboutPreimage(); } + Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { return function->getMonotonicityForRange(type, left, right); } + + RangeOrNull getPreimage(const IDataType & type, const Field & point) const override + { + return function->getPreimage(type, point); + } private: std::shared_ptr function; DataTypes arguments; diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp new file mode 100644 index 00000000000..a377bb4bba6 --- /dev/null +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -0,0 +1,199 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/** Given a monotonic non-decreasing function f(x), which satisfies f(x) = c for any value x within [b, e). + * We could convert it into its equivalent form, x >= b AND x < e, which is free from the invocation of the function. + * And we could apply the similar transformation to other comparisons. The suggested transformations list: + * + * f(x) == c -> x >= b AND x < e + * f(x) != c -> x < b OR x >= e + * f(x) > c -> x >= e + * f(x) >= c -> x >= b + * f(x) < c -> x < b + * f(x) <= c -> x < e + * + * This function generates a new AST with the transformed relation. + */ +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTypePair & column, const std::pair& range) +{ + const DateLUTImpl & date_lut = DateLUT::instance(); + + const String & column_name = column.name; + String start_date_or_date_time; + String end_date_or_date_time; + + if (isDateOrDate32(column.type.get())) + { + start_date_or_date_time = date_lut.dateToString(range.first.get()); + end_date_or_date_time = date_lut.dateToString(range.second.get()); + } + else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) + { + start_date_or_date_time = date_lut.timeToString(range.first.get()); + end_date_or_date_time = date_lut.timeToString(range.second.get()); + } + else [[unlikely]] return {}; + + if (comparator == "equals") + { + return makeASTFunction("and", + makeASTFunction("greaterOrEquals", + std::make_shared(column_name), + std::make_shared(start_date_or_date_time) + ), + makeASTFunction("less", + std::make_shared(column_name), + std::make_shared(end_date_or_date_time) + ) + ); + } + else if (comparator == "notEquals") + { + return makeASTFunction("or", + makeASTFunction("less", + std::make_shared(column_name), + std::make_shared(start_date_or_date_time) + ), + makeASTFunction("greaterOrEquals", + std::make_shared(column_name), + std::make_shared(end_date_or_date_time) + ) + ); + } + else if (comparator == "greater") + { + return makeASTFunction("greaterOrEquals", + std::make_shared(column_name), + std::make_shared(end_date_or_date_time) + ); + } + else if (comparator == "lessOrEquals") + { + return makeASTFunction("less", + std::make_shared(column_name), + std::make_shared(end_date_or_date_time) + ); + } + else if (comparator == "less" || comparator == "greaterOrEquals") + { + return makeASTFunction(comparator, + std::make_shared(column_name), + std::make_shared(start_date_or_date_time) + ); + } + else [[unlikely]] + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}", + comparator); + } +} + +void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction & function, ASTPtr & ast, const Data & data) +{ + const static std::unordered_map swap_relations = { + {"equals", "equals"}, + {"notEquals", "notEquals"}, + {"less", "greater"}, + {"greater", "less"}, + {"lessOrEquals", "greaterOrEquals"}, + {"greaterOrEquals", "lessOrEquals"}, + }; + + if (!swap_relations.contains(function.name)) return; + + if (!function.arguments || function.arguments->children.size() != 2) return; + + size_t func_id = function.arguments->children.size(); + + for (size_t i = 0; i < function.arguments->children.size(); i++) + { + if (const auto * func = function.arguments->children[i]->as()) + { + func_id = i; + } + } + + if (func_id == function.arguments->children.size()) return; + + size_t literal_id = 1 - func_id; + const auto * literal = function.arguments->children[literal_id]->as(); + + if (!literal || literal->value.getType() != Field::Types::UInt64) return; + + String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); + + const auto * ast_func = function.arguments->children[func_id]->as(); + /// Currently we only handle single-argument functions. + if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return; + + const auto * column_id = ast_func->arguments->children.at(0)->as(); + if (!column_id) return; + + auto pos = IdentifierSemantic::getMembership(*column_id); + if (!pos) + pos = IdentifierSemantic::chooseTableColumnMatch(*column_id, data.tables, true); + if (!pos) + return; + + if (*pos >= data.tables.size()) + return; + + auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); + if (!data_type_and_name) return; + + const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); + if (!converter) return; + + ColumnsWithTypeAndName args; + args.emplace_back(data_type_and_name->type, "tmp"); + auto converter_base = converter->build(args); + if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; + + auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value); + if (!preimage_range) return; + + const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); + if (!new_ast) return; + + ast = new_ast; +} + +bool OptimizeDateOrDateTimeConverterWithPreimageMatcher::needChildVisit(ASTPtr & ast, ASTPtr & /*child*/) +{ + const static std::unordered_set relations = { + "equals", + "notEquals", + "less", + "greater", + "lessOrEquals", + "greaterOrEquals", + }; + + if (const auto * ast_function = ast->as()) + { + return !relations.contains(ast_function->name); + } + + return true; +} + +} diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h new file mode 100644 index 00000000000..778fa462364 --- /dev/null +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTFunction; + +/** Replace predicate having Date/DateTime converters with their preimages to improve performance. + * Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01' + * Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'. + * The similar optimization also applies to other converters. + */ +class OptimizeDateOrDateTimeConverterWithPreimageMatcher +{ +public: + struct Data + { + const TablesWithColumns & tables; + ContextPtr context; + }; + + static void visit(ASTPtr & ast, Data & data) + { + if (const auto * ast_function = ast->as()) + visit(*ast_function, ast, data); + } + + static void visit(const ASTFunction & function, ASTPtr & ast, const Data & data); + + static bool needChildVisit(ASTPtr & ast, ASTPtr & child); +}; + +using OptimizeDateOrDateTimeConverterWithPreimageVisitor = InDepthNodeVisitor; +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c38b3c79026..fd4d2c9d846 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -677,6 +678,21 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } +void optimizeDateFilters(ASTSelectQuery * select_query, const std::vector & tables_with_columns, ContextPtr context) +{ + /// Predicates in HAVING clause has been moved to WHERE clause. + if (select_query->where()) + { + OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; + OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refWhere()); + } + if (select_query->prewhere()) + { + OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; + OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refPrewhere()); + } +} + void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -780,6 +796,9 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } + /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc. + optimizeDateFilters(select_query, tables_with_columns, context); + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference index cd689b93034..872a5dd1d7d 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -1,2 +1,54 @@ 2021-12-31 23:00:00 0 2021-12-31 23:00:00 0 +Date +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 +DateTime +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 +Date32 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 +DateTime64 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql index abb13f1005e..0a2fa6cc93b 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -11,3 +11,79 @@ INSERT INTO source values ('2021-12-31 23:00:00', 0); SELECT * FROM source WHERE toYYYYMM(ts) = 202112; SELECT * FROM source WHERE toYear(ts) = 2021; + +DROP TABLE IF EXISTS source; +CREATE TABLE source +( + `dt` Date, + `ts` DateTime, + `dt_32` Date32, + `ts_64` DateTime64(3), + `n` Int32 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(ts) +ORDER BY tuple(); + +INSERT INTO source values ('2022-12-31', '2022-12-31 23:59:59', '2022-12-31', '2022-12-31 23:59:59.123', 0); +INSERT INTO source values ('2023-01-01', '2023-01-01 00:00:00', '2023-01-01', '2023-01-01 00:00:00.000', 1); +INSERT INTO source values ('2023-12-01', '2023-12-01 00:00:00', '2023-12-01', '2023-12-01 00:00:00.000', 2); +INSERT INTO source values ('2023-12-31', '2023-12-31 23:59:59', '2023-12-31', '2023-12-31 23:59:59.123', 3); +INSERT INTO source values ('2024-01-01', '2024-01-01 00:00:00', '2024-01-01', '2024-01-01 00:00:00.000', 4); + +SELECT 'Date'; +SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312; +SELECT count(*) FROM source WHERE toYear(dt) = 2023; +SELECT count(*) FROM source WHERE toYear(dt) <> 2023; +SELECT count(*) FROM source WHERE toYear(dt) < 2023; +SELECT count(*) FROM source WHERE toYear(dt) <= 2023; +SELECT count(*) FROM source WHERE toYear(dt) > 2023; +SELECT count(*) FROM source WHERE toYear(dt) >= 2023; + +SELECT 'DateTime'; +SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312; +SELECT count(*) FROM source WHERE toYear(ts) = 2023; +SELECT count(*) FROM source WHERE toYear(ts) <> 2023; +SELECT count(*) FROM source WHERE toYear(ts) < 2023; +SELECT count(*) FROM source WHERE toYear(ts) <= 2023; +SELECT count(*) FROM source WHERE toYear(ts) > 2023; +SELECT count(*) FROM source WHERE toYear(ts) >= 2023; + +SELECT 'Date32'; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312; +SELECT count(*) FROM source WHERE toYear(dt_32) = 2023; +SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023; +SELECT count(*) FROM source WHERE toYear(dt_32) < 2023; +SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023; +SELECT count(*) FROM source WHERE toYear(dt_32) > 2023; +SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023; + +SELECT 'DateTime64'; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312; +SELECT count(*) FROM source WHERE toYear(ts_64) = 2023; +SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023; +SELECT count(*) FROM source WHERE toYear(ts_64) < 2023; +SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023; +SELECT count(*) FROM source WHERE toYear(ts_64) > 2023; +SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023; +DROP TABLE source; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference new file mode 100644 index 00000000000..9235e7e106a --- /dev/null +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference @@ -0,0 +1,87 @@ +SELECT value1 +FROM date_t +WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 < \'1993-01-01\') OR (date1 >= \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 >= \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 < \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1998-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) OR ((date1 >= \'1994-01-01\') AND (date1 < \'1995-01-01\'))) AND ((id >= 1) AND (id <= 3)) +SELECT + value1, + toYear(date1) AS year1 +FROM date_t +WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +PREWHERE (date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\') +WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) +SELECT value1 +FROM date_t +WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 >= \'1992-03-01\') AND (date1 < \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 < \'1992-03-01\') OR (date1 >= \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 >= \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 < \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date_t +WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\'))) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM datetime_t +WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM datetime_t +WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date32_t +WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM date32_t +WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM datetime64_t +WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +SELECT value1 +FROM datetime64_t +WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql new file mode 100644 index 00000000000..266be59b0a3 --- /dev/null +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS date_t; +CREATE TABLE date_t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; +DROP TABLE date_t; + +DROP TABLE IF EXISTS datetime_t; +CREATE TABLE datetime_t (id UInt32, value1 String, date1 Datetime) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +DROP TABLE datetime_t; + +DROP TABLE IF EXISTS date32_t; +CREATE TABLE date32_t (id UInt32, value1 String, date1 Date32) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +DROP TABLE date32_t; + +DROP TABLE IF EXISTS datetime64_t; +CREATE TABLE datetime64_t (id UInt32, value1 String, date1 Datetime64) ENGINE ReplacingMergeTree() ORDER BY id; + +EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +DROP TABLE datetime64_t; From 7f1ee68c87160089d70f4cef04c975c38b01218e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 4 Jul 2023 23:08:54 +0200 Subject: [PATCH 2/4] refine --- src/Functions/DateTimeTransforms.h | 8 +++---- src/Functions/IFunction.h | 4 ++++ ...OrDateTimeConverterWithPreimageVisitor.cpp | 21 +++++++++---------- 3 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 84c71c89b11..e59a9046277 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -734,11 +734,11 @@ struct ToYearImpl const DateLUTImpl & date_lut = DateLUT::instance(); - auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); + auto start_time = date_lut.makeDayNum(year, 1, 1); auto end_time = date_lut.addYears(start_time, 1); if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; + return {std::make_pair(Field(Int32(start_time)), Field(Int32(end_time)))}; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", @@ -1412,11 +1412,11 @@ struct ToYYYYMMImpl const DateLUTImpl & date_lut = DateLUT::instance(); - auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); + auto start_time = date_lut.makeDayNum(year, month, 1); auto end_time = date_lut.addMonths(start_time, 1); if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; + return {std::make_pair(Field(Int32(start_time)), Field(Int32(end_time)))}; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 433cb61d04e..928475652f4 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -11,6 +11,10 @@ #include "config.h" +#if USE_EMBEDDED_COMPILER +# include +#endif + #include /// This file contains user interface for functions. diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index a377bb4bba6..9c2fdf6dee9 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -4,6 +4,7 @@ #include #include #include +#include "base/DayNum.h" #include #include #include @@ -37,20 +38,18 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTy const DateLUTImpl & date_lut = DateLUT::instance(); const String & column_name = column.name; - String start_date_or_date_time; - String end_date_or_date_time; - if (isDateOrDate32(column.type.get())) + auto start_date = range.first.get(); + auto end_date = range.second.get(); + String start_date_or_date_time = date_lut.dateToString(ExtendedDayNum(static_cast(start_date))); + String end_date_or_date_time = date_lut.dateToString(ExtendedDayNum(static_cast(end_date))); + + if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); + start_date_or_date_time += " 00:00:00"; + end_date_or_date_time += " 00:00:00"; } - else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) - { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); - } - else [[unlikely]] return {}; + else if (!isDateOrDate32(column.type.get())) return {}; if (comparator == "equals") { From 9544c035b9d8b4646defd770b829715043b145d7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 5 Jul 2023 11:15:31 +0200 Subject: [PATCH 3/4] Revert "refine" This reverts commit 7f1ee68c87160089d70f4cef04c975c38b01218e. --- src/Functions/DateTimeTransforms.h | 8 +++---- src/Functions/IFunction.h | 4 ---- ...OrDateTimeConverterWithPreimageVisitor.cpp | 21 ++++++++++--------- 3 files changed, 15 insertions(+), 18 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index e59a9046277..84c71c89b11 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -734,11 +734,11 @@ struct ToYearImpl const DateLUTImpl & date_lut = DateLUT::instance(); - auto start_time = date_lut.makeDayNum(year, 1, 1); + auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); auto end_time = date_lut.addYears(start_time, 1); if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(Int32(start_time)), Field(Int32(end_time)))}; + return {std::make_pair(Field(start_time), Field(end_time))}; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", @@ -1412,11 +1412,11 @@ struct ToYYYYMMImpl const DateLUTImpl & date_lut = DateLUT::instance(); - auto start_time = date_lut.makeDayNum(year, month, 1); + auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); auto end_time = date_lut.addMonths(start_time, 1); if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(Int32(start_time)), Field(Int32(end_time)))}; + return {std::make_pair(Field(start_time), Field(end_time))}; else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 928475652f4..433cb61d04e 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -11,10 +11,6 @@ #include "config.h" -#if USE_EMBEDDED_COMPILER -# include -#endif - #include /// This file contains user interface for functions. diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index 9c2fdf6dee9..a377bb4bba6 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -4,7 +4,6 @@ #include #include #include -#include "base/DayNum.h" #include #include #include @@ -38,18 +37,20 @@ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTy const DateLUTImpl & date_lut = DateLUT::instance(); const String & column_name = column.name; + String start_date_or_date_time; + String end_date_or_date_time; - auto start_date = range.first.get(); - auto end_date = range.second.get(); - String start_date_or_date_time = date_lut.dateToString(ExtendedDayNum(static_cast(start_date))); - String end_date_or_date_time = date_lut.dateToString(ExtendedDayNum(static_cast(end_date))); - - if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) + if (isDateOrDate32(column.type.get())) { - start_date_or_date_time += " 00:00:00"; - end_date_or_date_time += " 00:00:00"; + start_date_or_date_time = date_lut.dateToString(range.first.get()); + end_date_or_date_time = date_lut.dateToString(range.second.get()); } - else if (!isDateOrDate32(column.type.get())) return {}; + else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) + { + start_date_or_date_time = date_lut.timeToString(range.first.get()); + end_date_or_date_time = date_lut.timeToString(range.second.get()); + } + else [[unlikely]] return {}; if (comparator == "equals") { From 2e5643cc4133f207b46534a4cf8a7875d7c18a8e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 5 Jul 2023 11:57:18 +0200 Subject: [PATCH 4/4] use UTC LUT --- src/Functions/DateTimeTransforms.h | 4 ++-- src/Functions/IFunction.h | 4 ++++ .../OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 84c71c89b11..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -732,7 +732,7 @@ struct ToYearImpl auto year = point.get(); if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; - const DateLUTImpl & date_lut = DateLUT::instance(); + const DateLUTImpl & date_lut = DateLUT::instance("UTC"); auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); auto end_time = date_lut.addYears(start_time, 1); @@ -1410,7 +1410,7 @@ struct ToYYYYMMImpl if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12)) return std::nullopt; - const DateLUTImpl & date_lut = DateLUT::instance(); + const DateLUTImpl & date_lut = DateLUT::instance("UTC"); auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); auto end_time = date_lut.addMonths(start_time, 1); diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 433cb61d04e..09758d59e4a 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -13,6 +13,10 @@ #include +#if USE_EMBEDDED_COMPILER +# include +#endif + /// This file contains user interface for functions. namespace llvm diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index a377bb4bba6..6a9251cec49 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -34,7 +34,7 @@ namespace ErrorCodes */ ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTypePair & column, const std::pair& range) { - const DateLUTImpl & date_lut = DateLUT::instance(); + const DateLUTImpl & date_lut = DateLUT::instance("UTC"); const String & column_name = column.name; String start_date_or_date_time;