Special case of conversion code for types with custom serialization/deserialization

Fixes CASTing from String or FixedString to IPv4 or IPv6 and back.
This commit is contained in:
Vasily Nemkov 2021-08-18 23:34:07 +03:00 committed by Dmitry Novik
parent ea4fd19e28
commit 175b94fa2d
11 changed files with 192 additions and 55 deletions

View File

@ -0,0 +1,88 @@
#pragma once
#include <cstring>
#include <cassert>
#include <Columns/IColumn.h>
#include <Common/PODArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <IO/WriteBufferFromVector.h>
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 <typename ColumnType>
class WriteHelper
{
ColumnType & col;
WriteBufferFromVector<typename ColumnType::Chars> 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<ColumnType, ColumnFixedString>)
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<ColumnType, ColumnFixedString>)
{
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();
}
};
}
}

View File

@ -59,6 +59,7 @@ class DataTypeDecimalBase : public IDataType
public:
using FieldType = T;
using ColumnType = ColumnDecimal<T>;
static constexpr auto type_id = TypeId<T>;
static constexpr bool is_parametric = true;

View File

@ -38,6 +38,7 @@ class DataTypeEnum final : public IDataTypeEnum, public EnumValues<Type>
public:
using FieldType = Type;
using ColumnType = ColumnVector<FieldType>;
static constexpr auto type_id = sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16;
using typename EnumValues<Type>::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
{

View File

@ -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"; }

View File

@ -20,6 +20,7 @@ class DataTypeNumberBase : public IDataType
public:
static constexpr bool is_parametric = false;
static constexpr auto family_name = TypeName<T>;
static constexpr auto type_id = TypeId<T>;
using FieldType = T;
using ColumnType = ColumnVector<T>;

View File

@ -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;

View File

@ -15,9 +15,10 @@ public:
using FieldType = UUID;
using ColumnType = ColumnVector<UUID>;
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;

View File

@ -56,7 +56,7 @@ public:
const auto & source_data = typeid_cast<const ColumnDecimal<DateTime64> &>(col).getData();
Int32 scale_diff = typeid_cast<const DataTypeDateTime64 &>(*src.type).getScale() - target_scale;
const Int32 scale_diff = typeid_cast<const DataTypeDateTime64 &>(*src.type).getScale() - target_scale;
if (scale_diff == 0)
{
for (size_t i = 0; i < input_rows_count; ++i)

View File

@ -34,6 +34,7 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnStringHelpers.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Core/AccurateComparison.h>
@ -839,39 +840,42 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
};
/// Generic conversion of any type to String.
/// Generic conversion of any type to String or FixedString via serialization to text.
template <typename StringColumnType>
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<StringColumnType, ColumnString> || std::is_same_v<StringColumnType, ColumnFixedString>,
"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<ColumnString::Chars> 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<StringColumnType &>(*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>(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 <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name, ConvertReturnNullOnErrorTag>
: ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::Normal> {};
/// 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 <typename StringColumnType>
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<StringColumnType, ColumnString> || std::is_same_v<StringColumnType, ColumnFixedString>,
"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<ColumnString>(&col_from))
if (const StringColumnType * col_from_string = checkAndGetColumn<StringColumnType>(&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<ToDataType, DataTypeString>)
{
return ConvertImplGenericToString::execute(arguments, result_type);
return ConvertImplGenericToString<ColumnString>::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<DataTypeString>(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<ColumnString>::execute;
}
else
{
@ -2730,10 +2727,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(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<ColumnString>::execute;
}
const auto * from_type = checkAndGetDataType<DataTypeArray>(from_type_untyped.get());
@ -2801,10 +2795,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(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<ColumnString>::execute;
}
const auto * from_type = checkAndGetDataType<DataTypeTuple>(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<decltype(types)>;
using ToDataType = typename Types::RightType;
using FromDataType = typename Types::LeftType;
if constexpr (WhichDataType(FromDataType::type_id).isStringOrFixedString())
{
if (to_type->getCustomSerialization())
{
ret = &ConvertImplGenericFromString<typename FromDataType::ColumnType>::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<typename ToDataType::ColumnType>::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<void>(to_type->getTypeId(), make_default_wrapper))
return ret;

View File

@ -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

View File

@ -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);