#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int UNSUPPORTED_METHOD; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; extern const int TYPE_MISMATCH; } /** Functions that use plug-ins (external) dictionaries_loader. * * Get the value of the attribute of the specified type. * dictGetType(dictionary, attribute, id), * 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. * * Get an array of identifiers, consisting of the source and parents chain. * dictGetHierarchy(dictionary, id). * * Is the first identifier the child of the second. * dictIsIn(dictionary, child_id, parent_id). */ class FunctionDictHelper { public: explicit FunctionDictHelper(const Context & context_) : context(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { auto dict = context.getExternalDictionariesLoader().getDictionary(dictionary_name, context); if (!access_checked) { context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName()); access_checked = true; } return dict; } std::shared_ptr getDictionary(const ColumnWithTypeAndName & column) { const auto * dict_name_col = checkAndGetColumnConst(column.column.get()); return getDictionary(dict_name_col->getValue()); } bool isDictGetFunctionInjective(const Block & sample_columns) { /// Assume non-injective by default if (!sample_columns) return false; if (sample_columns.columns() < 3) throw Exception{"Wrong arguments count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto * dict_name_col = checkAndGetColumnConst(sample_columns.getByPosition(0).column.get()); if (!dict_name_col) throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; const auto * attr_name_col = checkAndGetColumnConst(sample_columns.getByPosition(1).column.get()); if (!attr_name_col) throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; return getDictionary(dict_name_col->getValue())->isInjective(attr_name_col->getValue()); } DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { return context.getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, context); } private: const Context & context; /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic 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(const Context & context) { return std::make_shared(context); } explicit FunctionDictHas(const Context & context_) : helper(context_) {} String getName() const override { return name; } private: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } bool isDeterministic() const override { return false; } bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2) throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (!WhichDataType(arguments[1]).isUInt64() && !isTuple(arguments[1])) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { /** 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. */ if (input_rows_count == 0) return result_type->createColumn(); auto dictionary = helper.getDictionary(arguments[0]); auto dictionary_key_type = dictionary->getKeyType(); 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); ColumnPtr range_col = nullptr; DataTypePtr range_col_type = nullptr; if (dictionary_key_type == DictionaryKeyType::range) { if (arguments.size() != 3) throw Exception{"Wrong argument count for function " + getName() + " when dictionary has key type range", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; range_col = arguments[2].column; range_col_type = arguments[2].type; if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64))) throw Exception{"Illegal type " + range_col_type->getName() + " of fourth argument of function " + getName() + " must be convertible to Int64.", ErrorCodes::ILLEGAL_COLUMN}; } if (dictionary_key_type == DictionaryKeyType::simple) { if (!key_column_type.isUInt64()) throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(), ErrorCodes::TYPE_MISMATCH}; return dictionary->hasKeys({key_column}, {std::make_shared()}); } else if (dictionary_key_type == DictionaryKeyType::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. ColumnPtr key_column_full = key_column->convertToFullColumnIfConst(); const auto & key_columns = typeid_cast(*key_column_full).getColumnsCopy(); const auto & key_types = static_cast(*key_column_with_type.type).getElements(); return dictionary->hasKeys(key_columns, key_types); } else return dictionary->hasKeys({key_column, range_col}, {std::make_shared(), range_col_type}); } mutable FunctionDictHelper helper; }; enum class DictionaryGetFunctionType { get, getOrDefault }; /// This variant of function derives the result type automatically. template class FunctionDictGetNoType final : public IFunction { public: static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault"; static FunctionPtr create(const Context & context) { return std::make_shared(context); } explicit FunctionDictGetNoType(const Context & context_) : helper(context_) {} String getName() const override { return name; } 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 dictionary_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) dictionary_name = name_col->getValue(); else throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; 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) return std::make_shared(types); 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(arguments[0].column.get())) dictionary_name = name_col->getValue(); else throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; 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{"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; 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{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; 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{"Illegal type " + range_col_type->getName() + " of fourth argument of function " + getName() + " must be convertible to Int64.", ErrorCodes::ILLEGAL_COLUMN}; ++current_arguments_index; } Columns default_cols; if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault) { if (current_arguments_index >= arguments.size()) throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto & column_before_cast = arguments[current_arguments_index]; if (const DataTypeTuple * type_tuple = typeid_cast(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()); } 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()); 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) { const auto * tuple_column = checkAndGetColumn(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 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) { 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 ({}).", getName(), key_col_with_type.type->getName()); if (attribute_names.size() > 1) { const auto & result_tuple_type = assert_cast(*result_type); Columns result_columns = dictionary->getColumns( attribute_names, result_tuple_type.getElements(), {key_column}, {std::make_shared()}, default_cols); result = ColumnTuple::create(std::move(result_columns)); } else result = dictionary->getColumn( attribute_names[0], result_type, {key_column}, {std::make_shared()}, default_cols.front()); } else if (dictionary_key_type == DictionaryKeyType::complex) { 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 ({}).", 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(); 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 ({}).", getName(), key_col_with_type.type->getName()); const auto & key_columns = typeid_cast(*key_column_full).getColumnsCopy(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); if (attribute_names.size() > 1) { const auto & result_tuple_type = assert_cast(*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) { 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 ({}).", getName(), key_col_with_type.type->getName()); if (attribute_names.size() > 1) { const auto & result_tuple_type = assert_cast(*result_type); Columns result_columns = dictionary->getColumns( attribute_names, result_tuple_type.getElements(), {key_column, range_col}, {std::make_shared(), 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(), range_col_type}, default_cols.front()); } else throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS}; return result; } private: Strings getAttributeNamesFromColumn(const ColumnPtr & column, const DataTypePtr & type) const { Strings attribute_names; if (const auto * name_col = checkAndGetColumnConst(column.get())) attribute_names.emplace_back(name_col->getValue()); else if (const auto * tuple_col_const = checkAndGetColumnConst(column.get())) { const ColumnTuple & tuple_col = assert_cast(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(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 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(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) result = std::make_shared(DataType::maxPrecision(), 0); else result = std::make_shared(); 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 impl; }; template using FunctionDictGet = FunctionDictGetImpl; 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; using FunctionDictGetUInt16 = FunctionDictGet; using FunctionDictGetUInt32 = FunctionDictGet; using FunctionDictGetUInt64 = FunctionDictGet; using FunctionDictGetInt8 = FunctionDictGet; using FunctionDictGetInt16 = FunctionDictGet; using FunctionDictGetInt32 = FunctionDictGet; using FunctionDictGetInt64 = FunctionDictGet; using FunctionDictGetFloat32 = FunctionDictGet; using FunctionDictGetFloat64 = FunctionDictGet; using FunctionDictGetDate = FunctionDictGet; using FunctionDictGetDateTime = FunctionDictGet; using FunctionDictGetUUID = FunctionDictGet; using FunctionDictGetDecimal32 = FunctionDictGet, NameDictGetDecimal32>; using FunctionDictGetDecimal64 = FunctionDictGet, NameDictGetDecimal64>; using FunctionDictGetDecimal128 = FunctionDictGet, NameDictGetDecimal128>; using FunctionDictGetString = FunctionDictGet; template using FunctionDictGetOrDefault = FunctionDictGetImpl; 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; using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault; using FunctionDictGetUInt32OrDefault = FunctionDictGetOrDefault; using FunctionDictGetUInt64OrDefault = FunctionDictGetOrDefault; using FunctionDictGetInt8OrDefault = FunctionDictGetOrDefault; using FunctionDictGetInt16OrDefault = FunctionDictGetOrDefault; using FunctionDictGetInt32OrDefault = FunctionDictGetOrDefault; using FunctionDictGetInt64OrDefault = FunctionDictGetOrDefault; using FunctionDictGetFloat32OrDefault = FunctionDictGetOrDefault; using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault; using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault; using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault; using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault; using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault, NameDictGetDecimal32OrDefault>; using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault, NameDictGetDecimal64OrDefault>; using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault, NameDictGetDecimal128OrDefault>; using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault; /// Functions to work with hierarchies. class FunctionDictGetHierarchy final : public IFunction { public: static constexpr auto name = "dictGetHierarchy"; static FunctionPtr create(const Context & context) { return std::make_shared(context); } explicit FunctionDictGetHierarchy(const Context & context_) : helper(context_) {} String getName() const override { return name; } private: size_t getNumberOfArguments() const override { return 2; } bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; } bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (!WhichDataType(arguments[1]).isUInt64()) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return std::make_shared(std::make_shared()); } bool isDeterministic() const override { return false; } 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(); auto dictionary = helper.getDictionary(arguments[0]); 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()); return result; } mutable FunctionDictHelper helper; }; class FunctionDictIsIn final : public IFunction { public: static constexpr auto name = "dictIsIn"; static FunctionPtr create(const Context & context) { return std::make_shared(context); } explicit FunctionDictIsIn(const Context & context_) : helper(context_) {} String getName() const override { return name; } private: size_t getNumberOfArguments() const override { return 3; } bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (!WhichDataType(arguments[1]).isUInt64()) throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; if (!WhichDataType(arguments[2]).isUInt64()) throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return std::make_shared(); } bool isDeterministic() const override { return false; } 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(); auto dict = helper.getDictionary(arguments[0]); 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()); return res; } mutable FunctionDictHelper helper; }; }