From 175b94fa2dd1316c5cd4df3f33bcd7fd68a262a7 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 18 Aug 2021 23:34:07 +0300 Subject: [PATCH] Special case of conversion code for types with custom serialization/deserialization Fixes CASTing from String or FixedString to IPv4 or IPv6 and back. --- src/Columns/ColumnStringHelpers.h | 88 ++++++++++++ src/DataTypes/DataTypeDecimalBase.h | 1 + src/DataTypes/DataTypeEnum.h | 3 +- src/DataTypes/DataTypeFixedString.h | 7 +- src/DataTypes/DataTypeNumberBase.h | 1 + src/DataTypes/DataTypeString.h | 3 + src/DataTypes/DataTypeUUID.h | 3 +- src/Functions/FunctionUnixTimestamp64.h | 2 +- src/Functions/FunctionsConversion.h | 125 +++++++++++------- ...ipv4_and_ipv6_to_and_from_string.reference | 6 + ...02007_ipv4_and_ipv6_to_and_from_string.sql | 8 ++ 11 files changed, 192 insertions(+), 55 deletions(-) create mode 100644 src/Columns/ColumnStringHelpers.h create mode 100644 tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.reference create mode 100644 tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.sql diff --git a/src/Columns/ColumnStringHelpers.h b/src/Columns/ColumnStringHelpers.h new file mode 100644 index 00000000000..9a7b21a8892 --- /dev/null +++ b/src/Columns/ColumnStringHelpers.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int TOO_LARGE_STRING_SIZE; +} + +namespace ColumnStringHelpers +{ + +/** Simplifies writing data to the ColumnString or ColumnFixedString via WriteBuffer. + * + * Take care of little subtle details, like padding or proper offsets. + */ +template +class WriteHelper +{ + ColumnType & col; + WriteBufferFromVector buffer; + size_t prev_row_buffer_size = 0; + +public: + WriteHelper(ColumnType & col_, size_t expected_rows, size_t expected_row_size [[maybe_unused]] = 0) + : col(col_), + buffer(col.getChars()) + { + if constexpr (std::is_same_v) + col.reserve(expected_rows); + else + { + if (const size_t estimated_total_size = expected_rows * expected_row_size) + col.reserve(estimated_total_size); + } + } + + ~WriteHelper() + { + if (buffer.count()) + rowWritten(); + + buffer.finalize(); + } + + auto & getWriteBuffer() + { + return buffer; + } + + inline void rowWritten() + { + if constexpr (std::is_same_v) + { + if (buffer.count() > prev_row_buffer_size + col.getN()) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string for FixedString column"); + + // Pad with zeroes on the right to maintain FixedString invariant. + const auto excess_bytes = buffer.count() % col.getN(); + const auto fill_bytes = col.getN() - excess_bytes; + for (size_t i = 0; i < fill_bytes; ++i) + buffer.write('\0'); + } + else + { + writeChar(0, buffer); + col.getOffsets().push_back(buffer.count()); + } + + prev_row_buffer_size = buffer.count(); + } +}; + +} + +} diff --git a/src/DataTypes/DataTypeDecimalBase.h b/src/DataTypes/DataTypeDecimalBase.h index c0585095eeb..dc8c99b06bc 100644 --- a/src/DataTypes/DataTypeDecimalBase.h +++ b/src/DataTypes/DataTypeDecimalBase.h @@ -59,6 +59,7 @@ class DataTypeDecimalBase : public IDataType public: using FieldType = T; using ColumnType = ColumnDecimal; + static constexpr auto type_id = TypeId; static constexpr bool is_parametric = true; diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 92c72b87afa..2f607fc2aa6 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -38,6 +38,7 @@ class DataTypeEnum final : public IDataTypeEnum, public EnumValues public: using FieldType = Type; using ColumnType = ColumnVector; + static constexpr auto type_id = sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; using typename EnumValues::Values; static constexpr bool is_parametric = true; @@ -52,7 +53,7 @@ public: std::string doGetName() const override { return type_name; } const char * getFamilyName() const override; - TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; } + TypeIndex getTypeId() const override { return type_id; } FieldType readValue(ReadBuffer & istr) const { diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index d82ea9824f3..505d6c30b18 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -9,6 +9,8 @@ namespace DB { +class ColumnFixedString; + namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; @@ -21,7 +23,10 @@ private: size_t n; public: + using ColumnType = ColumnFixedString; + static constexpr bool is_parametric = true; + static constexpr auto type_id = TypeIndex::FixedString; DataTypeFixedString(size_t n_) : n(n_) { @@ -32,7 +37,7 @@ public: } std::string doGetName() const override; - TypeIndex getTypeId() const override { return TypeIndex::FixedString; } + TypeIndex getTypeId() const override { return type_id; } const char * getFamilyName() const override { return "FixedString"; } diff --git a/src/DataTypes/DataTypeNumberBase.h b/src/DataTypes/DataTypeNumberBase.h index c24778691d9..44e80030f1b 100644 --- a/src/DataTypes/DataTypeNumberBase.h +++ b/src/DataTypes/DataTypeNumberBase.h @@ -20,6 +20,7 @@ class DataTypeNumberBase : public IDataType public: static constexpr bool is_parametric = false; static constexpr auto family_name = TypeName; + static constexpr auto type_id = TypeId; using FieldType = T; using ColumnType = ColumnVector; diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index 0fc38e9c6f0..e4f055670a6 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -8,10 +8,13 @@ namespace DB { +class ColumnString; + class DataTypeString final : public IDataType { public: using FieldType = String; + using ColumnType = ColumnString; static constexpr bool is_parametric = false; static constexpr auto type_id = TypeIndex::String; diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index 5ed7a912607..af9f1f35ca5 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -15,9 +15,10 @@ public: using FieldType = UUID; using ColumnType = ColumnVector; + static constexpr auto type_id = TypeIndex::UUID; const char * getFamilyName() const override { return "UUID"; } - TypeIndex getTypeId() const override { return TypeIndex::UUID; } + TypeIndex getTypeId() const override { return type_id; } Field getDefault() const override; diff --git a/src/Functions/FunctionUnixTimestamp64.h b/src/Functions/FunctionUnixTimestamp64.h index 2e631ea22f0..94718cf7a48 100644 --- a/src/Functions/FunctionUnixTimestamp64.h +++ b/src/Functions/FunctionUnixTimestamp64.h @@ -56,7 +56,7 @@ public: const auto & source_data = typeid_cast &>(col).getData(); - Int32 scale_diff = typeid_cast(*src.type).getScale() - target_scale; + const Int32 scale_diff = typeid_cast(*src.type).getScale() - target_scale; if (scale_diff == 0) { for (size_t i = 0; i < input_rows_count; ++i) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6ba2c7e02a5..6ea1299e1c8 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -839,39 +840,42 @@ struct ConvertImpl struct ConvertImplGenericToString { - static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) + static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) { + static_assert(std::is_same_v || std::is_same_v, + "Can be used only to serialize to ColumnString or ColumnFixedString"); + ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); const auto & col_with_type_and_name = columnGetNested(arguments[0]); const IDataType & type = *col_with_type_and_name.type; const IColumn & col_from = *col_with_type_and_name.column; - size_t size = col_from.size(); + auto col_to = result_type->createColumn(); - auto col_to = ColumnString::create(); - - ColumnString::Chars & data_to = col_to->getChars(); - ColumnString::Offsets & offsets_to = col_to->getOffsets(); - - data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitrary. - offsets_to.resize(size); - - WriteBufferFromVector write_buffer(data_to); - - FormatSettings format_settings; - auto serialization = type.getDefaultSerialization(); - for (size_t i = 0; i < size; ++i) { - serialization->serializeText(col_from, i, write_buffer, format_settings); - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); - } + // 2 is arbitrary + const size_t estimated_value_size_bytes = arguments[0].type->haveMaximumSizeOfValue() ? arguments[0].type->getMaximumSizeOfValueInMemory() : 2; - write_buffer.finalize(); + ColumnStringHelpers::WriteHelper write_helper( + assert_cast(*col_to), + input_rows_count, + estimated_value_size_bytes); + + auto & write_buffer = write_helper.getWriteBuffer(); + + FormatSettings format_settings; + auto serialization = type.getDefaultSerialization(); + for (size_t i = 0; i < input_rows_count; ++i) + { + serialization->serializeText(col_from, i, write_buffer, format_settings); + write_helper.rowWritten(); + } + } if (result_type->isNullable() && null_map) return ColumnNullable::create(std::move(col_to), std::move(null_map)); @@ -995,7 +999,8 @@ inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & else message_buf << " at begin of string"; - if (isNativeNumber(to_type)) + // Currently there are no functions toIPv{4,6}Or{Null,Zero} + if (isNativeNumber(to_type) && !(to_type.getName() == "IPv4" || to_type.getName() == "IPv6")) message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT); @@ -1274,40 +1279,35 @@ template struct ConvertImpl, DataTypeFixedString>, ToDataType, Name, ConvertReturnNullOnErrorTag> : ConvertThroughParsing {}; -/// Generic conversion of any type from String. Used for complex types: Array and Tuple. +/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization. +template struct ConvertImplGenericFromString { - static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) + static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) { + static_assert(std::is_same_v || std::is_same_v, + "Can be used only to parse from ColumnString or ColumnFixedString"); + const IColumn & col_from = *arguments[0].column; - size_t size = col_from.size(); - const IDataType & data_type_to = *result_type; - - if (const ColumnString * col_from_string = checkAndGetColumn(&col_from)) + if (const StringColumnType * col_from_string = checkAndGetColumn(&col_from)) { auto res = data_type_to.createColumn(); IColumn & column_to = *res; - column_to.reserve(size); - - const ColumnString::Chars & chars = col_from_string->getChars(); - const IColumn::Offsets & offsets = col_from_string->getOffsets(); - - size_t current_offset = 0; + column_to.reserve(input_rows_count); FormatSettings format_settings; auto serialization = data_type_to.getDefaultSerialization(); - for (size_t i = 0; i < size; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { - ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1); + const auto & val = col_from_string->getDataAt(i); + ReadBufferFromMemory read_buffer(val.data, val.size); serialization->deserializeWholeText(column_to, read_buffer, format_settings); if (!read_buffer.eof()) throwExceptionForIncompletelyParsedValue(read_buffer, result_type); - - current_offset = offsets[i]; } return res; @@ -1756,7 +1756,7 @@ private: /// Generic conversion of any type to String. if (std::is_same_v) { - return ConvertImplGenericToString::execute(arguments, result_type); + return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count); } else throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), @@ -2710,10 +2710,7 @@ private: /// Conversion from String through parsing. if (checkAndGetDataType(from_type_untyped.get())) { - return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) - { - return ConvertImplGenericFromString::execute(arguments, result_type); - }; + return &ConvertImplGenericFromString::execute; } else { @@ -2730,10 +2727,7 @@ private: /// Conversion from String through parsing. if (checkAndGetDataType(from_type_untyped.get())) { - return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) - { - return ConvertImplGenericFromString::execute(arguments, result_type); - }; + return &ConvertImplGenericFromString::execute; } const auto * from_type = checkAndGetDataType(from_type_untyped.get()); @@ -2801,10 +2795,7 @@ private: /// Conversion from String through parsing. if (checkAndGetDataType(from_type_untyped.get())) { - return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t /*input_rows_count*/) - { - return ConvertImplGenericFromString::execute(arguments, result_type); - }; + return &ConvertImplGenericFromString::execute; } const auto * from_type = checkAndGetDataType(from_type_untyped.get()); @@ -3315,6 +3306,38 @@ private: return false; }; + auto make_custom_serialization_wrapper = [&](const auto & types) -> bool + { + using Types = std::decay_t; + using ToDataType = typename Types::RightType; + using FromDataType = typename Types::LeftType; + + if constexpr (WhichDataType(FromDataType::type_id).isStringOrFixedString()) + { + if (to_type->getCustomSerialization()) + { + ret = &ConvertImplGenericFromString::execute; + return true; + } + } + if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) + { + if (from_type->getCustomSerialization()) + { + ret = [](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count); + }; + return true; + } + } + + return false; + }; + + if (callOnTwoTypeIndexes(from_type->getTypeId(), to_type->getTypeId(), make_custom_serialization_wrapper)) + return ret; + if (callOnIndexAndDataType(to_type->getTypeId(), make_default_wrapper)) return ret; diff --git a/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.reference b/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.reference new file mode 100644 index 00000000000..8a4df1605fb --- /dev/null +++ b/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.reference @@ -0,0 +1,6 @@ +127.0.0.1 IPv4 +127.0.0.1 String +2001:db8:0:85a3::ac1f:8001 IPv6 +2001:db8:0:85a3::ac1f:8001 String +0.0.0.0 IPv4 +:: IPv6 diff --git a/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.sql b/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.sql new file mode 100644 index 00000000000..1b704c141ec --- /dev/null +++ b/tests/queries/0_stateless/02007_ipv4_and_ipv6_to_and_from_string.sql @@ -0,0 +1,8 @@ +SELECT CAST('127.0.0.1' as IPv4) as v, toTypeName(v); +SELECT CAST(toIPv4('127.0.0.1') as String) as v, toTypeName(v); + +SELECT CAST('2001:0db8:0000:85a3:0000:0000:ac1f:8001' as IPv6) as v, toTypeName(v); +SELECT CAST(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001') as String) as v, toTypeName(v); + +SELECT toIPv4('hello') as v, toTypeName(v); +SELECT toIPv6('hello') as v, toTypeName(v); \ No newline at end of file