mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Better test, fix style
This commit is contained in:
parent
60c721c21b
commit
fb20e80db4
@ -576,7 +576,7 @@ ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col)
|
||||
template <typename StringColumnType>
|
||||
struct ConvertImplGenericToString
|
||||
{
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const FormatSettings & format_settings)
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const ContextPtr & context)
|
||||
{
|
||||
static_assert(std::is_same_v<StringColumnType, ColumnString> || std::is_same_v<StringColumnType, ColumnFixedString>,
|
||||
"Can be used only to serialize to ColumnString or ColumnFixedString");
|
||||
@ -597,6 +597,7 @@ struct ConvertImplGenericToString
|
||||
|
||||
auto & write_buffer = write_helper.getWriteBuffer();
|
||||
|
||||
FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{};
|
||||
auto serialization = type.getDefaultSerialization();
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
{
|
||||
@ -1820,7 +1821,7 @@ struct ConvertImpl
|
||||
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)
|
||||
static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count, const ContextPtr & context)
|
||||
{
|
||||
const IColumn & column_from = *arguments[0].column;
|
||||
const IDataType & data_type_to = *result_type;
|
||||
@ -1828,7 +1829,7 @@ struct ConvertImplGenericFromString
|
||||
auto serialization = data_type_to.getDefaultSerialization();
|
||||
const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr;
|
||||
|
||||
executeImpl(column_from, *res, *serialization, input_rows_count, null_map, result_type.get());
|
||||
executeImpl(column_from, *res, *serialization, input_rows_count, null_map, result_type.get(), context);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -1838,11 +1839,12 @@ struct ConvertImplGenericFromString
|
||||
const ISerialization & serialization_from,
|
||||
size_t input_rows_count,
|
||||
const PaddedPODArray<UInt8> * null_map,
|
||||
const IDataType * result_type)
|
||||
const IDataType * result_type,
|
||||
const ContextPtr & context)
|
||||
{
|
||||
column_to.reserve(input_rows_count);
|
||||
|
||||
FormatSettings format_settings;
|
||||
FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{};
|
||||
for (size_t i = 0; i < input_rows_count; ++i)
|
||||
{
|
||||
if (null_map && (*null_map)[i])
|
||||
@ -2299,7 +2301,7 @@ private:
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeString>)
|
||||
{
|
||||
if (from_type->getCustomSerialization())
|
||||
return ConvertImplGenericToString<ColumnString>::execute(arguments, result_type, input_rows_count, context ? getFormatSettings(context) : FormatSettings());
|
||||
return ConvertImplGenericToString<ColumnString>::execute(arguments, result_type, input_rows_count, context);
|
||||
}
|
||||
|
||||
bool done = false;
|
||||
@ -2332,7 +2334,7 @@ private:
|
||||
/// Generic conversion of any type to String.
|
||||
if (std::is_same_v<ToDataType, DataTypeString>)
|
||||
{
|
||||
return ConvertImplGenericToString<ColumnString>::execute(arguments, result_type, input_rows_count, context ? getFormatSettings(context) : FormatSettings());
|
||||
return ConvertImplGenericToString<ColumnString>::execute(arguments, result_type, input_rows_count, context);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
|
||||
@ -3288,8 +3290,17 @@ private:
|
||||
if (checkAndGetDataType<DataTypeString>(from_type.get()))
|
||||
{
|
||||
if (cast_type == CastType::accurateOrNull)
|
||||
return &ConvertImplGenericFromString<false>::execute;
|
||||
return &ConvertImplGenericFromString<true>::execute;
|
||||
{
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr
|
||||
{
|
||||
return ConvertImplGenericFromString<false>::execute(arguments, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr
|
||||
{
|
||||
return ConvertImplGenericFromString<true>::execute(arguments, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
return createWrapper<ToDataType>(from_type, to_type, requested_result_is_nullable);
|
||||
@ -3452,7 +3463,10 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return &ConvertImplGenericFromString<true>::execute;
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr
|
||||
{
|
||||
return ConvertImplGenericFromString<true>::execute(arguments, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
else if (const auto * agg_type = checkAndGetDataType<DataTypeAggregateFunction>(from_type_untyped.get()))
|
||||
{
|
||||
@ -3495,7 +3509,10 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return &ConvertImplGenericFromString<true>::execute;
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr
|
||||
{
|
||||
return ConvertImplGenericFromString<true>::execute(arguments, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
DataTypePtr from_type_holder;
|
||||
@ -3586,7 +3603,10 @@ private:
|
||||
/// Conversion from String through parsing.
|
||||
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
|
||||
{
|
||||
return &ConvertImplGenericFromString<true>::execute;
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr
|
||||
{
|
||||
return ConvertImplGenericFromString<true>::execute(arguments, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
const auto * from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
|
||||
@ -3929,9 +3949,9 @@ private:
|
||||
}
|
||||
else if (checkAndGetDataType<DataTypeString>(from_type.get()))
|
||||
{
|
||||
return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count)
|
||||
return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count)
|
||||
{
|
||||
auto res = ConvertImplGenericFromString<true>::execute(arguments, result_type, nullable_source, input_rows_count)->assumeMutable();
|
||||
auto res = ConvertImplGenericFromString<true>::execute(arguments, result_type, nullable_source, input_rows_count, context)->assumeMutable();
|
||||
res->finalize();
|
||||
return res;
|
||||
};
|
||||
@ -4104,8 +4124,8 @@ private:
|
||||
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);
|
||||
return ConvertImplGenericFromString<false>::execute(args, result_type, column_nullable, input_rows_count, context);
|
||||
return ConvertImplGenericFromString<true>::execute(args, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
@ -4265,8 +4285,8 @@ private:
|
||||
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);
|
||||
return ConvertImplGenericFromString<false>::execute(args, result_type, column_nullable, input_rows_count, context);
|
||||
return ConvertImplGenericFromString<true>::execute(args, result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
}
|
||||
|
||||
@ -5020,9 +5040,9 @@ private:
|
||||
wrapped_result_type = makeNullable(result_type);
|
||||
if (this->cast_type == CastType::accurateOrNull)
|
||||
return ConvertImplGenericFromString<false>::execute(
|
||||
arguments, wrapped_result_type, column_nullable, input_rows_count);
|
||||
arguments, wrapped_result_type, column_nullable, input_rows_count, context);
|
||||
return ConvertImplGenericFromString<true>::execute(
|
||||
arguments, wrapped_result_type, column_nullable, input_rows_count);
|
||||
arguments, wrapped_result_type, column_nullable, input_rows_count, context);
|
||||
};
|
||||
return true;
|
||||
}
|
||||
@ -5060,7 +5080,7 @@ private:
|
||||
{
|
||||
ret = [this](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, context ? getFormatSettings(context) : FormatSettings());
|
||||
return ConvertImplGenericToString<typename ToDataType::ColumnType>::execute(arguments, result_type, input_rows_count, context);
|
||||
};
|
||||
return true;
|
||||
}
|
||||
|
@ -271,7 +271,7 @@ MergeTreeReaderWide::FileStreams::iterator MergeTreeReaderWide::addStream(const
|
||||
|
||||
if (read_without_marks)
|
||||
return streams.emplace(stream_name, create_stream.operator()<MergeTreeReaderStreamSingleColumnWholePart>()).first;
|
||||
|
||||
|
||||
marks_loader->startAsyncLoad();
|
||||
return streams.emplace(stream_name, create_stream.operator()<MergeTreeReaderStreamSingleColumn>()).first;
|
||||
}
|
||||
|
@ -2,8 +2,8 @@ MergeTree compact + horizontal merge
|
||||
CollapsingMergeTree
|
||||
100000 String
|
||||
100000 UInt64
|
||||
50000 UInt64
|
||||
50000 String
|
||||
50000 UInt64
|
||||
VersionedCollapsingMergeTree
|
||||
100000 String
|
||||
100000 UInt64
|
||||
@ -11,34 +11,34 @@ VersionedCollapsingMergeTree
|
||||
75000 UInt64
|
||||
MergeTree wide + horizontal merge
|
||||
CollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
100000 UInt64
|
||||
50000 String
|
||||
50000 UInt64
|
||||
VersionedCollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
100000 UInt64
|
||||
75000 String
|
||||
75000 UInt64
|
||||
MergeTree compact + vertical merge
|
||||
CollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
50000 UInt64
|
||||
100000 UInt64
|
||||
50000 String
|
||||
50000 UInt64
|
||||
VersionedCollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
75000 UInt64
|
||||
100000 UInt64
|
||||
75000 String
|
||||
75000 UInt64
|
||||
MergeTree wide + vertical merge
|
||||
CollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
100000 UInt64
|
||||
50000 String
|
||||
50000 UInt64
|
||||
VersionedCollapsingMergeTree
|
||||
100000 UInt64
|
||||
100000 String
|
||||
75000 UInt64
|
||||
100000 UInt64
|
||||
75000 String
|
||||
75000 UInt64
|
||||
|
@ -18,9 +18,9 @@ function test()
|
||||
$CH_CLIENT -q "insert into test select number, 1, number from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, -1, 'str_' || toString(number) from numbers(50000, 100000)"
|
||||
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count()"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count()"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
|
||||
$CH_CLIENT -q "drop table test"
|
||||
|
||||
echo "VersionedCollapsingMergeTree"
|
||||
@ -29,9 +29,9 @@ function test()
|
||||
$CH_CLIENT -q "insert into test select number, 1, 1, number from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, -1, number >= 75000 ? 2 : 1, 'str_' || toString(number) from numbers(50000, 100000)"
|
||||
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count()"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count()"
|
||||
$CH_CLIENT -q "select count(), dynamicType(d) from test group by dynamicType(d) order by count(), dynamicType(d)"
|
||||
$CH_CLIENT -q "drop table test"
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user