mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
parent
ea4fd19e28
commit
175b94fa2d
88
src/Columns/ColumnStringHelpers.h
Normal file
88
src/Columns/ColumnStringHelpers.h
Normal 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();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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"; }
|
||||
|
||||
|
@ -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>;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
@ -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);
|
Loading…
Reference in New Issue
Block a user