mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
FunctionsExternalDictionaries refactored
This commit is contained in:
parent
a45459e095
commit
3d0778cba4
@ -40,7 +40,6 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionDictGetStringOrDefault>();
|
factory.registerFunction<FunctionDictGetStringOrDefault>();
|
||||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>();
|
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>();
|
||||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>();
|
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>();
|
||||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrNull>>();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
#include <DataTypes/DataTypeTuple.h>
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/DataTypeUUID.h>
|
#include <DataTypes/DataTypeUUID.h>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/DataTypeLowCardinality.h>
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/assert_cast.h>
|
#include <Common/assert_cast.h>
|
||||||
@ -118,6 +119,20 @@ public:
|
|||||||
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
|
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
DictionaryAttribute getDictionaryAttribute(std::shared_ptr<const IDictionaryBase> dictionary, const String& attribute_name) const
|
||||||
|
{
|
||||||
|
const DictionaryStructure & structure = dictionary->getStructure();
|
||||||
|
|
||||||
|
auto find_iter = std::find_if(structure.attributes.begin(), structure.attributes.end(), [&](const auto &attribute)
|
||||||
|
{
|
||||||
|
return attribute.name == attribute_name;
|
||||||
|
});
|
||||||
|
|
||||||
|
if (find_iter == structure.attributes.end())
|
||||||
|
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
|
||||||
|
|
||||||
|
return *find_iter;
|
||||||
|
}
|
||||||
private:
|
private:
|
||||||
const Context & context;
|
const Context & context;
|
||||||
const ExternalDictionariesLoader & external_loader;
|
const ExternalDictionariesLoader & external_loader;
|
||||||
@ -179,15 +194,24 @@ private:
|
|||||||
|
|
||||||
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
|
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
|
||||||
const auto key_column = key_column_with_type.column;
|
const auto key_column = key_column_with_type.column;
|
||||||
|
const auto key_column_type = WhichDataType(key_column_with_type.type);
|
||||||
|
|
||||||
if (dictionary_identifier_type == DictionaryIdentifierType::simple)
|
if (dictionary_identifier_type == DictionaryIdentifierType::simple)
|
||||||
{
|
{
|
||||||
|
if (!key_column_type.isUInt64())
|
||||||
|
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
|
||||||
|
ErrorCodes::TYPE_MISMATCH};
|
||||||
|
|
||||||
return dictionary->has({key_column}, {std::make_shared<DataTypeUInt64>()});
|
return dictionary->has({key_column}, {std::make_shared<DataTypeUInt64>()});
|
||||||
}
|
}
|
||||||
else if (dictionary_identifier_type == DictionaryIdentifierType::complex)
|
else if (dictionary_identifier_type == DictionaryIdentifierType::complex)
|
||||||
{
|
{
|
||||||
|
if (!key_column_type.isTuple())
|
||||||
|
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
|
||||||
|
ErrorCodes::TYPE_MISMATCH};
|
||||||
|
|
||||||
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
|
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
|
||||||
ColumnPtr key_column_full = key_column_with_type.column->convertToFullColumnIfConst();
|
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
|
||||||
|
|
||||||
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
|
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
|
||||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_column_with_type.type).getElements();
|
const auto & key_types = static_cast<const DataTypeTuple &>(*key_column_with_type.type).getElements();
|
||||||
@ -195,10 +219,7 @@ private:
|
|||||||
return dictionary->has(key_columns, key_types);
|
return dictionary->has(key_columns, key_types);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Has not supported for range dictionary", dictionary->getDictionaryID().getNameForLogs());
|
||||||
/// TODO: Add support for range
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
mutable FunctionDictHelper helper;
|
mutable FunctionDictHelper helper;
|
||||||
@ -207,82 +228,66 @@ private:
|
|||||||
enum class DictionaryGetFunctionType
|
enum class DictionaryGetFunctionType
|
||||||
{
|
{
|
||||||
get,
|
get,
|
||||||
getOrDefault,
|
getOrDefault
|
||||||
getOrNull
|
|
||||||
};
|
};
|
||||||
|
|
||||||
template <typename DataType, typename Name, DictionaryGetFunctionType dictionary_get_function_type>
|
/// This variant of function derives the result type automatically.
|
||||||
class FunctionDictGetImpl final : public IFunction
|
template <DictionaryGetFunctionType dictionary_get_function_type>
|
||||||
|
class FunctionDictGetNoType final : public IFunction
|
||||||
{
|
{
|
||||||
using Type = typename DataType::FieldType;
|
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault";
|
||||||
|
|
||||||
static FunctionPtr create(const Context & context, UInt32 dec_scale = 0)
|
static FunctionPtr create(const Context & context)
|
||||||
{
|
{
|
||||||
return std::make_shared<FunctionDictGetImpl>(context, dec_scale);
|
return std::make_shared<FunctionDictGetNoType>(context);
|
||||||
}
|
}
|
||||||
|
|
||||||
explicit FunctionDictGetImpl(const Context & context_, UInt32 dec_scale = 0)
|
explicit FunctionDictGetNoType(const Context & context_) : helper(context_) {}
|
||||||
: helper(context_)
|
|
||||||
, decimal_scale(dec_scale)
|
|
||||||
{}
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
private:
|
bool isVariadic() const override { return true; }
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
|
|
||||||
bool isVariadic() const override { return true; }
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const final { return true; }
|
bool useDefaultImplementationForConstants() const final { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
|
||||||
|
|
||||||
bool isDeterministic() const override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
|
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
|
||||||
{
|
{
|
||||||
DataTypePtr result;
|
return helper.isDictGetFunctionInjective(sample_columns);
|
||||||
/// TODO: Decimal will not work properly during FunctionDictGetImpl call decimal_scale will not be iniitalized.
|
|
||||||
if constexpr (IsDataTypeDecimal<DataType>)
|
|
||||||
result = std::make_shared<DataType>(DataType::maxPrecision(), decimal_scale);
|
|
||||||
else
|
|
||||||
result = std::make_shared<DataType>();
|
|
||||||
|
|
||||||
if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getOrNull)
|
|
||||||
{
|
|
||||||
result = std::make_shared<DataTypeNullable>(result);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return result;
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
|
||||||
{
|
{
|
||||||
if (arguments.size() < 3)
|
if (arguments.size() < 3)
|
||||||
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||||
|
|
||||||
|
String dictionary_name;
|
||||||
|
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
|
||||||
|
dictionary_name = name_col->getValue<String>();
|
||||||
|
else
|
||||||
|
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
|
||||||
|
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
|
||||||
|
attribute_name = name_col->getValue<String>();
|
||||||
|
else
|
||||||
|
throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName()
|
||||||
|
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
dictionary = helper.getDictionary(dictionary_name);
|
||||||
|
|
||||||
|
return helper.getDictionaryAttribute(dictionary, attribute_name).type;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||||
|
{
|
||||||
if (input_rows_count == 0)
|
if (input_rows_count == 0)
|
||||||
return result_type->createColumn();
|
return result_type->createColumn();
|
||||||
|
|
||||||
const auto * dictionary_name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
|
||||||
if (!dictionary_name_col)
|
|
||||||
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
|
|
||||||
|
|
||||||
String dictionary_name = dictionary_name_col->getValue<String>();
|
|
||||||
|
|
||||||
auto dictionary = helper.getDictionary(dictionary_name);
|
|
||||||
if (!dictionary)
|
|
||||||
throw Exception("First argument of function " + getName() + " does not name a dictionary", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
const auto * attr_name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get());
|
|
||||||
if (!attr_name_col)
|
|
||||||
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
|
|
||||||
|
|
||||||
String attr_name = attr_name_col->getValue<String>();
|
|
||||||
|
|
||||||
/// TODO: Use accurateCast if argument is integer
|
/// TODO: Use accurateCast if argument is integer
|
||||||
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
|
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
|
||||||
throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function "
|
throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function "
|
||||||
@ -319,19 +324,19 @@ private:
|
|||||||
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
|
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
|
||||||
{
|
{
|
||||||
if (current_arguments_index >= arguments.size())
|
if (current_arguments_index >= arguments.size())
|
||||||
throw Exception{"Wrong argument count for function test " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||||
|
|
||||||
default_col = arguments[current_arguments_index].column;
|
default_col = arguments[current_arguments_index].column;
|
||||||
}
|
}
|
||||||
|
|
||||||
ColumnPtr res;
|
ColumnPtr result;
|
||||||
|
|
||||||
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
|
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
|
||||||
const auto key_column = key_col_with_type.column;
|
const auto key_column = key_col_with_type.column;
|
||||||
|
|
||||||
if (dictionary_identifier_type == DictionaryIdentifierType::simple)
|
if (dictionary_identifier_type == DictionaryIdentifierType::simple)
|
||||||
{
|
{
|
||||||
res = dictionary->getColumn(attr_name, result_type, {key_column}, {std::make_shared<DataTypeUInt64>()}, default_col);
|
result = dictionary->getColumn(attribute_name, result_type, {key_column}, {std::make_shared<DataTypeUInt64>()}, default_col);
|
||||||
}
|
}
|
||||||
else if (dictionary_identifier_type == DictionaryIdentifierType::complex)
|
else if (dictionary_identifier_type == DictionaryIdentifierType::complex)
|
||||||
{
|
{
|
||||||
@ -341,21 +346,88 @@ private:
|
|||||||
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
|
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
|
||||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||||
|
|
||||||
res = dictionary->getColumn(attr_name, result_type, key_columns, key_types, default_col);
|
result = dictionary->getColumn(attribute_name, result_type, key_columns, key_types, default_col);
|
||||||
}
|
}
|
||||||
else if (dictionary_identifier_type == DictionaryIdentifierType::range)
|
else if (dictionary_identifier_type == DictionaryIdentifierType::range)
|
||||||
{
|
{
|
||||||
res = dictionary->getColumn(
|
result = dictionary->getColumn(
|
||||||
attr_name, result_type, {key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type}, default_col);
|
attribute_name, result_type, {key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type}, default_col);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS};
|
throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS};
|
||||||
|
|
||||||
return res;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
mutable FunctionDictHelper helper;
|
mutable FunctionDictHelper helper;
|
||||||
UInt32 decimal_scale;
|
/// Initialized in getReturnTypeImpl
|
||||||
|
mutable std::shared_ptr<const IDictionaryBase> dictionary;
|
||||||
|
mutable String attribute_name;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename DataType, typename Name, DictionaryGetFunctionType dictionary_get_function_type>
|
||||||
|
class FunctionDictGetImpl final : public IFunction
|
||||||
|
{
|
||||||
|
using Type = typename DataType::FieldType;
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
|
||||||
|
static FunctionPtr create(const Context &context)
|
||||||
|
{
|
||||||
|
return std::make_shared<FunctionDictGetImpl>(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
explicit FunctionDictGetImpl(const Context & context_) : impl(context_) {}
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
private:
|
||||||
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
|
|
||||||
|
bool isVariadic() const override { return true; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForConstants() const final { return true; }
|
||||||
|
|
||||||
|
bool isDeterministic() const override { return false; }
|
||||||
|
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
|
||||||
|
|
||||||
|
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
|
||||||
|
{
|
||||||
|
return impl.isInjective(sample_columns);
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
|
{
|
||||||
|
DataTypePtr result;
|
||||||
|
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
result = std::make_shared<DataType>(DataType::maxPrecision(), 0);
|
||||||
|
else
|
||||||
|
result = std::make_shared<DataType>();
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||||
|
{
|
||||||
|
auto return_type = impl.getReturnTypeImpl(arguments);
|
||||||
|
|
||||||
|
if (!areTypesEqual(return_type, result_type))
|
||||||
|
throw Exception{"Dictionary attribute has different type " + return_type->getName() + " expected " + result_type->getName(),
|
||||||
|
ErrorCodes::TYPE_MISMATCH};
|
||||||
|
|
||||||
|
return impl.executeImpl(arguments, return_type, input_rows_count);
|
||||||
|
}
|
||||||
|
|
||||||
|
static bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs)
|
||||||
|
{
|
||||||
|
return removeNullable(recursiveRemoveLowCardinality(lhs))->equals(*removeNullable(recursiveRemoveLowCardinality(rhs)));
|
||||||
|
}
|
||||||
|
|
||||||
|
const FunctionDictGetNoType<dictionary_get_function_type> impl;
|
||||||
};
|
};
|
||||||
|
|
||||||
template<typename DataType, typename Name>
|
template<typename DataType, typename Name>
|
||||||
@ -436,307 +508,6 @@ using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecim
|
|||||||
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
||||||
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
|
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
|
||||||
|
|
||||||
template<typename DataType, typename Name>
|
|
||||||
using FunctionDictGetOrNull = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::getOrNull>;
|
|
||||||
|
|
||||||
struct NameDictGetUInt8OrNull { static constexpr auto name = "dictGetUInt8OrNull"; };
|
|
||||||
struct NameDictGetUInt16OrNull { static constexpr auto name = "dictGetUInt16OrNull"; };
|
|
||||||
struct NameDictGetUInt32OrNull { static constexpr auto name = "dictGetUInt32OrNull"; };
|
|
||||||
struct NameDictGetUInt64OrNull { static constexpr auto name = "dictGetUInt64OrNull"; };
|
|
||||||
struct NameDictGetInt8OrNull { static constexpr auto name = "dictGetInt8OrNull"; };
|
|
||||||
struct NameDictGetInt16OrNull { static constexpr auto name = "dictGetInt16OrNull"; };
|
|
||||||
struct NameDictGetInt32OrNull { static constexpr auto name = "dictGetInt32OrNull"; };
|
|
||||||
struct NameDictGetInt64OrNull { static constexpr auto name = "dictGetInt64OrNull"; };
|
|
||||||
struct NameDictGetFloat32OrNull { static constexpr auto name = "dictGetFloat32OrNull"; };
|
|
||||||
struct NameDictGetFloat64OrNull { static constexpr auto name = "dictGetFloat64OrNull"; };
|
|
||||||
struct NameDictGetDateOrNull { static constexpr auto name = "dictGetDateOrNull"; };
|
|
||||||
struct NameDictGetDateTimeOrNull { static constexpr auto name = "dictGetDateTimeOrNull"; };
|
|
||||||
struct NameDictGetUUIDOrNull { static constexpr auto name = "dictGetUUIDOrNull"; };
|
|
||||||
struct NameDictGetDecimal32OrNull { static constexpr auto name = "dictGetDecimal32OrNull"; };
|
|
||||||
struct NameDictGetDecimal64OrNull { static constexpr auto name = "dictGetDecimal64OrNull"; };
|
|
||||||
struct NameDictGetDecimal128OrNull { static constexpr auto name = "dictGetDecimal128OrNull"; };
|
|
||||||
struct NameDictGetStringOrNull { static constexpr auto name = "dictGetStringOrNull"; };
|
|
||||||
|
|
||||||
using FunctionDictGetUInt8OrNull = FunctionDictGetOrNull<DataTypeUInt8, NameDictGetUInt8OrNull>;
|
|
||||||
using FunctionDictGetUInt16OrNull = FunctionDictGetOrNull<DataTypeUInt16, NameDictGetUInt16OrNull>;
|
|
||||||
using FunctionDictGetUInt32OrNull = FunctionDictGetOrNull<DataTypeUInt32, NameDictGetUInt32OrNull>;
|
|
||||||
using FunctionDictGetUInt64OrNull = FunctionDictGetOrNull<DataTypeUInt64, NameDictGetUInt64OrNull>;
|
|
||||||
using FunctionDictGetInt8OrNull = FunctionDictGetOrNull<DataTypeInt8, NameDictGetInt8OrNull>;
|
|
||||||
using FunctionDictGetInt16OrNull = FunctionDictGetOrNull<DataTypeInt16, NameDictGetInt16OrNull>;
|
|
||||||
using FunctionDictGetInt32OrNull = FunctionDictGetOrNull<DataTypeInt32, NameDictGetInt32OrNull>;
|
|
||||||
using FunctionDictGetInt64OrNull = FunctionDictGetOrNull<DataTypeInt64, NameDictGetInt64OrNull>;
|
|
||||||
using FunctionDictGetFloat32OrNull = FunctionDictGetOrNull<DataTypeFloat32, NameDictGetFloat32OrNull>;
|
|
||||||
using FunctionDictGetFloat64OrNull = FunctionDictGetOrNull<DataTypeFloat64, NameDictGetFloat64OrNull>;
|
|
||||||
using FunctionDictGetDateOrNull = FunctionDictGetOrNull<DataTypeDate, NameDictGetDateOrNull>;
|
|
||||||
using FunctionDictGetDateTimeOrNull = FunctionDictGetOrNull<DataTypeDateTime, NameDictGetDateTimeOrNull>;
|
|
||||||
using FunctionDictGetUUIDOrNull = FunctionDictGetOrNull<DataTypeUUID, NameDictGetUUIDOrNull>;
|
|
||||||
using FunctionDictGetDecimal32OrNull = FunctionDictGetOrNull<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrNull>;
|
|
||||||
using FunctionDictGetDecimal64OrNull = FunctionDictGetOrNull<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrNull>;
|
|
||||||
using FunctionDictGetDecimal128OrNull = FunctionDictGetOrNull<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrNull>;
|
|
||||||
using FunctionDictGetStringOrNull = FunctionDictGetOrNull<DataTypeString, NameDictGetStringOrNull>;
|
|
||||||
|
|
||||||
/// TODO: Use new API
|
|
||||||
/// This variant of function derives the result type automatically.
|
|
||||||
template <DictionaryGetFunctionType dictionary_get_function_type>
|
|
||||||
class FunctionDictGetNoType final : public IFunction
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet"
|
|
||||||
: (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault ? "dictGetOrDefault" : "dictGetOrNull");
|
|
||||||
|
|
||||||
static FunctionPtr create(const Context & context)
|
|
||||||
{
|
|
||||||
return std::make_shared<FunctionDictGetNoType>(context);
|
|
||||||
}
|
|
||||||
|
|
||||||
explicit FunctionDictGetNoType(const Context & context_) : context(context_), helper(context_) {}
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
private:
|
|
||||||
bool isVariadic() const override { return true; }
|
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const final { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
|
|
||||||
|
|
||||||
bool isDeterministic() const override { return false; }
|
|
||||||
|
|
||||||
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
|
|
||||||
{
|
|
||||||
return helper.isDictGetFunctionInjective(sample_columns);
|
|
||||||
}
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
||||||
{
|
|
||||||
if (arguments.size() < 3)
|
|
||||||
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
|
||||||
|
|
||||||
String dict_name;
|
|
||||||
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
|
|
||||||
{
|
|
||||||
dict_name = name_col->getValue<String>();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
|
|
||||||
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
|
||||||
|
|
||||||
String attr_name;
|
|
||||||
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
|
|
||||||
{
|
|
||||||
attr_name = name_col->getValue<String>();
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName()
|
|
||||||
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
|
||||||
|
|
||||||
auto dict = helper.getDictionary(dict_name);
|
|
||||||
const DictionaryStructure & structure = dict->getStructure();
|
|
||||||
|
|
||||||
for (const auto& attribute : structure.attributes)
|
|
||||||
{
|
|
||||||
if (attribute.name != attr_name)
|
|
||||||
{
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
WhichDataType dt = removeNullable(attribute.type);
|
|
||||||
|
|
||||||
if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::get)
|
|
||||||
{
|
|
||||||
switch (dt.idx)
|
|
||||||
{
|
|
||||||
case TypeIndex::String:
|
|
||||||
case TypeIndex::FixedString:
|
|
||||||
impl = FunctionDictGetString::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt8:
|
|
||||||
impl = FunctionDictGetUInt8::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt16:
|
|
||||||
impl = FunctionDictGetUInt16::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt32:
|
|
||||||
impl = FunctionDictGetUInt32::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt64:
|
|
||||||
impl = FunctionDictGetUInt64::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int8:
|
|
||||||
impl = FunctionDictGetInt8::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int16:
|
|
||||||
impl = FunctionDictGetInt16::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int32:
|
|
||||||
impl = FunctionDictGetInt32::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int64:
|
|
||||||
impl = FunctionDictGetInt64::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float32:
|
|
||||||
impl = FunctionDictGetFloat32::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float64:
|
|
||||||
impl = FunctionDictGetFloat64::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Date:
|
|
||||||
impl = FunctionDictGetDate::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::DateTime:
|
|
||||||
impl = FunctionDictGetDateTime::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UUID:
|
|
||||||
impl = FunctionDictGetUUID::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal32:
|
|
||||||
impl = FunctionDictGetDecimal32::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal64:
|
|
||||||
impl = FunctionDictGetDecimal64::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal128:
|
|
||||||
impl = FunctionDictGetDecimal128::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw Exception("Unknown dictGet type", ErrorCodes::UNKNOWN_TYPE);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
|
|
||||||
{
|
|
||||||
switch (dt.idx)
|
|
||||||
{
|
|
||||||
case TypeIndex::String:
|
|
||||||
impl = FunctionDictGetStringOrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt8:
|
|
||||||
impl = FunctionDictGetUInt8OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt16:
|
|
||||||
impl = FunctionDictGetUInt16OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt32:
|
|
||||||
impl = FunctionDictGetUInt32OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt64:
|
|
||||||
impl = FunctionDictGetUInt64OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int8:
|
|
||||||
impl = FunctionDictGetInt8OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int16:
|
|
||||||
impl = FunctionDictGetInt16OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int32:
|
|
||||||
impl = FunctionDictGetInt32OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int64:
|
|
||||||
impl = FunctionDictGetInt64OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float32:
|
|
||||||
impl = FunctionDictGetFloat32OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float64:
|
|
||||||
impl = FunctionDictGetFloat64OrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Date:
|
|
||||||
impl = FunctionDictGetDateOrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::DateTime:
|
|
||||||
impl = FunctionDictGetDateTimeOrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UUID:
|
|
||||||
impl = FunctionDictGetUUIDOrDefault::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal32:
|
|
||||||
impl = FunctionDictGetDecimal32OrDefault::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal64:
|
|
||||||
impl = FunctionDictGetDecimal64OrDefault::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal128:
|
|
||||||
impl = FunctionDictGetDecimal128OrDefault::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw Exception("Unknown dictGetOrDefault type", ErrorCodes::UNKNOWN_TYPE);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if constexpr (dictionary_get_function_type == DictionaryGetFunctionType::getOrNull)
|
|
||||||
{
|
|
||||||
switch (dt.idx)
|
|
||||||
{
|
|
||||||
case TypeIndex::String:
|
|
||||||
impl = FunctionDictGetStringOrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt8:
|
|
||||||
impl = FunctionDictGetUInt8OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt16:
|
|
||||||
impl = FunctionDictGetUInt16OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt32:
|
|
||||||
impl = FunctionDictGetUInt32OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UInt64:
|
|
||||||
impl = FunctionDictGetUInt64OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int8:
|
|
||||||
impl = FunctionDictGetInt8OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int16:
|
|
||||||
impl = FunctionDictGetInt16OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int32:
|
|
||||||
impl = FunctionDictGetInt32OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Int64:
|
|
||||||
impl = FunctionDictGetInt64OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float32:
|
|
||||||
impl = FunctionDictGetFloat32OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Float64:
|
|
||||||
impl = FunctionDictGetFloat64OrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Date:
|
|
||||||
impl = FunctionDictGetDateOrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::DateTime:
|
|
||||||
impl = FunctionDictGetDateTimeOrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::UUID:
|
|
||||||
impl = FunctionDictGetUUIDOrNull::create(context);
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal32:
|
|
||||||
impl = FunctionDictGetDecimal32OrNull::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal64:
|
|
||||||
impl = FunctionDictGetDecimal64OrNull::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
case TypeIndex::Decimal128:
|
|
||||||
impl = FunctionDictGetDecimal128OrNull::create(context, getDecimalScale(*attribute.type));
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw Exception("Unknown dictGetOrNull type", ErrorCodes::UNKNOWN_TYPE);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return attribute.type;
|
|
||||||
}
|
|
||||||
|
|
||||||
throw Exception{"No such attribute '" + attr_name + "'", ErrorCodes::BAD_ARGUMENTS};
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
|
||||||
{
|
|
||||||
return impl->executeImpl(arguments, result_type, input_rows_count);
|
|
||||||
}
|
|
||||||
|
|
||||||
const Context & context;
|
|
||||||
mutable FunctionDictHelper helper;
|
|
||||||
mutable FunctionPtr impl; // underlying function used by dictGet function without explicit type info
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Functions to work with hierarchies.
|
/// Functions to work with hierarchies.
|
||||||
|
|
||||||
class FunctionDictGetHierarchy final : public IFunction
|
class FunctionDictGetHierarchy final : public IFunction
|
||||||
|
Loading…
Reference in New Issue
Block a user