From 68abf43767d6f6e998373094147b7901b6222063 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Oct 2022 00:00:14 +0000 Subject: [PATCH 1/7] Better INTERVAL parsing and execution --- src/Functions/FunctionBinaryArithmetic.h | 71 ++++++++++++ src/Functions/vectorFunctions.cpp | 105 ++++++++++++++++++ src/Parsers/ExpressionListParsers.cpp | 81 ++++++++------ .../02457_tuple_of_intervals.reference | 17 +++ .../0_stateless/02457_tuple_of_intervals.sql | 21 ++++ 5 files changed, 262 insertions(+), 33 deletions(-) create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.reference create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 174e98dd81f..399cffac85e 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -708,6 +708,41 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + + /// Exactly one argument must be Date or DateTime + if (first_is_date_or_datetime == second_is_date_or_datetime) + return {}; + + if (!isTuple(type0) && !isTuple(type1)) + return {}; + + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + /// We construct another function and call it. + if constexpr (!is_plus && !is_minus) + return {}; + + if (isTuple(type0) && second_is_date_or_datetime && is_minus) + throw Exception("Wrong order of arguments for function " + String(name) + ": argument of Tuple type cannot be first", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + std::string function_name; + if (is_plus) + { + function_name = "addTupleOfIntervals"; + } + else if (is_minus) + { + function_name = "subtractTupleOfIntervals"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -906,6 +941,20 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeDateTimeTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + /// Tuple argument must be second. + if (isTuple(arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1121,6 +1170,22 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + /// Tuple argument must be second. + if (isTuple(new_arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1553,6 +1618,12 @@ public: return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 20571f67eff..4e9f2a71f8c 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -415,6 +415,108 @@ public: } }; +template +class FunctionDateOrDateTimeOperationTupleOfIntervals : public ITupleFunction +{ +public: + static constexpr auto name = Impl::name; + + explicit FunctionDateOrDateTimeOperationTupleOfIntervals(ContextPtr context_) : ITupleFunction(context_) {} + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), getName()}; + + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + + if (!cur_tuple) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}. Should be a tuple", + arguments[0].type->getName(), getName()}; + + const auto & cur_types = cur_tuple->getElements(); + + Columns cur_elements; + if (arguments[1].column) + cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_types.size(); + if (tuple_size == 0) + return arguments[0].type; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + DataTypePtr res_type = arguments[0].type; + for (size_t i = 0; i < tuple_size; ++i) + { + try + { + ColumnWithTypeAndName left{res_type, {}}; + ColumnWithTypeAndName right{cur_elements.empty() ? nullptr : cur_elements[i], cur_types[i], {}}; + auto plus_elem = plus->build({left, right}); + res_type = plus_elem->getResultType(); + } + catch (DB::Exception & e) + { + e.addMessage("While executing function {} for tuple element {}", getName(), i); + throw; + } + } + + return res_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto & cur_types = cur_tuple->getElements(); + auto cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_elements.size(); + if (tuple_size == 0) + return arguments[0].column; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + ColumnWithTypeAndName res; + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnWithTypeAndName column{cur_elements[i], cur_types[i], {}}; + auto elem_plus = plus->build(ColumnsWithTypeAndName{i == 0 ? arguments[0] : res, column}); + auto res_type = elem_plus->getResultType(); + res.column = elem_plus->execute({i == 0 ? arguments[0] : res, column}, res_type, input_rows_count); + res.type = res_type; + } + + return res.column; + } +}; + +struct AddTupleOfIntervalsImpl +{ + static constexpr auto name = "addTupleOfIntervals"; + static constexpr auto func_name = "plus"; +}; + +struct SubtractTupleOfIntervalsImpl +{ + static constexpr auto name = "subtractTupleOfIntervals"; + static constexpr auto func_name = "minus"; +}; + +using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + +using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1282,6 +1384,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f7a016a59e4..353f22b03b6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1684,6 +1684,16 @@ private: class IntervalLayer : public Layer { public: + bool getResult(ASTPtr & node) override + { + if (elements.size() == 1) + node = elements[0]; + else + node = makeASTFunction("tuple", std::move(elements)); + + return true; + } + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// INTERVAL 1 HOUR or INTERVAL expr HOUR @@ -1693,49 +1703,54 @@ public: if (state == 0) { + state = 1; + auto begin = pos; auto init_expected = expected; ASTPtr string_literal; + String literal; + //// A String literal followed INTERVAL keyword, /// the literal can be a part of an expression or /// include Number and INTERVAL TYPE at the same time - if (ParserStringLiteral{}.parse(pos, string_literal, expected)) + if (ParserStringLiteral{}.parse(pos, string_literal, expected) + && string_literal->as().value.tryGet(literal)) { - String literal; - if (string_literal->as().value.tryGet(literal)) + Tokens tokens(literal.data(), literal.data() + literal.size()); + IParser::Pos token_pos(tokens, 0); + Expected token_expected; + ASTPtr expr; + + if (!ParserNumber{}.parse(token_pos, expr, token_expected)) + return false; + + /// case: INTERVAL '1' HOUR + /// back to begin + if (!token_pos.isValid()) { - Tokens tokens(literal.data(), literal.data() + literal.size()); - IParser::Pos token_pos(tokens, 0); - Expected token_expected; - ASTPtr expr; - - if (!ParserNumber{}.parse(token_pos, expr, token_expected)) - { - return false; - } - else - { - /// case: INTERVAL '1' HOUR - /// back to begin - if (!token_pos.isValid()) - { - pos = begin; - expected = init_expected; - } - else - { - /// case: INTERVAL '1 HOUR' - if (!parseIntervalKind(token_pos, token_expected, interval_kind)) - return false; - - elements = {makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)}; - finished = true; - return true; - } - } + pos = begin; + expected = init_expected; + return true; } + + /// case: INTERVAL '1 HOUR' + if (!parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + + /// case: INTERVAL '1 HOUR 1 SECOND ...' + while (token_pos.isValid()) + { + if (!ParserNumber{}.parse(token_pos, expr, token_expected) || + !parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + } + + finished = true; } - state = 1; return true; } diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference new file mode 100644 index 00000000000..40bbfb35d91 --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -0,0 +1,17 @@ +SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) +- +2022-11-12 +2022-11-12 +2022-11-12 +- +2023-07-11 00:01:59 +2023-07-11 00:01:59 +2023-07-11 00:01:59 +- +2021-07-31 23:00:00 +2021-07-31 23:00:00 +2021-07-31 23:00:00 +- +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql new file mode 100644 index 00000000000..2c2feaf522a --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -0,0 +1,21 @@ +EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; +SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); +SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; +SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); +SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; +SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); +SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; +SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); +SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; From af1d306b12756e3c2f5d5de4bb7df0086c95ba77 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:16:12 +0000 Subject: [PATCH 2/7] Add Interval arithmetics --- src/DataTypes/IDataType.h | 1 + src/Functions/FunctionBinaryArithmetic.h | 100 +++++++- src/Functions/FunctionUnaryArithmetic.h | 10 +- src/Functions/vectorFunctions.cpp | 227 +++++++++++++++++- .../02457_tuple_of_intervals.reference | 31 ++- .../0_stateless/02457_tuple_of_intervals.sql | 53 ++-- 6 files changed, 380 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c93128ced95..45353796f3c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -408,6 +408,7 @@ inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); } inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); } inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type).isMap(); } +inline bool isInterval(const DataTypePtr & data_type) {return WhichDataType(data_type).isInterval(); } inline bool isNothing(const DataTypePtr & data_type) { return WhichDataType(data_type).isNothing(); } inline bool isUUID(const DataTypePtr & data_type) { return WhichDataType(data_type).isUUID(); } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 399cffac85e..e4919d3e9d7 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -633,7 +634,8 @@ class FunctionBinaryArithmetic : public IFunction DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256, DataTypeDate, DataTypeDateTime, - DataTypeFixedString, DataTypeString>; + DataTypeFixedString, DataTypeString, + DataTypeInterval>; using Floats = TypeList; @@ -709,10 +711,10 @@ class FunctionBinaryArithmetic : public IFunction } static FunctionOverloadResolverPtr - getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); - bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); /// Exactly one argument must be Date or DateTime if (first_is_date_or_datetime == second_is_date_or_datetime) @@ -735,7 +737,7 @@ class FunctionBinaryArithmetic : public IFunction { function_name = "addTupleOfIntervals"; } - else if (is_minus) + else { function_name = "subtractTupleOfIntervals"; } @@ -743,6 +745,47 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals + /// and the second argument is the Inteval of a different kind. + /// We construct another function (example: addIntervals) and call it + + if constexpr (!is_plus && !is_minus) + return {}; + + const auto * tuple_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_1 = checkAndGetDataType(type1.get()); + + if ((!tuple_data_type_0 && !interval_data_type_0) || !interval_data_type_1) + return {}; + + if (interval_data_type_0 && interval_data_type_0->equals(*interval_data_type_1)) + return {}; + + if (tuple_data_type_0) + { + auto & tuple_types = tuple_data_type_0->getElements(); + for (auto & type : tuple_types) + if (!isInterval(type)) + return {}; + } + + std::string function_name; + if (is_plus) + { + function_name = "addInterval"; + } + else + { + function_name = "subtractInterval"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -955,6 +998,16 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeIntervalTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1171,7 +1224,7 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -1186,6 +1239,18 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1237,6 +1302,21 @@ public: type_res = std::make_shared(); return true; } + else if constexpr (std::is_same_v || std::is_same_v) + { + if constexpr (std::is_same_v && + std::is_same_v) + { + if constexpr (is_plus || is_minus) + { + if (left.getKind() == right.getKind()) + { + type_res = std::make_shared(left.getKind()); + return true; + } + } + } + } else { using ResultDataType = typename BinaryOperationTraits::ResultDataType; @@ -1619,11 +1699,17 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) { return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeIntervalTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 445eb45fd9d..f5ddc5cb67c 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -145,7 +146,8 @@ class FunctionUnaryArithmetic : public IFunction DataTypeDecimal, DataTypeDecimal, DataTypeDecimal, - DataTypeFixedString + DataTypeFixedString, + DataTypeInterval >(type, std::forward(f)); } @@ -211,6 +213,12 @@ public: return false; result = std::make_shared(type.getN()); } + else if constexpr (std::is_same_v) + { + if constexpr (!IsUnaryOperation::negate) + return false; + result = std::make_shared(type.getKind()); + } else { using T0 = typename DataType::FieldType; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 4e9f2a71f8c..007875a8b81 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -517,6 +518,172 @@ using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfInterv using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; +template +struct FunctionTupleOperationInterval : public ITupleFunction +{ +public: + static constexpr auto name = is_minus ? "subtractInterval" : "addInterval"; + + explicit FunctionTupleOperationInterval(ContextPtr context_) : ITupleFunction(context_) {} + + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isTuple(arguments[0]) && !isInterval(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0]->getName(), getName()); + + if (!isInterval(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0]->getName(), getName()); + + DataTypes types; + + const auto * tuple = checkAndGetDataType(arguments[0].get()); + + if (tuple) + { + const auto & cur_types = tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + types.back()->getName(), getName()); + + types = cur_types; + } + else + { + types = {arguments[0]}; + } + + const auto * interval_last = checkAndGetDataType(types.back().get()); + const auto * interval_new = checkAndGetDataType(arguments[1].get()); + + if (!interval_last->equals(*interval_new)) + types.push_back(arguments[1]); + + return std::make_shared(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!isInterval(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0].type->getName(), getName()); + + Columns tuple_columns; + + const auto * first_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * first_interval = checkAndGetDataType(arguments[0].type.get()); + const auto * second_interval = checkAndGetDataType(arguments[1].type.get()); + + bool can_be_merged; + + if (first_interval) + { + can_be_merged = first_interval->equals(*second_interval); + + if (can_be_merged) + tuple_columns.resize(1); + else + tuple_columns.resize(2); + + tuple_columns[0] = arguments[0].column->convertToFullColumnIfConst(); + } + else if (first_tuple) + { + const auto & cur_types = first_tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + type->getName(), getName()); + + auto cur_elements = getTupleElements(*arguments[0].column); + size_t tuple_size = cur_elements.size(); + + if (tuple_size == 0) + { + can_be_merged = false; + } + else + { + const auto * tuple_last_interval = checkAndGetDataType(cur_types.back().get()); + can_be_merged = tuple_last_interval->equals(*second_interval); + } + + if (can_be_merged) + tuple_columns.resize(tuple_size); + else + tuple_columns.resize(tuple_size + 1); + + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = cur_elements[i]; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0].type->getName(), getName()); + + + ColumnPtr & last_column = tuple_columns.back(); + + if (can_be_merged) + { + ColumnWithTypeAndName left{last_column, arguments[1].type, {}}; + + if constexpr (is_minus) + { + auto minus = FunctionFactory::instance().get("minus", context); + auto elem_minus = minus->build({left, arguments[1]}); + last_column = elem_minus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + else + { + auto plus = FunctionFactory::instance().get("plus", context); + auto elem_plus = plus->build({left, arguments[1]}); + last_column = elem_plus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + } + else + { + if constexpr (is_minus) + { + auto negate = FunctionFactory::instance().get("negate", context); + auto elem_negate = negate->build({arguments[1]}); + last_column = elem_negate->execute({arguments[1]}, arguments[1].type, input_rows_count); + } + else + { + last_column = arguments[1].column; + } + } + + return ColumnTuple::create(tuple_columns); + } +}; + +using FunctionTupleAddInterval = FunctionTupleOperationInterval; + +using FunctionTupleSubtractInterval = FunctionTupleOperationInterval; + + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1384,8 +1551,64 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction( + { + R"( +Consecutively adds a tuple of intervals to a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT addTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Consecutively subtracts a tuple of intervals from a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT subtractTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Adds an interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); + factory.registerFunction( + { + R"( +Adds an negated interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT subtractInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT subtractInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT subtractInterval(INTERVAL 2 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index 40bbfb35d91..dd190dce891 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,17 +1,16 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) -- -2022-11-12 -2022-11-12 -2022-11-12 -- -2023-07-11 00:01:59 -2023-07-11 00:01:59 -2023-07-11 00:01:59 -- -2021-07-31 23:00:00 -2021-07-31 23:00:00 -2021-07-31 23:00:00 -- -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 +--- +3 IntervalSecond +(1,2) Tuple(IntervalHour, IntervalSecond) +(1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(2,1) Tuple(IntervalSecond, IntervalHour) +--- +-3 IntervalSecond +(-1,-2) Tuple(IntervalHour, IntervalSecond) +(-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(-2,-1) Tuple(IntervalSecond, IntervalHour) +--- +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 2c2feaf522a..d4065ab98f8 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -1,21 +1,42 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; -SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); -SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; -SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); -SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '-'; -SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; -SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); -SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; -SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); -SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); + +SELECT '---'; + +WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, + '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, + '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, + '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, + '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, + '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, + '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + + +WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, + '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, + '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, + '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file From cf6471c6b9965a82acb16b4a576e5f8a1ecf123d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:48:04 +0000 Subject: [PATCH 3/7] Fix style --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e4919d3e9d7..c13cc67a1bc 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -749,7 +749,7 @@ class FunctionBinaryArithmetic : public IFunction getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals - /// and the second argument is the Inteval of a different kind. + /// and the second argument is the Interval of a different kind. /// We construct another function (example: addIntervals) and call it if constexpr (!is_plus && !is_minus) From c11e4bfbbfbc36999814e5983e19e41b1eecd2ea Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 15:13:26 +0000 Subject: [PATCH 4/7] Fix build --- src/Functions/vectorFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 007875a8b81..78f35342d60 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -555,7 +555,7 @@ public: { const auto & cur_types = tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", @@ -607,7 +607,7 @@ public: { const auto & cur_types = first_tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", From a50c0a7f85f7cef911a946d4ba9a54b61023ad97 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 18 Oct 2022 07:23:00 +0000 Subject: [PATCH 5/7] Better test --- .../02457_tuple_of_intervals.reference | 21 +++++++--- .../0_stateless/02457_tuple_of_intervals.sql | 42 ++++++++++++++----- 2 files changed, 48 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index dd190dce891..d5ffbc33dc0 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,16 +1,27 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) --- +-1 +2022-10-12 +2022-10-10 +(2) +(0) +2022-10-12 +2022-10-10 +2022-10-12 +(2) Tuple(IntervalSecond) +(0) Tuple(IntervalSecond) +--- 3 IntervalSecond (1,2) Tuple(IntervalHour, IntervalSecond) (1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (2,1) Tuple(IntervalSecond, IntervalHour) ---- -3 IntervalSecond (-1,-2) Tuple(IntervalHour, IntervalSecond) (-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (-2,-1) Tuple(IntervalSecond, IntervalHour) --- -1 -1 -1 -1 +1 2022-03-01 +1 2022-02-28 +1 2023-07-11 00:01:59 +1 2021-07-31 23:00:00 +1 2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index d4065ab98f8..494914d4d4f 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -2,13 +2,29 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; SELECT '---'; +SELECT negate(INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); +SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); +SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; +SELECT tuple(INTERVAL 1 DAY) - '2022-10-11'::Date; -- { serverError 43 } + +WITH tuple(INTERVAL 1 SECOND) + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH tuple(INTERVAL 1 SECOND) - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } +WITH INTERVAL 1 SECOND - tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } + +SELECT '---'; + WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '---'; - WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); @@ -16,27 +32,33 @@ WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT ex SELECT '---'; -WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, - '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, - '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, - '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +WITH '2022-01-30'::Date + INTERVAL 1 MONTH + INTERVAL 1 DAY AS e1, + '2022-01-30'::Date + (INTERVAL 1 MONTH + INTERVAL 1 DAY) AS e2, + '2022-01-30'::Date + (INTERVAL 1 MONTH, INTERVAL 1 DAY) AS e3, + '2022-01-30'::Date + INTERVAL '1 MONTH 1 DAY' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-01-30'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-01-30'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-01-30'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-01-30'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; \ No newline at end of file From 6242e93c81c22a1306c4dd35c263f65e22048fe9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 24 Oct 2022 02:11:08 +0000 Subject: [PATCH 6/7] Fixes & better tests --- src/Functions/vectorFunctions.cpp | 8 ++++---- .../0_stateless/02457_tuple_of_intervals.reference | 5 +++++ tests/queries/0_stateless/02457_tuple_of_intervals.sql | 7 +++++++ 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 78f35342d60..20835f59cc1 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -434,7 +434,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. Should be a date or a date with time", arguments[0].type->getName(), getName()}; @@ -545,7 +545,7 @@ public: if (!isInterval(arguments[1])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0]->getName(), getName()); + arguments[1]->getName(), getName()); DataTypes types; @@ -559,7 +559,7 @@ public: if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", - types.back()->getName(), getName()); + type->getName(), getName()); types = cur_types; } @@ -582,7 +582,7 @@ public: if (!isInterval(arguments[1].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0].type->getName(), getName()); + arguments[1].type->getName(), getName()); Columns tuple_columns; diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index d5ffbc33dc0..e635aec1163 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -5,6 +5,11 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toInterv 2022-10-10 (2) (0) +2022-11-12 +2022-09-10 +(1,2) +(1,0) +--- 2022-10-12 2022-10-10 2022-10-12 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 494914d4d4f..be9ccb50d92 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -8,6 +8,13 @@ SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '---'; + SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; From 48cc1d8492c5838958ec14a7da6a2490ca696986 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 8 Nov 2022 00:15:19 +0100 Subject: [PATCH 7/7] Update DateLUTImpl.h --- src/Common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 3afbb6735dc..2f8aa487621 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1331,7 +1331,7 @@ public: } template - inline auto addQuarters(DateOrTime d, Int64 delta) const + inline auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const { return addMonths(d, delta * 3); }