dbms: function 'transform': added support for non-const argument with default value [#METR-15987].

This commit is contained in:
Alexey Milovidov 2015-06-03 06:29:28 +03:00
parent 9c78e95702
commit c341bd2c60
5 changed files with 890 additions and 574 deletions

View File

@ -3,7 +3,6 @@
#include <Poco/Net/DNS.h>
#include <math.h>
#include <mutex>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
@ -22,7 +21,6 @@
#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 <DB/Interpreters/ExpressionActions.h>
#include <statdaemons/ext/range.hpp>
@ -57,8 +55,6 @@ namespace DB
* sleep(n) - спит n секунд каждый блок.
*
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
*
* transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
*/
@ -892,505 +888,4 @@ using FunctionIsInfinite = FunctionNumericPredicate<IsInfiniteImpl>;
using FunctionIsNaN = FunctionNumericPredicate<IsNaNImpl>;
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
/** 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);
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
}
/// TODO Больше проверок.
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(), block, arguments);
const auto in = block.getByPosition(arguments.front()).column.get();
if (in->isConst())
{
executeConst(block, arguments, result);
return;
}
auto column_result = block.getByPosition(result).type->createColumn();
auto out = column_result.get();
if (!executeNum<UInt8>(in, out)
&& !executeNum<UInt16>(in, out)
&& !executeNum<UInt32>(in, out)
&& !executeNum<UInt64>(in, out)
&& !executeNum<Int8>(in, out)
&& !executeNum<Int16>(in, out)
&& !executeNum<Int32>(in, out)
&& !executeNum<Int64>(in, out)
&& !executeNum<Float32>(in, out)
&& !executeNum<Float64>(in, out)
&& !executeString(in, out))
throw Exception(
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = column_result;
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
Block tmp_block;
ColumnNumbers tmp_arguments;
tmp_block.insert(block.getByPosition(arguments[0]));
tmp_block.getByPosition(0).column = static_cast<IColumnConst *>(tmp_block.getByPosition(0).column->cloneResized(1).get())->convertToFullColumn();
tmp_arguments.push_back(0);
for (size_t i = 1; i < arguments.size(); ++i)
{
tmp_block.insert(block.getByPosition(arguments[i]));
tmp_arguments.push_back(i);
}
tmp_block.insert(block.getByPosition(result));
size_t tmp_result = arguments.size();
execute(tmp_block, tmp_arguments, tmp_result);
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
block.rowsInFirstColumn(),
(*tmp_block.getByPosition(tmp_result).column)[0]);
}
template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped)
{
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)
&& !executeNumToNumWithDefault<T, UInt16>(in, out_untyped)
&& !executeNumToNumWithDefault<T, UInt32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, UInt64>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int8>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int16>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Int64>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Float32>(in, out_untyped)
&& !executeNumToNumWithDefault<T, Float64>(in, out_untyped)
&& !executeNumToString<T>(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
bool executeString(const IColumn * in_untyped, IColumn * out_untyped)
{
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
{
if (!executeStringToNum<UInt8>(in, out_untyped)
&& !executeStringToNum<UInt16>(in, out_untyped)
&& !executeStringToNum<UInt32>(in, out_untyped)
&& !executeStringToNum<UInt64>(in, out_untyped)
&& !executeStringToNum<Int8>(in, out_untyped)
&& !executeStringToNum<Int16>(in, out_untyped)
&& !executeStringToNum<Int32>(in, out_untyped)
&& !executeStringToNum<Int64>(in, out_untyped)
&& !executeStringToNum<Float32>(in, out_untyped)
&& !executeStringToNum<Float64>(in, out_untyped)
&& !executeStringToString(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
return false;
}
template <typename T, typename U>
bool executeNumToNumWithDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
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)
{
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)
{
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)
{
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())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
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())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
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())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
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;
}
}
/// Разные варианты хэш-таблиц для реализации отображения.
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
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;
Field default_value; /// Null, если не задано.
bool prepared = false;
std::mutex mutex;
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
{
if (prepared)
return;
const size_t size = from.size();
if (0 == size)
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
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);
Array converted_to;
const Array * used_to = &to;
/// Задано ли значение по-умолчанию.
if (arguments.size() == 4)
{
const IColumnConst * default_col = dynamic_cast<const IColumnConst *>(&*block.getByPosition(arguments[3]).column);
if (!default_col)
throw Exception("Fourth argument of function " + getName() + " (default value) must be constant", ErrorCodes::ILLEGAL_COLUMN);
default_value = (*default_col)[0];
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
if (default_value.getType() == Field::Types::Float64 && to[0].getType() != Field::Types::Float64)
{
converted_to.resize(to.size());
for (size_t i = 0, size = to.size(); i < size; ++i)
converted_to[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
used_to = &converted_to;
}
else if (default_value.getType() != Field::Types::Float64 && to[0].getType() == Field::Types::Float64)
{
default_value = apply_visitor(FieldVisitorConvertToNumber<Float64>(), default_value);
}
}
/// Замечание: не делается проверка дубликатов в массиве 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>()] = (*used_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] = (*used_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;
}
};
}

View File

@ -0,0 +1,804 @@
#pragma once
#include <mutex>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Common/Arena.h>
#include <DB/Core/StringRef.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Functions/IFunction.h>
namespace DB
{
/** transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
*/
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
/** 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);
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
}
/// TODO Больше проверок.
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(), block, arguments);
const auto in = block.getByPosition(arguments.front()).column.get();
if (in->isConst())
{
executeConst(block, arguments, result);
return;
}
const IColumn * default_column = nullptr;
if (arguments.size() == 4)
default_column = block.getByPosition(arguments[3]).column.get();
auto column_result = block.getByPosition(result).type->createColumn();
auto out = column_result.get();
if (!executeNum<UInt8>(in, out, default_column)
&& !executeNum<UInt16>(in, out, default_column)
&& !executeNum<UInt32>(in, out, default_column)
&& !executeNum<UInt64>(in, out, default_column)
&& !executeNum<Int8>(in, out, default_column)
&& !executeNum<Int16>(in, out, default_column)
&& !executeNum<Int32>(in, out, default_column)
&& !executeNum<Int64>(in, out, default_column)
&& !executeNum<Float32>(in, out, default_column)
&& !executeNum<Float64>(in, out, default_column)
&& !executeString(in, out, default_column))
throw Exception(
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = column_result;
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// Составим блок из полноценных столбцов размера 1 и вычислим функцию как обычно.
Block tmp_block;
ColumnNumbers tmp_arguments;
tmp_block.insert(block.getByPosition(arguments[0]));
tmp_block.getByPosition(0).column = static_cast<IColumnConst *>(tmp_block.getByPosition(0).column->cloneResized(1).get())->convertToFullColumn();
tmp_arguments.push_back(0);
for (size_t i = 1; i < arguments.size(); ++i)
{
tmp_block.insert(block.getByPosition(arguments[i]));
tmp_arguments.push_back(i);
}
tmp_block.insert(block.getByPosition(result));
size_t tmp_result = arguments.size();
execute(tmp_block, tmp_arguments, tmp_result);
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(
block.rowsInFirstColumn(),
(*tmp_block.getByPosition(tmp_result).column)[0]);
}
template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped)
{
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
{
if (!default_untyped)
{
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 (default_untyped->isConst())
{
if (!executeNumToNumWithConstDefault<T, UInt8>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt16>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, UInt64>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int8>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int16>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Int64>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Float32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Float64>(in, out_untyped)
&& !executeNumToStringWithConstDefault<T>(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else
{
if (!executeNumToNumWithNonConstDefault<T, UInt8>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt16>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, UInt64>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int8>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int16>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Int64>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Float32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Float64>(in, out_untyped, default_untyped)
&& !executeNumToStringWithNonConstDefault<T>(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
bool executeString(const IColumn * in_untyped, IColumn * out_untyped, const IColumn * default_untyped)
{
if (const auto in = typeid_cast<const ColumnString *>(in_untyped))
{
if (!default_untyped)
{
if (!executeStringToString(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else if (default_untyped->isConst())
{
if (!executeStringToNumWithConstDefault<UInt8>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt16>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<UInt64>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int8>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int16>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Int64>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Float32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Float64>(in, out_untyped)
&& !executeStringToStringWithConstDefault(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
else
{
if (!executeStringToNumWithNonConstDefault<UInt8>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt16>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<UInt64>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int8>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int16>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Int64>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Float32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Float64>(in, out_untyped, default_untyped)
&& !executeStringToStringWithNonConstDefault(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
return false;
}
template <typename T, typename U>
bool executeNumToNumWithConstDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplNumToNumWithConstDefault<T, U>(in->getData(), out->getData(), const_default_value.get<U>());
return true;
}
template <typename T, typename U>
bool executeNumToNumWithNonConstDefault(const ColumnVector<T> * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
if (!executeNumToNumWithNonConstDefault2<T, U, UInt8>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt16>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, UInt64>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int8>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int16>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Int64>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
template <typename T, typename U, typename V>
bool executeNumToNumWithNonConstDefault2(const ColumnVector<T> * in, ColumnVector<U> * out, const IColumn * default_untyped)
{
auto col_default = typeid_cast<const ColumnVector<V> *>(default_untyped);
if (!col_default)
return false;
executeImplNumToNumWithNonConstDefault<T, U, V>(in->getData(), out->getData(), col_default->getData());
return true;
}
template <typename T>
bool executeNumToStringWithConstDefault(const ColumnVector<T> * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
const String & default_str = const_default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplNumToStringWithConstDefault<T>(in->getData(), out->getChars(), out->getOffsets(), default_string_ref);
return true;
}
template <typename T>
bool executeNumToStringWithNonConstDefault(const ColumnVector<T> * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplNumToStringWithNonConstDefault<T>(
in->getData(),
out->getChars(), out->getOffsets(),
default_col->getChars(), default_col->getOffsets());
return true;
}
template <typename U>
bool executeStringToNumWithConstDefault(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
executeImplStringToNumWithConstDefault<U>(in->getChars(), in->getOffsets(), out->getData(), const_default_value.get<U>());
return true;
}
template <typename U>
bool executeStringToNumWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnVector<U> *>(out_untyped);
if (!out)
return false;
if (!executeStringToNumWithNonConstDefault2<U, UInt8>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt16>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, UInt64>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int8>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int16>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Int64>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
return true;
}
template <typename U, typename V>
bool executeStringToNumWithNonConstDefault2(const ColumnString * in, ColumnVector<U> * out, const IColumn * default_untyped)
{
auto col_default = typeid_cast<const ColumnVector<V> *>(default_untyped);
if (!col_default)
return false;
executeImplStringToNumWithNonConstDefault<U, V>(in->getChars(), in->getOffsets(), out->getData(), col_default->getData());
return true;
}
bool executeStringToString(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
executeImplStringToString(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets());
return true;
}
bool executeStringToStringWithConstDefault(const ColumnString * in, IColumn * out_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
const String & default_str = const_default_value.get<const String &>();
StringRef default_string_ref{default_str.data(), default_str.size() + 1};
executeImplStringToStringWithConstDefault(in->getChars(), in->getOffsets(), out->getChars(), out->getOffsets(), default_string_ref);
return true;
}
bool executeStringToStringWithNonConstDefault(const ColumnString * in, IColumn * out_untyped, const IColumn * default_untyped)
{
auto out = typeid_cast<ColumnString *>(out_untyped);
if (!out)
return false;
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
executeImplStringToStringWithNonConstDefault(
in->getChars(), in->getOffsets(),
out->getChars(), out->getOffsets(),
default_col->getChars(), default_col->getOffsets());
return true;
}
template <typename T, typename U>
void executeImplNumToNumWithConstDefault(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())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default;
}
}
template <typename T, typename U, typename V>
void executeImplNumToNumWithNonConstDefault(const PODArray<T> & src, PODArray<U> & dst, const PODArray<V> & 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())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default[i];
}
}
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())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = src[i];
}
}
template <typename T>
void executeImplNumToStringWithConstDefault(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_dst_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_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
current_dst_offset += ref.size;
dst_offsets[i] = current_dst_offset;
}
}
template <typename T>
void executeImplNumToStringWithNonConstDefault(const PODArray<T> & src,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets,
const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets_t & dst_default_offsets)
{
const auto & table = *table_num_to_string;
size_t size = src.size();
dst_offsets.resize(size);
ColumnString::Offset_t current_dst_offset = 0;
ColumnString::Offset_t current_dst_default_offset = 0;
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
StringRef ref;
if (it != table.end())
ref = it->second;
else
{
ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);
ref.size = dst_default_offsets[i] - current_dst_default_offset;
}
dst_data.resize(current_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
current_dst_offset += ref.size;
current_dst_default_offset = dst_default_offsets[i];
dst_offsets[i] = current_dst_offset;
}
}
template <typename U>
void executeImplStringToNumWithConstDefault(
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_src_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = dst_default;
}
}
template <typename U, typename V>
void executeImplStringToNumWithNonConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
PODArray<U> & dst, const PODArray<V> & dst_default)
{
const auto & table = *table_string_to_num;
size_t size = src_offsets.size();
dst.resize(size);
ColumnString::Offset_t current_src_offset = 0;
for (size_t i = 0; i < size; ++i)
{
StringRef ref{&src_data[current_src_offset], src_offsets[i] - current_src_offset};
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
else
dst[i] = dst_default[i];
}
}
template <bool with_default>
void executeImplStringToStringWithOrWithoutConstDefault(
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;
}
}
void executeImplStringToString(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets)
{
executeImplStringToStringWithOrWithoutConstDefault<false>(src_data, src_offsets, dst_data, dst_offsets, {});
}
void executeImplStringToStringWithConstDefault(
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)
{
executeImplStringToStringWithOrWithoutConstDefault<true>(src_data, src_offsets, dst_data, dst_offsets, dst_default);
}
void executeImplStringToStringWithNonConstDefault(
const ColumnString::Chars_t & src_data, const ColumnString::Offsets_t & src_offsets,
ColumnString::Chars_t & dst_data, ColumnString::Offsets_t & dst_offsets,
const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets_t & dst_default_offsets)
{
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;
ColumnString::Offset_t current_dst_default_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;
if (it != table.end())
dst_ref = it->second;
else
{
dst_ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);
dst_ref.size = dst_default_offsets[i] - current_dst_default_offset;
}
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;
current_dst_default_offset = dst_default_offsets[i];
dst_offsets[i] = current_dst_offset;
}
}
/// Разные варианты хэш-таблиц для реализации отображения.
using NumToNum = HashMap<UInt64, UInt64, HashCRC32<UInt64>>;
using NumToString = HashMap<UInt64, StringRef, HashCRC32<UInt64>>; /// Везде StringRef-ы с завершающим нулём.
using StringToNum = HashMap<StringRef, UInt64, StringRefHash>;
using StringToString = HashMap<StringRef, StringRef, StringRefHash>;
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;
Field const_default_value; /// Null, если не задано.
bool prepared = false;
std::mutex mutex;
/// Может вызываться из разных потоков. Срабатывает только при первом вызове.
void prepare(const Array & from, const Array & to, Block & block, const ColumnNumbers & arguments)
{
if (prepared)
return;
const size_t size = from.size();
if (0 == size)
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
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);
Array converted_to;
const Array * used_to = &to;
/// Задано ли значение по-умолчанию.
if (arguments.size() == 4)
{
const IColumnConst * default_col = dynamic_cast<const IColumnConst *>(&*block.getByPosition(arguments[3]).column);
if (default_col)
{
const_default_value = (*default_col)[0];
/// Нужно ли преобразовать элементы to и default_value к наименьшему общему типу, который является Float64?
if (const_default_value.getType() == Field::Types::Float64 && to[0].getType() != Field::Types::Float64)
{
converted_to.resize(to.size());
for (size_t i = 0, size = to.size(); i < size; ++i)
converted_to[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), to[i]);
used_to = &converted_to;
}
else if (const_default_value.getType() != Field::Types::Float64 && to[0].getType() == Field::Types::Float64)
{
const_default_value = apply_visitor(FieldVisitorConvertToNumber<Float64>(), const_default_value);
}
}
/// TODO Преобразование для to в случае неконстантного default
}
/// Замечание: не делается проверка дубликатов в массиве 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>()] = (*used_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] = (*used_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;
}
};
}

View File

@ -20,7 +20,7 @@ VerticalRowOutputStream::VerticalRowOutputStream(WriteBuffer & ostr_, const Bloc
typedef std::vector<size_t> Widths_t;
Widths_t name_widths(columns);
size_t max_name_width = 0;
for (size_t i = 0; i < columns; ++i)
{
data_types[i] = sample.getByPosition(i).type;
@ -41,7 +41,7 @@ void VerticalRowOutputStream::writeField(const Field & field)
writeEscapedString(names[field_number], ostr);
writeCString(": ", ostr);
writeString(pads[field_number], ostr);
data_types[field_number]->serializeTextEscaped(field, ostr);
writeChar('\n', ostr);

View File

@ -1,6 +1,5 @@
#include <math.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsArithmetic.h>
#include <DB/Functions/FunctionsMiscellaneous.h>
@ -310,70 +309,6 @@ void FunctionVisibleWidth::execute(Block & block, const ColumnNumbers & argument
}
/// TODO: Убрать copy-paste из FunctionsConditional.h
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
template <typename T1, typename T2>
DataTypePtr getSmallestCommonNumericTypeImpl()
{
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
template <typename T1>
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
}
@ -403,8 +338,6 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
factory.registerFunction<FunctionIsFinite>();
factory.registerFunction<FunctionIsInfinite>();
factory.registerFunction<FunctionIsNaN>();
factory.registerFunction<FunctionTransform>();
}
}

View File

@ -0,0 +1,84 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsTransform.h>
namespace DB
{
/// TODO: Убрать copy-paste из FunctionsConditional.h
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
template <typename T1, typename T2>
DataTypePtr getSmallestCommonNumericTypeImpl()
{
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
template <typename T1>
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
}
namespace DB
{
void registerFunctionsMiscellaneous(FunctionFactory & factory)
{
factory.registerFunction<FunctionTransform>();
}
}