mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Add generic implementation of function transform
This commit is contained in:
parent
396eb70426
commit
cb2d395410
@ -16,6 +16,7 @@
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -79,15 +80,6 @@ namespace
|
||||
args_size);
|
||||
|
||||
const DataTypePtr & type_x = arguments[0];
|
||||
const auto & type_x_nn = removeNullable(type_x);
|
||||
|
||||
if (!type_x_nn->isValueRepresentedByNumber() && !isString(type_x_nn) && !isNothing(type_x_nn))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Unsupported type {} of first argument "
|
||||
"of function {}, must be numeric type or Date/DateTime or String",
|
||||
type_x->getName(),
|
||||
getName());
|
||||
|
||||
const DataTypeArray * type_arr_from = checkAndGetDataType<DataTypeArray>(arguments[1].get());
|
||||
|
||||
@ -99,14 +91,13 @@ namespace
|
||||
|
||||
const auto type_arr_from_nested = type_arr_from->getNestedType();
|
||||
|
||||
if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber())
|
||||
|| (isString(type_x) != isString(type_arr_from_nested)))
|
||||
auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested});
|
||||
if (!src)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"First argument and elements of array "
|
||||
"of second argument of function {} must have compatible types: "
|
||||
"both numeric or both strings.",
|
||||
"of the second argument of function {} must have compatible types",
|
||||
getName());
|
||||
}
|
||||
|
||||
@ -157,8 +148,8 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr
|
||||
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
ColumnPtr executeImpl(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
initialize(arguments, result_type);
|
||||
|
||||
@ -172,22 +163,36 @@ namespace
|
||||
default_non_const = castColumn(arguments[3], result_type);
|
||||
|
||||
auto column_result = result_type->createColumn();
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const)
|
||||
&& !executeString(in, *column_result, default_non_const))
|
||||
if (cache.table_num_to_idx)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const))
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
|
||||
}
|
||||
}
|
||||
else if (cache.table_string_to_idx)
|
||||
{
|
||||
if (!executeString(in, *column_result, default_non_const))
|
||||
executeContiguous(in, *column_result, default_non_const);
|
||||
}
|
||||
else if (cache.table_anything_to_idx)
|
||||
{
|
||||
executeAnything(in, *column_result, default_non_const);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized");
|
||||
|
||||
return column_result;
|
||||
}
|
||||
|
||||
@ -204,6 +209,47 @@ namespace
|
||||
return impl->execute(args, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
const size_t size = in->size();
|
||||
const auto & table = *cache.table_anything_to_idx;
|
||||
column_result.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
SipHash hash;
|
||||
in->updateHashWithValue(i, hash);
|
||||
|
||||
const auto * it = table.find(hash.get128());
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(*in, i);
|
||||
}
|
||||
}
|
||||
|
||||
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
const size_t size = in->size();
|
||||
const auto & table = *cache.table_string_to_idx;
|
||||
column_result.reserve(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const auto * it = table.find(in->getDataAt(i));
|
||||
if (it)
|
||||
column_result.insertFrom(*cache.to_column, it->getMapped());
|
||||
else if (cache.default_column)
|
||||
column_result.insertFrom(*cache.default_column, 0);
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(*in, i);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const
|
||||
{
|
||||
@ -593,9 +639,11 @@ namespace
|
||||
{
|
||||
using NumToIdx = HashMap<UInt64, size_t, HashCRC32<UInt64>>;
|
||||
using StringToIdx = HashMap<StringRef, size_t, StringRefHash>;
|
||||
using AnythingToIdx = HashMap<UInt128, size_t>;
|
||||
|
||||
std::unique_ptr<NumToIdx> table_num_to_idx;
|
||||
std::unique_ptr<StringToIdx> table_string_to_idx;
|
||||
std::unique_ptr<AnythingToIdx> table_anything_to_idx;
|
||||
|
||||
ColumnPtr from_column;
|
||||
ColumnPtr to_column;
|
||||
@ -648,18 +696,16 @@ namespace
|
||||
|
||||
std::lock_guard lock(cache.mutex);
|
||||
|
||||
ColumnPtr from_column_or_null_ptr = castColumnAccurateOrNull(
|
||||
const ColumnPtr & from_column_uncasted = array_from->getDataPtr();
|
||||
|
||||
cache.from_column = castColumn(
|
||||
{
|
||||
array_from->getDataPtr(),
|
||||
from_column_uncasted,
|
||||
typeid_cast<const DataTypeArray &>(*arguments[1].type).getNestedType(),
|
||||
arguments[1].name
|
||||
},
|
||||
from_type);
|
||||
|
||||
const ColumnNullable & from_column_or_null = assert_cast<const ColumnNullable &>(*from_column_or_null_ptr);
|
||||
|
||||
cache.from_column = from_column_or_null.getNestedColumnPtr();
|
||||
|
||||
cache.to_column = castColumn(
|
||||
{
|
||||
array_to->getDataPtr(),
|
||||
@ -696,13 +742,14 @@ namespace
|
||||
|
||||
/// Note: Doesn't check the duplicates in the `from` array.
|
||||
|
||||
if (from_type->isValueRepresentedByNumber())
|
||||
WhichDataType which(from_type);
|
||||
if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64())
|
||||
{
|
||||
cache.table_num_to_idx = std::make_unique<Cache::NumToIdx>();
|
||||
auto & table = *cache.table_num_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (!from_column_or_null.isNullAt(i))
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
|
||||
StringRef ref = cache.from_column->getDataAt(i);
|
||||
@ -718,7 +765,7 @@ namespace
|
||||
auto & table = *cache.table_string_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (!from_column_or_null.isNullAt(i))
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
StringRef ref = cache.from_column->getDataAt(i);
|
||||
table[ref] = i;
|
||||
@ -726,7 +773,19 @@ namespace
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected data type {} as the first argument in function `transform`", from_type->getName());
|
||||
{
|
||||
cache.table_anything_to_idx = std::make_unique<Cache::AnythingToIdx>();
|
||||
auto & table = *cache.table_anything_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
{
|
||||
SipHash hash;
|
||||
cache.from_column->updateHashWithValue(i, hash);
|
||||
table[hash.get128()] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
cache.initialized = true;
|
||||
}
|
||||
|
38
tests/queries/0_stateless/02798_generic_transform.reference
Normal file
38
tests/queries/0_stateless/02798_generic_transform.reference
Normal file
@ -0,0 +1,38 @@
|
||||
def
|
||||
def
|
||||
def
|
||||
hello
|
||||
def
|
||||
world
|
||||
def
|
||||
abc!
|
||||
def
|
||||
def
|
||||
|
||||
|
||||
|
||||
hello
|
||||
|
||||
world
|
||||
|
||||
abc
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
hello
|
||||
|
||||
world
|
||||
|
||||
abc
|
||||
|
||||
|
||||
123
|
||||
2023-03-03 00:00:00.000
|
||||
2023-02-02 00:00:00.000
|
||||
2023-01-01 00:00:00.000
|
||||
1 1
|
||||
42 42
|
||||
42
|
||||
42
|
12
tests/queries/0_stateless/02798_generic_transform.sql
Normal file
12
tests/queries/0_stateless/02798_generic_transform.sql
Normal file
@ -0,0 +1,12 @@
|
||||
SELECT transform((number, toString(number)), [(3, '3'), (5, '5'), (7, '7')], ['hello', 'world', 'abc!'], 'def') FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toNullable(toInt256(number)), [3, 5, 7], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10;
|
||||
SELECT transform(toUInt256(number), [3, 5, 7], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10;
|
||||
|
||||
select case 1::Nullable(Int32) when 1 then 123 else 0 end;
|
||||
|
||||
SELECT transform(arrayJoin(['c', 'b', 'a']), ['a', 'b'], [toDateTime64('2023-01-01', 3), toDateTime64('2023-02-02', 3)], toDateTime64('2023-03-03', 3));
|
||||
|
||||
SELECT transform(1, [1], [toDecimal32(1, 2)]), toDecimal32(1, 2);
|
||||
select transform(1, [1], [toDecimal32(42, 2)]), toDecimal32(42, 2);
|
||||
SELECT transform(1, [1], [toDecimal32(42, 2)], 0);
|
||||
SELECT transform(1, [1], [toDecimal32(42, 2)], toDecimal32(0, 2));
|
Loading…
Reference in New Issue
Block a user