#pragma once #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 ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int FUNCTION_NOT_ALLOWED; } template class FunctionsMultiStringFuzzySearch : public IFunction { static_assert(LimitArgs > 0); public: static constexpr auto name = Name::name; static FunctionPtr create(const Context & context) { if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) throw Exception( "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 3; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) throw Exception( "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!isUnsignedInteger(arguments[1])) throw Exception( "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const DataTypeArray * array_type = checkAndGetDataType(arguments[2].get()); if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) throw Exception( "Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return Impl::ReturnType(); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { using ResultType = typename Impl::ResultType; const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); const ColumnPtr & num_ptr = block.getByPosition(arguments[1]).column; const ColumnConst * col_const_num = nullptr; UInt32 edit_distance = 0; if ((col_const_num = checkAndGetColumnConst(num_ptr.get()))) edit_distance = col_const_num->getValue(); else if ((col_const_num = checkAndGetColumnConst(num_ptr.get()))) edit_distance = col_const_num->getValue(); else if ((col_const_num = checkAndGetColumnConst(num_ptr.get()))) edit_distance = col_const_num->getValue(); else throw Exception( "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The number is not const or does not fit in UInt32", ErrorCodes::ILLEGAL_COLUMN); const ColumnPtr & arr_ptr = block.getByPosition(arguments[2]).column; const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); if (!col_const_arr) throw Exception( "Illegal column " + block.getByPosition(arguments[2]).column->getName() + ". The array is not const", ErrorCodes::ILLEGAL_COLUMN); Array src_arr = col_const_arr->getValue(); if (src_arr.size() > LimitArgs) throw Exception( "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + ", should be at most " + std::to_string(LimitArgs), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); std::vector refs; refs.reserve(src_arr.size()); for (const auto & el : src_arr) refs.emplace_back(el.get()); auto col_res = ColumnVector::create(); auto col_offsets = ColumnArray::ColumnOffsets::create(); auto & vec_res = col_res->getData(); auto & offsets_res = col_offsets->getData(); /// The blame for resizing output is for the callee. if (col_haystack_vector) Impl::vector_constant( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res, edit_distance); else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); if constexpr (Impl::is_column_array) block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); else block.getByPosition(result).column = std::move(col_res); } }; }