mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
dbms: function 'transform': development [#METR-15987].
This commit is contained in:
parent
12d1fee442
commit
5e230a6b2d
@ -3,6 +3,7 @@
|
||||
#include <Poco/Net/DNS.h>
|
||||
|
||||
#include <math.h>
|
||||
#include <mutex>
|
||||
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
@ -21,6 +22,7 @@
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnReplicated.h>
|
||||
#include <DB/Common/UnicodeBar.h>
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
@ -54,6 +56,8 @@ namespace DB
|
||||
* sleep(n) - спит n секунд каждый блок.
|
||||
*
|
||||
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
|
||||
*
|
||||
* transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
|
||||
*/
|
||||
|
||||
|
||||
@ -884,4 +888,452 @@ using FunctionIsFinite = FunctionNumericPredicate<IsFiniteImpl>;
|
||||
using FunctionIsInfinite = FunctionNumericPredicate<IsInfiniteImpl>;
|
||||
using FunctionIsNaN = FunctionNumericPredicate<IsNaNImpl>;
|
||||
|
||||
|
||||
/** transform(x, [from...], [to...], default)
|
||||
* - преобразует значения согласно явно указанному отображению.
|
||||
*
|
||||
* x - что преобразовывать.
|
||||
* from - константный массив значений для преобразования.
|
||||
* to - константный массив значений, в которые должны быть преобразованы значения из from.
|
||||
* default - константа, какое значение использовать, если x не равен ни одному из значений во from.
|
||||
* from и to - массивы одинаковых размеров.
|
||||
*
|
||||
* Типы:
|
||||
* transform(T, Array(T), Array(U), U) -> U
|
||||
*
|
||||
* transform(x, [from...], [to...])
|
||||
* - eсли default не указан, то для значений x, для которых нет соответствующего элемента во from, возвращается не изменённое значение x.
|
||||
*
|
||||
* Типы:
|
||||
* transform(T, Array(T), Array(T)) -> T
|
||||
*
|
||||
* Замечание: реализация довольно громоздкая.
|
||||
*/
|
||||
class FunctionTransform : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "transform";
|
||||
static IFunction * create(const Context &) { return new FunctionTransform; }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
const auto args_size = arguments.size();
|
||||
if (args_size != 3 && args_size != 4)
|
||||
throw Exception{
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed " +
|
||||
toString(args_size) + ", should be 3 or 4",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const IDataType * type_x = arguments[0].get();
|
||||
|
||||
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
|
||||
throw Exception("Unsupported type " + type_x->getName()
|
||||
+ " of first argument of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
|
||||
|
||||
if (!type_arr_from)
|
||||
throw Exception("Second argument of function " + getName()
|
||||
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto type_arr_from_nested = type_arr_from->getNestedType();
|
||||
|
||||
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
|
||||
throw Exception("First argument and elements of array of second argument of function " + getName()
|
||||
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
|
||||
|
||||
if (!type_arr_to)
|
||||
throw Exception("Third argument of function " + getName()
|
||||
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto type_arr_to_nested = type_arr_to->getNestedType();
|
||||
|
||||
if (args_size == 3)
|
||||
{
|
||||
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
|
||||
throw Exception("Function " + getName()
|
||||
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return type_x->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
const IDataType * type_default = arguments[3].get();
|
||||
|
||||
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
|
||||
throw Exception("Unsupported type " + type_default->getName()
|
||||
+ " of fourth argument (default value) of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|
||||
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
|
||||
throw Exception("Function " + getName()
|
||||
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return type_arr_to_nested->clone();
|
||||
}
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const ColumnConstArray * array_from = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[1]).column);
|
||||
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[2]).column);
|
||||
|
||||
if (!array_from && !array_to)
|
||||
throw Exception("Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
prepare(array_from->getData(), array_to->getData());
|
||||
|
||||
/// Задано ли значение по-умолчанию.
|
||||
Field default_value;
|
||||
if (arguments.size() == 4)
|
||||
{
|
||||
const IColumnConst * default_col = dynamic_cast<const IColumnConst *>(&*block.getByPosition(arguments[3]).column);
|
||||
default_value = (*default_col)[0];
|
||||
}
|
||||
|
||||
const auto in = block.getByPosition(arguments.front()).column.get();
|
||||
auto column_result = block.getByPosition(result).type->createColumn();
|
||||
auto out = column_result.get();
|
||||
|
||||
if (!executeNum<UInt8>(in, out, default_value)
|
||||
&& !executeNum<UInt16>(in, out, default_value)
|
||||
&& !executeNum<UInt32>(in, out, default_value)
|
||||
&& !executeNum<UInt64>(in, out, default_value)
|
||||
&& !executeNum<Int8>(in, out, default_value)
|
||||
&& !executeNum<Int16>(in, out, default_value)
|
||||
&& !executeNum<Int32>(in, out, default_value)
|
||||
&& !executeNum<Int64>(in, out, default_value)
|
||||
&& !executeNum<Float32>(in, out, default_value)
|
||||
&& !executeNum<Float64>(in, out, default_value)
|
||||
&& !executeString(in, out, default_value))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = column_result;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
|
||||
{
|
||||
if (default_value.isNull())
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
|
||||
if (!out)
|
||||
throw Exception(
|
||||
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
|
||||
+ ", must be " + in->getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
executeImplNumToNum<T>(in->getData(), out->getData());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!executeNumToNumWithDefault<T, UInt8>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, UInt16>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, UInt32>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, UInt64>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Int8>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Int16>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Int32>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Int64>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Float32>(in, out_untyped, default_value)
|
||||
&& !executeNumToNumWithDefault<T, Float64>(in, out_untyped, default_value)
|
||||
&& !executeNumToString<T>(in, out_untyped, default_value))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto in = typeid_cast<const ColumnConst<T> *>(in_untyped))
|
||||
{
|
||||
/* TODO */
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool executeString(const IColumn * in_untyped, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
|
||||
{
|
||||
if (!executeStringToNum<UInt8>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<UInt16>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<UInt32>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<UInt64>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Int8>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Int16>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Int32>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Int64>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Float32>(in, out_untyped, default_value)
|
||||
&& !executeStringToNum<Float64>(in, out_untyped, default_value)
|
||||
&& !executeStringToString(in, out_untyped, default_value))
|
||||
throw Exception(
|
||||
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto in = typeid_cast<const ColumnConstString *>(in_untyped))
|
||||
{
|
||||
/* TODO */
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool executeNumToNumWithDefault(const ColumnVector<T> * in, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
executeImplNumToNumWithDefault<T, U>(in->getData(), out->getData(), default_value.get<U>());
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNumToString(const ColumnVector<T> * in, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
auto out = typeid_cast<ColumnString *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
const String & default_str = default_value.get<const String &>();
|
||||
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
|
||||
executeImplNumToStringWithDefault<T>(in->getData(), out->getChars(), out->getOffsets(), default_string_ref);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
bool executeStringToNum(const ColumnString * in, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
executeImplStringToNumWithDefault<U>(in->getChars(), in->getOffsets(), out->getData(), default_value.get<U>());
|
||||
return true;
|
||||
}
|
||||
|
||||
bool executeStringToString(const ColumnString * in, IColumn * out_untyped, const Field & default_value)
|
||||
{
|
||||
auto out = typeid_cast<ColumnString *>(out_untyped);
|
||||
if (!out)
|
||||
return false;
|
||||
|
||||
if (default_value.isNull())
|
||||
executeImplStringToString<false>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), {});
|
||||
else
|
||||
{
|
||||
const String & default_str = default_value.get<const String &>();
|
||||
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
|
||||
executeImplStringToString<true>(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
template <typename T, typename U>
|
||||
void executeImplNumToNumWithDefault(const PODArray<T> & src, PODArray<U> & dst, U dst_default)
|
||||
{
|
||||
const auto & table = *table_num_to_num;
|
||||
size_t size = src.size();
|
||||
dst.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
if (it != table.end())
|
||||
dst[i] = it->second;
|
||||
else
|
||||
dst[i] = dst_default;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToNum(const PODArray<T> & src, PODArray<T> & dst)
|
||||
{
|
||||
const auto & table = *table_num_to_num;
|
||||
size_t size = src.size();
|
||||
dst.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
if (it != table.end())
|
||||
dst[i] = it->second;
|
||||
else
|
||||
dst[i] = src[i];
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void executeImplNumToStringWithDefault(const PODArray<T> & src,
|
||||
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
|
||||
{
|
||||
const auto & table = *table_num_to_string;
|
||||
size_t size = src.size();
|
||||
dst_offsets.resize(size);
|
||||
ColumnString::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
auto it = table.find(src[i]);
|
||||
StringRef ref = it != table.end() ? it->second : dst_default;
|
||||
dst_data.resize(current_offset + ref.size);
|
||||
memcpy(&dst_data[current_offset], ref.data, ref.size);
|
||||
current_offset += ref.size;
|
||||
dst_offsets[i] = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
void executeImplStringToNumWithDefault(
|
||||
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
|
||||
PODArray<U> & dst, U dst_default)
|
||||
{
|
||||
const auto & table = *table_string_to_num;
|
||||
size_t size = src_offsets.size();
|
||||
dst.resize(size);
|
||||
ColumnString::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
StringRef ref{&src_data[current_offset], src_offsets[i] - current_offset};
|
||||
current_offset = src_offsets[i];
|
||||
auto it = table.find(ref);
|
||||
if (it != table.end())
|
||||
dst[i] = it->second;
|
||||
else
|
||||
dst[i] = dst_default;
|
||||
}
|
||||
}
|
||||
|
||||
template <bool with_default>
|
||||
void executeImplStringToString(
|
||||
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
|
||||
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets, StringRef dst_default)
|
||||
{
|
||||
const auto & table = *table_string_to_string;
|
||||
size_t size = src_offsets.size();
|
||||
dst_offsets.resize(size);
|
||||
ColumnString::Offset_t current_src_offset = 0;
|
||||
ColumnString::Offset_t current_dst_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
StringRef src_ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
|
||||
current_src_offset = src_offsets[i];
|
||||
|
||||
auto it = table.find(src_ref);
|
||||
|
||||
StringRef dst_ref = it != table.end() ? it->second : (with_default ? dst_default : src_ref);
|
||||
dst_data.resize(current_dst_offset + dst_ref.size);
|
||||
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
|
||||
current_dst_offset += dst_ref.size;
|
||||
dst_offsets[i] = current_dst_offset;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
/// Разные варианты хэш-таблиц для реализации отображения.
|
||||
|
||||
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
|
||||
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>;
|
||||
using StringToNum = HashMap<StringRef, UInt64>;
|
||||
using StringToString = HashMap<StringRef, StringRef>;
|
||||
|
||||
std::unique_ptr<NumToNum> table_num_to_num;
|
||||
std::unique_ptr<NumToString> table_num_to_string;
|
||||
std::unique_ptr<StringToNum> table_string_to_num;
|
||||
std::unique_ptr<StringToString> table_string_to_string;
|
||||
|
||||
Arena string_pool;
|
||||
|
||||
bool prepared = false;
|
||||
std::mutex mutex;
|
||||
|
||||
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
|
||||
void prepare(const Array & from, const Array & to)
|
||||
{
|
||||
if (prepared)
|
||||
return;
|
||||
|
||||
const size_t size = from.size();
|
||||
if (0 == size)
|
||||
return;
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
if (prepared)
|
||||
return;
|
||||
|
||||
if (from.size() != to.size())
|
||||
throw Exception("Second and third arguments of function " + getName() + " must be arrays of same size.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// Замечание: не делается проверка дубликатов в массиве from.
|
||||
|
||||
if (from[0].getType() != Field::Types::String && to[0].getType() != Field::Types::String)
|
||||
{
|
||||
table_num_to_num.reset(new NumToNum);
|
||||
auto & table = *table_num_to_num;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
table[from[i].get<UInt64>()] = to[i].get<UInt64>();
|
||||
}
|
||||
else if (from[0].getType() != Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
{
|
||||
table_num_to_string.reset(new NumToString);
|
||||
auto & table = *table_num_to_string;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; /// С завершающим нулём.
|
||||
table[from[i].get<UInt64>()] = ref;
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() != Field::Types::String)
|
||||
{
|
||||
table_string_to_num.reset(new StringToNum);
|
||||
auto & table = *table_string_to_num;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
StringRef ref{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
table[ref] = to[i].get<UInt64>();
|
||||
}
|
||||
}
|
||||
else if (from[0].getType() == Field::Types::String && to[0].getType() == Field::Types::String)
|
||||
{
|
||||
table_string_to_string.reset(new StringToString);
|
||||
auto & table = *table_string_to_string;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const String & str_from = from[i].get<const String &>();
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
StringRef ref_from{string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
|
||||
StringRef ref_to{string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
|
||||
table[ref_from] = ref_to;
|
||||
}
|
||||
}
|
||||
|
||||
prepared = true;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -337,6 +337,8 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionIsFinite>();
|
||||
factory.registerFunction<FunctionIsInfinite>();
|
||||
factory.registerFunction<FunctionIsNaN>();
|
||||
|
||||
factory.registerFunction<FunctionTransform>();
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user