From 317df5f5d27e02d96b83a4fb46aec78e9913579c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Dec 2020 00:57:33 +0300 Subject: [PATCH 1/2] try fix 'value is too short' when converting from String to Nullable(T) --- src/Core/callOnTypeIndex.h | 54 +++++++-------- src/Functions/FunctionsConversion.h | 102 ++++++++++++++++++++-------- src/Functions/IFunction.cpp | 35 ++++++---- src/Functions/IFunctionAdaptors.h | 2 + 4 files changed, 124 insertions(+), 69 deletions(-) diff --git a/src/Core/callOnTypeIndex.h b/src/Core/callOnTypeIndex.h index 2ef049762f1..ccbccd7105a 100644 --- a/src/Core/callOnTypeIndex.h +++ b/src/Core/callOnTypeIndex.h @@ -162,43 +162,43 @@ template class DataTypeNumber; template class DataTypeDecimal; -template -bool callOnIndexAndDataType(TypeIndex number, F && f) +template +bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args) { switch (number) { - case TypeIndex::UInt8: return f(TypePair, T>()); - case TypeIndex::UInt16: return f(TypePair, T>()); - case TypeIndex::UInt32: return f(TypePair, T>()); - case TypeIndex::UInt64: return f(TypePair, T>()); - case TypeIndex::UInt256: return f(TypePair, T>()); + case TypeIndex::UInt8: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::UInt16: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::UInt32: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::UInt64: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::UInt256: return f(TypePair, T>(), std::forward(args)...); - case TypeIndex::Int8: return f(TypePair, T>()); - case TypeIndex::Int16: return f(TypePair, T>()); - case TypeIndex::Int32: return f(TypePair, T>()); - case TypeIndex::Int64: return f(TypePair, T>()); - case TypeIndex::Int128: return f(TypePair, T>()); - case TypeIndex::Int256: return f(TypePair, T>()); + case TypeIndex::Int8: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Int16: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Int32: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Int64: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Int128: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Int256: return f(TypePair, T>(), std::forward(args)...); - case TypeIndex::Float32: return f(TypePair, T>()); - case TypeIndex::Float64: return f(TypePair, T>()); + case TypeIndex::Float32: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Float64: return f(TypePair, T>(), std::forward(args)...); - case TypeIndex::Decimal32: return f(TypePair, T>()); - case TypeIndex::Decimal64: return f(TypePair, T>()); - case TypeIndex::Decimal128: return f(TypePair, T>()); - case TypeIndex::Decimal256: return f(TypePair, T>()); + case TypeIndex::Decimal32: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Decimal64: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Decimal128: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Decimal256: return f(TypePair, T>(), std::forward(args)...); - case TypeIndex::Date: return f(TypePair()); - case TypeIndex::DateTime: return f(TypePair()); - case TypeIndex::DateTime64: return f(TypePair()); + case TypeIndex::Date: return f(TypePair(), std::forward(args)...); + case TypeIndex::DateTime: return f(TypePair(), std::forward(args)...); + case TypeIndex::DateTime64: return f(TypePair(), std::forward(args)...); - case TypeIndex::String: return f(TypePair()); - case TypeIndex::FixedString: return f(TypePair()); + case TypeIndex::String: return f(TypePair(), std::forward(args)...); + case TypeIndex::FixedString: return f(TypePair(), std::forward(args)...); - case TypeIndex::Enum8: return f(TypePair, T>()); - case TypeIndex::Enum16: return f(TypePair, T>()); + case TypeIndex::Enum8: return f(TypePair, T>(), std::forward(args)...); + case TypeIndex::Enum16: return f(TypePair, T>(), std::forward(args)...); - case TypeIndex::UUID: return f(TypePair()); + case TypeIndex::UUID: return f(TypePair(), std::forward(args)...); default: break; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 92607f718d5..911496e676d 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -108,10 +108,14 @@ struct AccurateOrNullConvertStrategyAdditions UInt32 scale { 0 }; }; + +struct ConvertDefaultBehaviorTag {}; +struct ConvertReturnNullOnErrorTag {}; + /** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment. * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) */ -template +template struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; @@ -279,7 +283,7 @@ struct ConvertImpl /** Conversion of DateTime to Date: throw off time component. */ -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl {}; @@ -301,7 +305,7 @@ struct ToDateTimeImpl } }; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl {}; /// Implementation of toDate function. @@ -357,21 +361,21 @@ struct ToDateTransform8Or16Signed * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. * (otherwise such usage would be frequent mistake). */ -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; @@ -456,9 +460,9 @@ struct ToDateTime64Transform } }; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl {}; /** Conversion of DateTime64 to Date or DateTime: discards fractional part. @@ -481,9 +485,9 @@ struct FromDateTime64Transform } }; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; -template struct ConvertImpl +template struct ConvertImpl : DateTimeTransformImpl> {}; @@ -547,7 +551,7 @@ struct FormatImpl> /// DataTypeEnum to DataType free conversion template -struct ConvertImpl, DataTypeNumber, Name> +struct ConvertImpl, DataTypeNumber, Name, ConvertDefaultBehaviorTag> { static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) { @@ -557,7 +561,7 @@ struct ConvertImpl, DataTypeNumber, Name> template -struct ConvertImpl, DataTypeString>, Name> +struct ConvertImpl, DataTypeString>, Name, ConvertDefaultBehaviorTag> { using FromFieldType = typename FromDataType::FieldType; using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; @@ -973,14 +977,24 @@ struct ConvertThroughParsing }; + + template -struct ConvertImpl, DataTypeString>, ToDataType, Name> +struct ConvertImpl, DataTypeString>, ToDataType, Name, ConvertDefaultBehaviorTag> : ConvertThroughParsing {}; template -struct ConvertImpl, DataTypeFixedString>, ToDataType, Name> +struct ConvertImpl, DataTypeFixedString>, ToDataType, Name, ConvertDefaultBehaviorTag> : ConvertThroughParsing {}; +template +struct ConvertImpl, DataTypeString>, ToDataType, Name, ConvertReturnNullOnErrorTag> + : ConvertThroughParsing {}; + +template +struct ConvertImpl, DataTypeFixedString>, ToDataType, Name, ConvertReturnNullOnErrorTag> + : ConvertThroughParsing {}; + /// Generic conversion of any type from String. Used for complex types: Array and Tuple. struct ConvertImplGenericFromString { @@ -1027,14 +1041,14 @@ struct ConvertImplGenericFromString template <> -struct ConvertImpl - : ConvertImpl {}; +struct ConvertImpl + : ConvertImpl {}; /** If types are identical, just take reference to column. */ template -struct ConvertImpl, T, Name> +struct ConvertImpl, T, Name, ConvertDefaultBehaviorTag> { template static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, @@ -1049,7 +1063,7 @@ struct ConvertImpl, T, Name> * Cutting sequences of zero bytes from end of strings. */ template -struct ConvertImpl +struct ConvertImpl { static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) { @@ -1153,6 +1167,9 @@ public: static constexpr bool to_datetime64 = std::is_same_v; + static constexpr bool to_string_or_fixed_string = std::is_same_v || + std::is_same_v; + static FunctionPtr create(const Context &) { return std::make_shared(); } static FunctionPtr create() { return std::make_shared(); } @@ -1166,6 +1183,14 @@ public: bool isInjective(const ColumnsWithTypeAndName &) const override { return std::is_same_v; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto getter = [&] (const auto & args) { return getReturnTypeImplRemovedNullable(args); }; + auto res = FunctionOverloadResolverAdaptor::getReturnTypeDefaultImplementationForNulls(arguments, getter); + to_nullable = res->isNullable(); + return res; + } + + DataTypePtr getReturnTypeImplRemovedNullable(const ColumnsWithTypeAndName & arguments) const { FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, nullptr}}; FunctionArgumentDescriptors optional_args; @@ -1243,11 +1268,17 @@ public: } } + bool useDefaultImplementationForNulls() const override { return to_nullable; } // FIXME bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } bool canBeExecutedOnDefaultArguments() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return executeImplRemovedNullable(arguments, result_type, input_rows_count); + } + + ColumnPtr executeImplRemovedNullable(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { try { @@ -1292,6 +1323,8 @@ public: } private: + mutable bool to_nullable = false; + ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { if (arguments.empty()) @@ -1301,11 +1334,12 @@ private: const IDataType * from_type = arguments[0].type.get(); ColumnPtr result_column; - auto call = [&](const auto & types) -> bool + auto call = [&](const auto & types, const auto & tag) -> bool { using Types = std::decay_t; using LeftDataType = typename Types::LeftType; using RightDataType = typename Types::RightType; + using SpecialTag = std::decay_t; if constexpr (IsDataTypeDecimal) { @@ -1349,10 +1383,10 @@ private: throw Exception("Wrong UUID conversion", ErrorCodes::CANNOT_CONVERT_TYPE); } else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); } else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count); return true; }; @@ -1365,7 +1399,7 @@ private: if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 { - if (!callOnIndexAndDataType(from_type->getTypeId(), call)) + if (!callOnIndexAndDataType(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{})) throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1373,7 +1407,19 @@ private: } } - bool done = callOnIndexAndDataType(from_type->getTypeId(), call); + bool done; + if constexpr (to_string_or_fixed_string) + { + done = callOnIndexAndDataType(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{}); + } + else + { + if (to_nullable && WhichDataType(from_type).isStringOrFixedString()) + done = callOnIndexAndDataType(from_type->getTypeId(), call, ConvertReturnNullOnErrorTag{}); + else + done = callOnIndexAndDataType(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{}); + } + if (!done) { /// Generic conversion of any type to String. diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index f51c44cf904..e4a1adb8525 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -465,25 +465,32 @@ void FunctionOverloadResolverAdaptor::checkNumberOfArguments(size_t number_of_ar ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } + +DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeDefaultImplementationForNulls(const ColumnsWithTypeAndName & arguments, + const DefaultReturnTypeGetter & getter) +{ + NullPresence null_presence = getNullPresense(arguments); + + if (null_presence.has_null_constant) + { + return makeNullable(std::make_shared()); + } + if (null_presence.has_nullable) + { + Block nested_columns = createBlockWithNestedColumns(arguments); + auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end())); + return makeNullable(return_type); + } + + return getter(arguments); +} + DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const { checkNumberOfArguments(arguments.size()); if (!arguments.empty() && impl->useDefaultImplementationForNulls()) - { - NullPresence null_presence = getNullPresense(arguments); - - if (null_presence.has_null_constant) - { - return makeNullable(std::make_shared()); - } - if (null_presence.has_nullable) - { - Block nested_columns = createBlockWithNestedColumns(arguments); - auto return_type = impl->getReturnType(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end())); - return makeNullable(return_type); - } - } + return getReturnTypeDefaultImplementationForNulls(arguments, [&](const auto & args) { return impl->getReturnType(args); }); return impl->getReturnType(arguments); } diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 737d904f6c9..cbb71c99b1b 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -129,6 +129,8 @@ public: return impl->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); } + using DefaultReturnTypeGetter = std::function; + static DataTypePtr getReturnTypeDefaultImplementationForNulls(const ColumnsWithTypeAndName & arguments, const DefaultReturnTypeGetter & getter); private: FunctionOverloadResolverImplPtr impl; From c49a6302fa5f359c7096a532197e4ec7dcd0c132 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Dec 2020 03:13:53 +0300 Subject: [PATCH 2/2] fix, add test --- src/Functions/FunctionsConversion.h | 24 +++++++----- .../01186_conversion_to_nullable.reference | 38 +++++++++++++++++++ .../01186_conversion_to_nullable.sql | 13 +++++++ 3 files changed, 65 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/01186_conversion_to_nullable.reference create mode 100644 tests/queries/0_stateless/01186_conversion_to_nullable.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 911496e676d..84a41a9c415 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -977,8 +977,6 @@ struct ConvertThroughParsing }; - - template struct ConvertImpl, DataTypeString>, ToDataType, Name, ConvertDefaultBehaviorTag> : ConvertThroughParsing {}; @@ -1044,6 +1042,9 @@ template <> struct ConvertImpl : ConvertImpl {}; +template <> +struct ConvertImpl + : ConvertImpl {}; /** If types are identical, just take reference to column. */ @@ -1187,6 +1188,7 @@ public: auto getter = [&] (const auto & args) { return getReturnTypeImplRemovedNullable(args); }; auto res = FunctionOverloadResolverAdaptor::getReturnTypeDefaultImplementationForNulls(arguments, getter); to_nullable = res->isNullable(); + checked_return_type = true; return res; } @@ -1268,17 +1270,16 @@ public: } } - bool useDefaultImplementationForNulls() const override { return to_nullable; } // FIXME + /// Function actually uses default implementation for nulls, + /// but we need to know if return type is Nullable or not, + /// so we use checked_return_type only to intercept the first call to getReturnTypeImpl(...). + bool useDefaultImplementationForNulls() const override { return checked_return_type; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } bool canBeExecutedOnDefaultArguments() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - return executeImplRemovedNullable(arguments, result_type, input_rows_count); - } - - ColumnPtr executeImplRemovedNullable(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { try { @@ -1323,6 +1324,7 @@ public: } private: + mutable bool checked_return_type = false; mutable bool to_nullable = false; ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const @@ -1359,12 +1361,12 @@ private: const ColumnWithTypeAndName & scale_column = arguments[1]; UInt32 scale = extractToDecimalScale(scale_column); - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, scale); } else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) { const auto * dt64 = assert_cast(arguments[0].type.get()); - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, dt64->getScale()); } else if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber) { @@ -1414,6 +1416,8 @@ private: } else { + /// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString) + /// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values. if (to_nullable && WhichDataType(from_type).isStringOrFixedString()) done = callOnIndexAndDataType(from_type->getTypeId(), call, ConvertReturnNullOnErrorTag{}); else diff --git a/tests/queries/0_stateless/01186_conversion_to_nullable.reference b/tests/queries/0_stateless/01186_conversion_to_nullable.reference new file mode 100644 index 00000000000..04146644154 --- /dev/null +++ b/tests/queries/0_stateless/01186_conversion_to_nullable.reference @@ -0,0 +1,38 @@ +42 +\N +0 +\N +\N +42 +\N +0 +\N +256 +2020-12-24 +\N +1970-01-01 +\N +1970-01-01 +2020-12-24 01:02:03 +\N +1970-01-01 03:00:00 +\N +2020-12-24 01:02:03.00 +\N +1970-01-01 03:00:00.00 +1970-01-01 03:00:00.00 +946721532 +\N +\N +42.00 +\N +\N +42.00000000 +\N +3.14159000 +42 +\N +test +42\0\0\0\0\0\0 +\N +test\0\0\0\0 diff --git a/tests/queries/0_stateless/01186_conversion_to_nullable.sql b/tests/queries/0_stateless/01186_conversion_to_nullable.sql new file mode 100644 index 00000000000..bf7df6234d2 --- /dev/null +++ b/tests/queries/0_stateless/01186_conversion_to_nullable.sql @@ -0,0 +1,13 @@ +select toUInt8(x) from values('x Nullable(String)', '42', NULL, '0', '', '256'); +select toInt64(x) from values('x Nullable(String)', '42', NULL, '0', '', '256'); + +select toDate(x) from values('x Nullable(String)', '2020-12-24', NULL, '0000-00-00', '', '9999-01-01'); +select toDateTime(x) from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', ''); +select toDateTime64(x, 2) from values('x Nullable(String)', '2020-12-24 01:02:03', NULL, '0000-00-00 00:00:00', ''); +select toUnixTimestamp(x) from values ('x Nullable(String)', '2000-01-01 13:12:12', NULL, ''); + +select toDecimal32(x, 2) from values ('x Nullable(String)', '42', NULL, '3.14159'); +select toDecimal64(x, 8) from values ('x Nullable(String)', '42', NULL, '3.14159'); + +select toString(x) from values ('x Nullable(String)', '42', NULL, 'test'); +select toFixedString(x, 8) from values ('x Nullable(String)', '42', NULL, 'test');