ClickHouse/src/Functions/FunctionsExternalDictionaries.h

1180 lines
49 KiB
C++
Raw Normal View History

#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
2017-07-13 20:58:19 +00:00
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
2021-03-31 14:46:13 +00:00
#include <Columns/ColumnNullable.h>
#include <Access/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
2021-02-27 20:39:34 +00:00
#include <Interpreters/castColumn.h>
2021-05-17 07:30:42 +00:00
#include <Functions/IFunction.h>
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
#include <Functions/FunctionHelpers.h>
2017-06-06 17:18:32 +00:00
#include <ext/range.h>
2015-04-16 06:12:35 +00:00
#include <type_traits>
namespace DB
{
2016-01-12 02:21:15 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int UNSUPPORTED_METHOD;
2017-06-13 02:06:53 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
2021-01-21 14:42:50 +00:00
extern const int TYPE_MISMATCH;
2016-01-12 02:21:15 +00:00
}
/** Functions that use plug-ins (external) dictionaries_loader.
*
2017-05-13 22:19:04 +00:00
* Get the value of the attribute of the specified type.
* dictGetType(dictionary, attribute, id),
2017-05-13 22:19:04 +00:00
* Type - placeholder for the type name, any numeric and string types are currently supported.
* The type must match the actual attribute type with which it was declared in the dictionary structure.
*
2017-05-13 22:19:04 +00:00
* Get an array of identifiers, consisting of the source and parents chain.
* dictGetHierarchy(dictionary, id).
*
2017-05-13 22:19:04 +00:00
* Is the first identifier the child of the second.
* dictIsIn(dictionary, child_id, parent_id).
*/
class FunctionDictHelper : WithConstContext
{
public:
explicit FunctionDictHelper(ContextConstPtr context_) : WithConstContext(context_) {}
std::shared_ptr<const IDictionary> getDictionary(const String & dictionary_name)
{
auto dict = getContext()->getExternalDictionariesLoader().getDictionary(dictionary_name, getContext());
if (!access_checked)
{
getContext()->checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName());
access_checked = true;
}
return dict;
}
2021-03-26 13:57:21 +00:00
std::shared_ptr<const IDictionary> getDictionary(const ColumnPtr & column)
{
2021-03-26 13:57:21 +00:00
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(column.get());
if (!dict_name_col)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected const String column");
return getDictionary(dict_name_col->getValue<String>());
}
2020-10-14 14:04:50 +00:00
bool isDictGetFunctionInjective(const Block & sample_columns)
{
/// Assume non-injective by default
2020-10-14 14:04:50 +00:00
if (!sample_columns)
return false;
if (sample_columns.columns() < 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong arguments count");
2020-10-18 14:49:19 +00:00
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function dictGet must be a constant string");
2020-10-18 14:49:19 +00:00
const auto * attr_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function dictGet... must be a constant string");
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
DictionaryStructure getDictionaryStructure(const String & dictionary_name) const
{
return getContext()->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, getContext());
}
private:
/// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated.
std::atomic<bool> access_checked = false;
/// We must not cache dictionary or dictionary's structure here, because there are places
/// where ExpressionActionsPtr is cached (StorageDistributed caching it for sharding_key_expr and
/// optimize_skip_unused_shards), and if the dictionary will be cached within "query" then
/// cached ExpressionActionsPtr will always have first version of the query and the dictionary
/// will not be updated after reload (see https://github.com/ClickHouse/ClickHouse/pull/16205)
};
class FunctionDictHas final : public IFunction
{
public:
static constexpr auto name = "dictHas";
static FunctionPtr create(ContextConstPtr context)
{
return std::make_shared<FunctionDictHas>(context);
}
explicit FunctionDictHas(ContextConstPtr context_) : helper(context_) {}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
bool isDeterministic() const override { return false; }
2018-03-16 01:22:02 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
2018-03-16 01:22:02 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {}",
getName());
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function, expected a string",
arguments[0]->getName(),
getName());
if (!WhichDataType(arguments[1]).isUInt64() &&
!isTuple(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of second argument of function {} must be UInt64 or tuple(...)",
arguments[1]->getName(),
getName());
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2020-10-14 14:04:50 +00:00
/** Do not require existence of the dictionary if the function is called for empty columns.
* This is needed to allow successful query analysis on a server,
* that is the initiator of a distributed query,
* in the case when the function will be invoked for real data only at the remote servers.
* This feature is controversial and implemented specially
* for backward compatibility with the case in Yandex Banner System.
*/
2018-09-14 18:55:12 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2021-03-26 13:57:21 +00:00
auto dictionary = helper.getDictionary(arguments[0].column);
auto dictionary_key_type = dictionary->getKeyType();
2019-09-11 11:14:55 +00:00
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
const auto key_column = key_column_with_type.column;
const auto key_column_type = WhichDataType(key_column_with_type.type);
2020-10-18 14:49:19 +00:00
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range)
{
if (arguments.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {} when dictionary has key type range",
getName());
range_col = arguments[2].column;
range_col_type = arguments[2].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of fourth argument of function {} must be convertible to Int64.",
range_col_type->getName(),
getName());
}
if (dictionary_key_type == DictionaryKeyType::simple)
{
if (!key_column_type.isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is simple. Actual type {}.",
getName(),
key_column_with_type.type->getName());
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()});
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!key_column_type.isTuple())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be tuple when dictionary is complex. Actual type {}.",
getName(),
key_column_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
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();
return dictionary->hasKeys(key_columns, key_types);
}
else
{
if (!key_column_type.isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument of function {} must be UInt64 when dictionary is range. Actual type {}.",
getName(),
key_column_with_type.type->getName());
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
}
}
2021-03-31 14:46:13 +00:00
private:
mutable FunctionDictHelper helper;
};
enum class DictionaryGetFunctionType
{
get,
getOrDefault
};
/// 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" : "dictGetOrDefault";
static FunctionPtr create(ContextConstPtr context)
{
return std::make_shared<FunctionDictGetNoType>(context);
}
explicit FunctionDictGetNoType(ContextConstPtr context_) : helper(context_) {}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
2018-03-16 03:03:17 +00:00
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(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Wrong argument count for function {}",
getName());
String dictionary_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function {}, expected a const string.",
arguments[0].type->getName(),
getName());
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
DataTypes types;
auto dictionary_structure = helper.getDictionaryStructure(dictionary_name);
for (auto & attribute_name : attribute_names)
{
/// We're extracting the return type from the dictionary's config, without loading the dictionary.
auto attribute = dictionary_structure.getAttribute(attribute_name);
types.emplace_back(attribute.type);
}
if (types.size() > 1)
2021-03-31 14:46:13 +00:00
return std::make_shared<DataTypeTuple>(types, attribute_names);
else
return types.front();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
String dictionary_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of first argument of function {}, expected a const string.",
arguments[0].type->getName(),
getName());
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
auto dictionary = helper.getDictionary(dictionary_name);
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of third argument of function {}, must be UInt64 or tuple(...).",
arguments[2].type->getName(),
getName());
auto dictionary_key_type = dictionary->getKeyType();
size_t current_arguments_index = 3;
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range)
{
if (current_arguments_index >= arguments.size())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {} should be {}",
getName(),
arguments.size(),
arguments.size() + 1);
range_col = arguments[current_arguments_index].column;
range_col_type = arguments[current_arguments_index].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal type {} of fourth argument of function must be convertible to Int64.",
range_col_type->getName(),
getName());
++current_arguments_index;
}
Columns default_cols;
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
{
if (current_arguments_index >= arguments.size())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {} should be {}",
getName(),
arguments.size(),
arguments.size() + 1);
2021-02-27 20:39:34 +00:00
const auto & column_before_cast = arguments[current_arguments_index];
2021-02-27 20:39:34 +00:00
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(column_before_cast.type.get()))
{
const DataTypes & nested_types = type_tuple->getElements();
for (const auto & nested_type : nested_types)
if (nested_type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values column nullable is not supported",
getName());
2021-02-27 20:39:34 +00:00
}
else if (column_before_cast.type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values column nullable is not supported",
getName());
2021-02-27 20:39:34 +00:00
auto result_type_no_nullable = removeNullable(result_type);
ColumnWithTypeAndName column_to_cast = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name};
auto result = castColumnAccurate(column_to_cast, result_type_no_nullable);
if (attribute_names.size() > 1)
{
2021-02-27 20:39:34 +00:00
const auto * tuple_column = checkAndGetColumn<ColumnTuple>(result.get());
if (!tuple_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values column must be tuple",
getName());
if (tuple_column->tupleSize() != attribute_names.size())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function {} default values tuple column must contain same column size as requested attributes",
getName());
default_cols = tuple_column->getColumnsCopy();
}
else
2021-02-27 20:39:34 +00:00
default_cols.emplace_back(result);
}
else
{
for (size_t i = 0; i < attribute_names.size(); ++i)
default_cols.emplace_back(nullptr);
}
ColumnPtr result;
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
const auto key_column = key_col_with_type.column;
if (dictionary_key_type == DictionaryKeyType::simple)
{
2021-03-05 14:12:50 +00:00
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be UInt64 when dictionary is simple. Actual type {}.",
2021-03-05 14:12:50 +00:00
getName(),
key_col_with_type.type->getName());
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
{key_column},
{std::make_shared<DataTypeUInt64>()},
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
{key_column},
{std::make_shared<DataTypeUInt64>()},
default_cols.front());
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!isTuple(key_col_with_type.type))
2021-03-05 14:12:50 +00:00
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be tuple when dictionary is complex. Actual type {}.",
2021-03-05 14:12:50 +00:00
getName(),
key_col_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_col_with_type.column->convertToFullColumnIfConst();
2021-03-01 11:34:34 +00:00
if (!isTuple(key_col_with_type.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be tuple when dictionary is complex. Actual type {}.",
2021-03-01 11:34:34 +00:00
getName(),
key_col_with_type.type->getName());
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();
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
key_columns,
key_types,
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
key_columns,
key_types,
default_cols.front());
}
else if (dictionary_key_type == DictionaryKeyType::range)
{
2021-03-05 14:12:50 +00:00
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be UInt64 when dictionary is range. Actual type {}.",
2021-03-05 14:12:50 +00:00
getName(),
key_col_with_type.type->getName());
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
{key_column, range_col},
{std::make_shared<DataTypeUInt64>(), range_col_type},
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
{key_column, range_col},
{std::make_shared<DataTypeUInt64>(), range_col_type},
default_cols.front());
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type");
2020-12-19 13:24:51 +00:00
return result;
}
private:
Strings getAttributeNamesFromColumn(const ColumnPtr & column, const DataTypePtr & type) const
{
Strings attribute_names;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(column.get()))
attribute_names.emplace_back(name_col->getValue<String>());
else if (const auto * tuple_col_const = checkAndGetColumnConst<ColumnTuple>(column.get()))
{
const ColumnTuple & tuple_col = assert_cast<const ColumnTuple &>(tuple_col_const->getDataColumn());
size_t tuple_size = tuple_col.tupleSize();
if (tuple_size < 1)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function {} must contain multiple constant string columns");
for (size_t i = 0; i < tuple_col.tupleSize(); ++i)
{
const auto * tuple_column = tuple_col.getColumnPtr(i).get();
const auto * attribute_name_column = checkAndGetColumn<ColumnString>(tuple_column);
if (!attribute_name_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function {} must contain multiple constant string columns",
getName());
attribute_names.emplace_back(attribute_name_column->getDataAt(0));
}
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of second argument of function {}, expected a const string or const tuple of const strings.",
type->getName(),
getName());
return attribute_names;
}
mutable FunctionDictHelper helper;
};
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(ContextConstPtr context)
{
return std::make_shared<FunctionDictGetImpl>(context);
}
explicit FunctionDictGetImpl(ContextConstPtr 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;
2021-01-12 21:26:32 +00:00
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>
using FunctionDictGet = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::get>;
struct NameDictGetUInt8 { static constexpr auto name = "dictGetUInt8"; };
struct NameDictGetUInt16 { static constexpr auto name = "dictGetUInt16"; };
struct NameDictGetUInt32 { static constexpr auto name = "dictGetUInt32"; };
struct NameDictGetUInt64 { static constexpr auto name = "dictGetUInt64"; };
struct NameDictGetInt8 { static constexpr auto name = "dictGetInt8"; };
struct NameDictGetInt16 { static constexpr auto name = "dictGetInt16"; };
struct NameDictGetInt32 { static constexpr auto name = "dictGetInt32"; };
struct NameDictGetInt64 { static constexpr auto name = "dictGetInt64"; };
struct NameDictGetFloat32 { static constexpr auto name = "dictGetFloat32"; };
struct NameDictGetFloat64 { static constexpr auto name = "dictGetFloat64"; };
struct NameDictGetDate { static constexpr auto name = "dictGetDate"; };
struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
struct NameDictGetString { static constexpr auto name = "dictGetString"; };
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>;
using FunctionDictGetUInt16 = FunctionDictGet<DataTypeUInt16, NameDictGetUInt16>;
using FunctionDictGetUInt32 = FunctionDictGet<DataTypeUInt32, NameDictGetUInt32>;
using FunctionDictGetUInt64 = FunctionDictGet<DataTypeUInt64, NameDictGetUInt64>;
using FunctionDictGetInt8 = FunctionDictGet<DataTypeInt8, NameDictGetInt8>;
using FunctionDictGetInt16 = FunctionDictGet<DataTypeInt16, NameDictGetInt16>;
using FunctionDictGetInt32 = FunctionDictGet<DataTypeInt32, NameDictGetInt32>;
using FunctionDictGetInt64 = FunctionDictGet<DataTypeInt64, NameDictGetInt64>;
using FunctionDictGetFloat32 = FunctionDictGet<DataTypeFloat32, NameDictGetFloat32>;
using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64, NameDictGetFloat64>;
using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>;
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>;
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
using FunctionDictGetString = FunctionDictGet<DataTypeString, NameDictGetString>;
template<typename DataType, typename Name>
using FunctionDictGetOrDefault = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::getOrDefault>;
struct NameDictGetUInt8OrDefault { static constexpr auto name = "dictGetUInt8OrDefault"; };
struct NameDictGetUInt16OrDefault { static constexpr auto name = "dictGetUInt16OrDefault"; };
struct NameDictGetUInt32OrDefault { static constexpr auto name = "dictGetUInt32OrDefault"; };
struct NameDictGetUInt64OrDefault { static constexpr auto name = "dictGetUInt64OrDefault"; };
struct NameDictGetInt8OrDefault { static constexpr auto name = "dictGetInt8OrDefault"; };
struct NameDictGetInt16OrDefault { static constexpr auto name = "dictGetInt16OrDefault"; };
struct NameDictGetInt32OrDefault { static constexpr auto name = "dictGetInt32OrDefault"; };
struct NameDictGetInt64OrDefault { static constexpr auto name = "dictGetInt64OrDefault"; };
struct NameDictGetFloat32OrDefault { static constexpr auto name = "dictGetFloat32OrDefault"; };
struct NameDictGetFloat64OrDefault { static constexpr auto name = "dictGetFloat64OrDefault"; };
struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault"; };
struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault"; };
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
struct NameDictGetStringOrDefault { static constexpr auto name = "dictGetStringOrDefault"; };
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>;
using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16, NameDictGetUInt16OrDefault>;
using FunctionDictGetUInt32OrDefault = FunctionDictGetOrDefault<DataTypeUInt32, NameDictGetUInt32OrDefault>;
using FunctionDictGetUInt64OrDefault = FunctionDictGetOrDefault<DataTypeUInt64, NameDictGetUInt64OrDefault>;
using FunctionDictGetInt8OrDefault = FunctionDictGetOrDefault<DataTypeInt8, NameDictGetInt8OrDefault>;
using FunctionDictGetInt16OrDefault = FunctionDictGetOrDefault<DataTypeInt16, NameDictGetInt16OrDefault>;
using FunctionDictGetInt32OrDefault = FunctionDictGetOrDefault<DataTypeInt32, NameDictGetInt32OrDefault>;
using FunctionDictGetInt64OrDefault = FunctionDictGetOrDefault<DataTypeInt64, NameDictGetInt64OrDefault>;
using FunctionDictGetFloat32OrDefault = FunctionDictGetOrDefault<DataTypeFloat32, NameDictGetFloat32OrDefault>;
using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64, NameDictGetFloat64OrDefault>;
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>;
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>;
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
2021-03-31 14:46:13 +00:00
class FunctionDictGetOrNull final : public IFunction
{
public:
static constexpr auto name = "dictGetOrNull";
static FunctionPtr create(ContextConstPtr context)
2021-03-31 14:46:13 +00:00
{
return std::make_shared<FunctionDictGetOrNull>(context);
}
explicit FunctionDictGetOrNull(ContextConstPtr context_)
2021-03-31 14:46:13 +00:00
: dictionary_get_func_impl(context_)
, dictionary_has_func_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 override { return true; }
bool useDefaultImplementationForNulls() const override { return false; }
bool isDeterministic() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; }
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
{
return dictionary_get_func_impl.isInjective(sample_columns);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments);
WhichDataType result_data_type(result_type);
if (result_data_type.isTuple())
{
const auto & data_type_tuple = static_cast<const DataTypeTuple &>(*result_type);
auto elements_types_copy = data_type_tuple.getElements();
for (auto & element_type : elements_types_copy)
element_type = makeNullable(element_type);
result_type = std::make_shared<DataTypeTuple>(elements_types_copy, data_type_tuple.getElementNames());
}
else
result_type = makeNullable(result_type);
return result_type;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
2021-03-31 14:46:13 +00:00
{
if (input_rows_count == 0)
return result_type->createColumn();
2021-03-31 19:56:30 +00:00
/** We call dictHas function to get which map is key presented in dictionary.
For key that presented in dictionary dict has result for that key index value will be 1. Otherwise 0.
We invert result, and then for key that is not presented in dictionary value will be 1. Otherwise 0.
This inverted result will be used as null column map.
After that we call dict get function, by contract for key that are not presented in dictionary we
return default value.
We create nullable column from dict get result column and null column map.
2 additional implementation details:
1. Result from dict get can be tuple if client requested multiple attributes we apply such operation on each result column.
2. If column is already nullable we merge column null map with null map that we get from dict has.
*/
2021-03-31 14:46:13 +00:00
2021-04-01 12:06:08 +00:00
auto dict_has_arguments = filterAttributeNameArgumentForDictHas(arguments);
2021-03-31 14:46:13 +00:00
auto is_key_in_dictionary_column = dictionary_has_func_impl.executeImpl(dict_has_arguments, std::make_shared<DataTypeUInt8>(), input_rows_count);
auto is_key_in_dictionary_column_mutable = is_key_in_dictionary_column->assumeMutable();
ColumnVector<UInt8> & is_key_in_dictionary_column_typed = assert_cast<ColumnVector<UInt8> &>(*is_key_in_dictionary_column_mutable);
2021-03-31 19:56:30 +00:00
PaddedPODArray<UInt8> & is_key_in_dictionary_data = is_key_in_dictionary_column_typed.getData();
for (auto & key : is_key_in_dictionary_data)
key = !key;
2021-03-31 14:46:13 +00:00
auto dictionary_get_result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments);
auto dictionary_get_result_column = dictionary_get_func_impl.executeImpl(arguments, dictionary_get_result_type, input_rows_count);
2021-03-31 14:46:13 +00:00
ColumnPtr result;
WhichDataType dictionary_get_result_data_type(dictionary_get_result_type);
2021-03-31 14:46:13 +00:00
auto dictionary_get_result_column_mutable = dictionary_get_result_column->assumeMutable();
if (dictionary_get_result_data_type.isTuple())
2021-03-31 14:46:13 +00:00
{
ColumnTuple & column_tuple = assert_cast<ColumnTuple &>(*dictionary_get_result_column_mutable);
const auto & columns = column_tuple.getColumns();
size_t tuple_size = columns.size();
MutableColumns new_columns(tuple_size);
for (size_t tuple_column_index = 0; tuple_column_index < tuple_size; ++tuple_column_index)
{
auto nullable_column_map = ColumnVector<UInt8>::create();
auto & nullable_column_map_data = nullable_column_map->getData();
nullable_column_map_data.assign(is_key_in_dictionary_data);
auto mutable_column = columns[tuple_column_index]->assumeMutable();
if (ColumnNullable * nullable_column = typeid_cast<ColumnNullable *>(mutable_column.get()))
{
auto & null_map_data = nullable_column->getNullMapData();
addNullMap(null_map_data, is_key_in_dictionary_data);
new_columns[tuple_column_index] = std::move(mutable_column);
}
else
new_columns[tuple_column_index] = ColumnNullable::create(std::move(mutable_column), std::move(nullable_column_map));
}
result = ColumnTuple::create(std::move(new_columns));
}
else
{
if (ColumnNullable * nullable_column = typeid_cast<ColumnNullable *>(dictionary_get_result_column_mutable.get()))
{
auto & null_map_data = nullable_column->getNullMapData();
addNullMap(null_map_data, is_key_in_dictionary_data);
result = std::move(dictionary_get_result_column);
}
else
result = ColumnNullable::create(std::move(dictionary_get_result_column), std::move(is_key_in_dictionary_column_mutable));
}
return result;
}
static void addNullMap(PaddedPODArray<UInt8> & null_map, PaddedPODArray<UInt8> & null_map_to_add)
{
assert(null_map.size() == null_map_to_add.size());
for (size_t i = 0; i < null_map.size(); ++i)
null_map[i] = null_map[i] || null_map_to_add[i];
}
2021-04-01 12:06:08 +00:00
static ColumnsWithTypeAndName filterAttributeNameArgumentForDictHas(const ColumnsWithTypeAndName & arguments)
2021-03-31 19:56:30 +00:00
{
ColumnsWithTypeAndName dict_has_arguments;
dict_has_arguments.reserve(arguments.size() - 1);
2021-04-01 12:06:08 +00:00
size_t attribute_name_argument_index = 1;
2021-03-31 19:56:30 +00:00
for (size_t i = 0; i < arguments.size(); ++i)
{
2021-04-01 12:06:08 +00:00
if (i == attribute_name_argument_index)
2021-03-31 19:56:30 +00:00
continue;
dict_has_arguments.emplace_back(arguments[i]);
}
return dict_has_arguments;
}
2021-03-31 14:46:13 +00:00
const FunctionDictGetNoType<DictionaryGetFunctionType::get> dictionary_get_func_impl;
const FunctionDictHas dictionary_has_func_impl;
};
/// Functions to work with hierarchies.
class FunctionDictGetHierarchy final : public IFunction
{
public:
static constexpr auto name = "dictGetHierarchy";
static FunctionPtr create(ContextConstPtr context)
{
return std::make_shared<FunctionDictGetHierarchy>(context);
}
explicit FunctionDictGetHierarchy(ContextConstPtr context_) : helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 2; }
2020-10-14 14:04:50 +00:00
bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function {}. Expected String. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[1]->getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2018-04-24 07:16:39 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2021-03-26 13:57:21 +00:00
auto dictionary = helper.getDictionary(arguments[0].column);
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary {} does not support hierarchy",
dictionary->getFullName());
ColumnPtr result = dictionary->getHierarchy(arguments[1].column, std::make_shared<DataTypeUInt64>());
return result;
}
mutable FunctionDictHelper helper;
};
class FunctionDictIsIn final : public IFunction
{
public:
static constexpr auto name = "dictIsIn";
static FunctionPtr create(ContextConstPtr context)
{
return std::make_shared<FunctionDictIsIn>(context);
}
explicit FunctionDictIsIn(ContextConstPtr context_)
: helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 3; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function {}. Expected String. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[1]->getName());
if (!WhichDataType(arguments[2]).isUInt64())
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected UInt64. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[2]->getName());
return std::make_shared<DataTypeUInt8>();
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2018-04-24 07:16:39 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2021-03-26 13:57:21 +00:00
auto dict = helper.getDictionary(arguments[0].column);
2019-09-11 11:14:55 +00:00
if (!dict->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary {} does not support hierarchy",
dict->getFullName());
ColumnPtr res = dict->isInHierarchy(arguments[1].column, arguments[2].column, std::make_shared<DataTypeUInt64>());
return res;
}
mutable FunctionDictHelper helper;
};
class FunctionDictGetChildren final : public IFunction
{
public:
static constexpr auto name = "dictGetChildren";
static FunctionPtr create(ContextConstPtr context)
{
return std::make_shared<FunctionDictGetChildren>(context);
}
explicit FunctionDictGetChildren(ContextConstPtr context_)
: helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
2021-03-26 13:57:21 +00:00
bool isDeterministic() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function {}. Expected String. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[1]->getName());
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
2021-03-26 13:57:21 +00:00
auto dictionary = helper.getDictionary(arguments[0].column);
2021-03-26 13:57:21 +00:00
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary {} does not support hierarchy",
2021-03-26 13:57:21 +00:00
dictionary->getFullName());
2021-03-26 13:57:21 +00:00
ColumnPtr result = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), 1);
2021-03-26 13:57:21 +00:00
return result;
}
mutable FunctionDictHelper helper;
};
2021-03-25 07:31:12 +00:00
class FunctionDictGetDescendants final : public IFunction
{
public:
2021-03-25 07:31:12 +00:00
static constexpr auto name = "dictGetDescendants";
static FunctionPtr create(ContextConstPtr context)
{
2021-03-25 07:31:12 +00:00
return std::make_shared<FunctionDictGetDescendants>(context);
}
explicit FunctionDictGetDescendants(ContextConstPtr context_)
: helper(context_) {}
String getName() const override { return name; }
private:
2021-03-26 13:57:21 +00:00
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
bool useDefaultImplementationForConstants() const final { return true; }
2021-03-26 13:57:21 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
bool isDeterministic() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
2021-03-26 13:57:21 +00:00
size_t arguments_size = arguments.size();
if (arguments_size < 2 || arguments_size > 3)
{
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Illegal arguments size of function {}. Expects 2 or 3 arguments size. Actual size {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments_size);
}
if (!isString(arguments[0]))
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of first argument of function {}. Expected const String. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[0]->getName());
if (!WhichDataType(arguments[1]).isUInt64())
2021-03-26 13:57:21 +00:00
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of second argument of function {}. Expected UInt64. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[1]->getName());
2021-03-26 13:57:21 +00:00
if (arguments.size() == 3 && !isUnsignedInteger(arguments[2]))
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer. Actual type {}",
2021-03-26 13:57:21 +00:00
getName(),
arguments[2]->getName());
}
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
2021-03-26 13:57:21 +00:00
auto dictionary = helper.getDictionary(arguments[0].column);
2021-03-26 13:57:21 +00:00
size_t level = 0;
2021-03-26 13:57:21 +00:00
if (arguments.size() == 3)
{
if (!isColumnConst(*arguments[2].column))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type of third argument of function {}. Expected const unsigned integer.",
2021-03-26 13:57:21 +00:00
getName());
level = static_cast<size_t>(arguments[2].column->get64(0));
}
if (!dictionary->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Dictionary {} does not support hierarchy",
2021-03-26 13:57:21 +00:00
dictionary->getFullName());
2021-03-26 13:57:21 +00:00
ColumnPtr res = dictionary->getDescendants(arguments[1].column, std::make_shared<DataTypeUInt64>(), level);
return res;
}
mutable FunctionDictHelper helper;
};
2018-08-26 00:52:58 +00:00
}