Better conversion from String to Variant

This commit is contained in:
avogar 2024-03-27 20:40:26 +00:00
parent c21c67f9f5
commit df06619399
11 changed files with 107 additions and 33 deletions

View File

@ -194,12 +194,12 @@ ReturnType deserializeImpl(
buf.dropCheckpoint();
if (buf.hasUnreadData())
{
restore_column_if_needed();
if constexpr (throw_exception)
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
restore_column_if_needed();
return ReturnType(false);
}
return ReturnType(true);

View File

@ -463,6 +463,9 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
/// nested_type will be nullptr, we should return nullptr in this case.
if (!nested_type)
return nullptr;
/// Common type for Nullable(Nothing) and Variant(...) is Variant(...)
if (isVariant(nested_type))
return nested_type;
return std::make_shared<DataTypeNullable>(nested_type);
}
}

View File

@ -100,7 +100,11 @@ protected:
validateDataType(type, data_type_validation_settings);
if (cast_type == CastType::accurateOrNull)
{
/// Variant handles NULLs by itself during conversions.
if (!isVariant(type))
return makeNullable(type);
}
if (internal)
return type;

View File

@ -1815,6 +1815,7 @@ struct ConvertImpl
/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization.
template <bool throw_on_error>
struct ConvertImplGenericFromString
{
static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count)
@ -1854,29 +1855,34 @@ struct ConvertImplGenericFromString
{
serialization_from.deserializeWholeText(column_to, read_buffer, format_settings);
}
catch (const Exception & e)
catch (const Exception &)
{
auto * nullable_column = typeid_cast<ColumnNullable *>(&column_to);
if (e.code() == ErrorCodes::CANNOT_PARSE_BOOL && nullable_column)
{
auto & col_nullmap = nullable_column->getNullMapData();
if (col_nullmap.size() != nullable_column->size())
col_nullmap.resize_fill(nullable_column->size());
if (nullable_column->size() == (i + 1))
nullable_column->popBack(1);
nullable_column->insertDefault();
continue;
}
if constexpr (throw_on_error)
throw;
/// Check if exception happened after we inserted the value
/// (deserializeWholeText should not do it, but let's check anyway).
if (column_to.size() > i)
column_to.popBack(column_to.size() - i);
column_to.insertDefault();
}
/// Usually deserializeWholeText checks for eof after parsing, but let's check one more time just in case.
if (!read_buffer.eof())
{
if constexpr (throw_on_error)
{
if (result_type)
throwExceptionForIncompletelyParsedValue(read_buffer, *result_type);
else
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT,
"Cannot parse string to column {}. Expected eof", column_to.getName());
throw Exception(
ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to column {}. Expected eof", column_to.getName());
}
else
{
if (column_to.size() > i)
column_to.popBack(column_to.size() - i);
column_to.insertDefault();
}
}
}
}
@ -3279,7 +3285,9 @@ private:
{
if (checkAndGetDataType<DataTypeString>(from_type.get()))
{
return &ConvertImplGenericFromString::execute;
if (cast_type == CastType::accurateOrNull)
return &ConvertImplGenericFromString<false>::execute;
return &ConvertImplGenericFromString<true>::execute;
}
return createWrapper<ToDataType>(from_type, to_type, requested_result_is_nullable);
@ -3442,7 +3450,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return &ConvertImplGenericFromString::execute;
return &ConvertImplGenericFromString<true>::execute;
}
else if (const auto * agg_type = checkAndGetDataType<DataTypeAggregateFunction>(from_type_untyped.get()))
{
@ -3485,7 +3493,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return &ConvertImplGenericFromString::execute;
return &ConvertImplGenericFromString<true>::execute;
}
DataTypePtr from_type_holder;
@ -3576,7 +3584,7 @@ private:
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return &ConvertImplGenericFromString::execute;
return &ConvertImplGenericFromString<true>::execute;
}
const auto * from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
@ -3921,7 +3929,7 @@ private:
{
return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count)
{
auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count)->assumeMutable();
auto res = ConvertImplGenericFromString<true>::execute(arguments, result_type, nullable_source, input_rows_count)->assumeMutable();
res->finalize();
return res;
};
@ -4076,6 +4084,29 @@ private:
return ColumnVariant::create(discriminators, variants);
}
WrapperType createStringToVariantWrapper() const
{
return [&](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr
{
auto column = arguments[0].column->convertToFullColumnIfLowCardinality();
auto args = arguments;
args[0].column = column;
const ColumnNullable * column_nullable = nullptr;
if (isColumnNullable(*args[0].column))
{
column_nullable = assert_cast<const ColumnNullable *>(args[0].column.get());
args[0].column = column_nullable->getNestedColumnPtr();
}
args[0].type = removeNullable(removeLowCardinality(args[0].type));
if (cast_type == CastType::accurateOrNull)
return ConvertImplGenericFromString<false>::execute(args, result_type, column_nullable, input_rows_count);
return ConvertImplGenericFromString<true>::execute(args, result_type, column_nullable, input_rows_count);
};
}
WrapperType createColumnToVariantWrapper(const DataTypePtr & from_type, const DataTypeVariant & to_variant) const
{
/// We allow converting NULL to Variant(...) as Variant can store NULLs.
@ -4089,6 +4120,9 @@ private:
};
}
if (isStringOrFixedString(removeNullable(removeLowCardinality(from_type))))
return createStringToVariantWrapper();
auto variant_discr_opt = to_variant.tryGetVariantDiscriminator(*removeNullableOrLowCardinalityNullable(from_type));
if (!variant_discr_opt)
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert type {} to {}. Conversion to Variant allowed only for types from this Variant", from_type->getName(), to_variant.getName());
@ -4691,7 +4725,7 @@ private:
if (to_type->getCustomSerialization() && to_type->getCustomName())
{
ret = [requested_result_is_nullable](
ret = [this, requested_result_is_nullable](
ColumnsWithTypeAndName & arguments,
const DataTypePtr & result_type,
const ColumnNullable * column_nullable,
@ -4700,7 +4734,10 @@ private:
auto wrapped_result_type = result_type;
if (requested_result_is_nullable)
wrapped_result_type = makeNullable(result_type);
return ConvertImplGenericFromString::execute(
if (this->cast_type == CastType::accurateOrNull)
return ConvertImplGenericFromString<false>::execute(
arguments, wrapped_result_type, column_nullable, input_rows_count);
return ConvertImplGenericFromString<true>::execute(
arguments, wrapped_result_type, column_nullable, input_rows_count);
};
return true;

View File

@ -14,7 +14,7 @@ function test1_insert()
echo "test1 insert"
$CH_CLIENT -nmq "insert into test select number, NULL from numbers(3);
insert into test select number + 3, number from numbers(3);
insert into test select number + 6, 'str_' || toString(number) from numbers(3);
insert into test select number + 6, ('str_' || toString(number))::Variant(String) from numbers(3);
insert into test select number + 9, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(3);
insert into test select number + 12, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(3);
insert into test select number + 15, range(number + 1)::Array(UInt64) from numbers(3);"
@ -40,7 +40,7 @@ function test2_insert()
echo "test2 insert"
$CH_CLIENT -nmq "insert into test select number, NULL from numbers(3);
insert into test select number + 3, number % 2 ? NULL : number from numbers(3);
insert into test select number + 6, number % 2 ? NULL : 'str_' || toString(number) from numbers(3);
insert into test select number + 6, number % 2 ? NULL : ('str_' || toString(number))::Variant(String) from numbers(3);
insert into test select number + 9, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(('lc_str_' || toString(number))::LowCardinality(String), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);
insert into test select number + 12, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);
insert into test select number + 15, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(range(number + 1)::Array(UInt64), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(3);"
@ -64,7 +64,7 @@ select v.\`Array(UInt64)\`.size0 from test order by id;"
function test3_insert()
{
echo "test3 insert"
$CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST('str_' || toString(number), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number + 1)::Array(UInt64), type)) as res from numbers(18);"
$CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST(('str_' || toString(number))::Variant(String), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number + 1)::Array(UInt64), type)) as res from numbers(18);"
}
function test3_select()

View File

@ -14,7 +14,7 @@ function test4_insert()
echo "test4 insert"
$CH_CLIENT -nmq "insert into test select number, NULL from numbers(100000);
insert into test select number + 100000, number from numbers(100000);
insert into test select number + 200000, 'str_' || toString(number) from numbers(100000);
insert into test select number + 200000, ('str_' || toString(number))::Variant(String) from numbers(100000);
insert into test select number + 300000, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(100000);
insert into test select number + 400000, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(100000);
insert into test select number + 500000, range(number % 20 + 1)::Array(UInt64) from numbers(100000);"

View File

@ -15,7 +15,7 @@ function test5_insert()
$CH_CLIENT -nmq "
insert into test select number, NULL from numbers(200000);
insert into test select number + 200000, number % 2 ? NULL : number from numbers(200000);
insert into test select number + 400000, number % 2 ? NULL : 'str_' || toString(number) from numbers(200000);
insert into test select number + 400000, number % 2 ? NULL : ('str_' || toString(number))::Variant(String) from numbers(200000);
insert into test select number + 600000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(('lc_str_' || toString(number))::LowCardinality(String), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);
insert into test select number + 800000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);
insert into test select number + 1000000, number % 2 ? CAST(NULL, 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') : CAST(range(number % 20 + 1)::Array(UInt64), 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))') from numbers(200000);"

View File

@ -12,7 +12,7 @@ CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspic
function test6_insert()
{
echo "test6 insert"
$CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST('str_' || toString(number), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number % 20 + 1)::Array(UInt64), type)) as res from numbers(1200000);"
$CH_CLIENT -q "insert into test with 'Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))' as type select number, multiIf(number % 6 == 0, CAST(NULL, type), number % 6 == 1, CAST(('str_' || toString(number))::Variant(String), type), number % 6 == 2, CAST(number, type), number % 6 == 3, CAST(('lc_str_' || toString(number))::LowCardinality(String), type), number % 6 == 4, CAST(tuple(number, number + 1)::Tuple(a UInt32, b UInt32), type), CAST(range(number % 20 + 1)::Array(UInt64), type)) as res from numbers(1200000);"
}
function test6_select()

View File

@ -6,7 +6,7 @@
\N
Hello
Hello
NULL
\N
Hello
Hello
\N

View File

@ -0,0 +1,13 @@
42 UInt64
abc String
\N None
[1,2,3] Array(UInt64)
[1, 2, 3 String
\N None
42 UInt64
42 UInt64
42 UInt64
\N None
42 UInt64
\N None
\N None

View File

@ -0,0 +1,17 @@
set allow_experimental_variant_type=1;
select CAST('42', 'Variant(String, UInt64)') as v, variantType(v);
select CAST('abc', 'Variant(String, UInt64)') as v, variantType(v);
select CAST('null', 'Variant(String, UInt64)') as v, variantType(v);
select CAST('[1, 2, 3]', 'Variant(String, Array(UInt64))') as v, variantType(v);
select CAST('[1, 2, 3', 'Variant(String, Array(UInt64))') as v, variantType(v);
select CAST('42', 'Variant(Date)') as v, variantType(v); -- {serverError INCORRECT_DATA}
select accurateCastOrNull('42', 'Variant(Date)') as v, variantType(v);
select CAST('42'::FixedString(2), 'Variant(String, UInt64)') as v, variantType(v);
select CAST('42'::LowCardinality(String), 'Variant(String, UInt64)') as v, variantType(v);
select CAST('42'::Nullable(String), 'Variant(String, UInt64)') as v, variantType(v);
select CAST(NULL::Nullable(String), 'Variant(String, UInt64)') as v, variantType(v);
select CAST('42'::LowCardinality(Nullable(String)), 'Variant(String, UInt64)') as v, variantType(v);
select CAST(NULL::LowCardinality(Nullable(String)), 'Variant(String, UInt64)') as v, variantType(v);
select CAST(NULL::LowCardinality(Nullable(FixedString(2))), 'Variant(String, UInt64)') as v, variantType(v);