| 1 | #pragma once |
| 2 | |
| 3 | #include <DataTypes/DataTypesNumber.h> |
| 4 | #include <DataTypes/DataTypesDecimal.h> |
| 5 | #include <DataTypes/DataTypeArray.h> |
| 6 | #include <DataTypes/DataTypeString.h> |
| 7 | #include <DataTypes/DataTypeDate.h> |
| 8 | #include <DataTypes/DataTypeDateTime.h> |
| 9 | #include <DataTypes/DataTypeTuple.h> |
| 10 | #include <DataTypes/DataTypeUUID.h> |
| 11 | |
| 12 | #include <Common/typeid_cast.h> |
| 13 | #include <Common/assert_cast.h> |
| 14 | |
| 15 | #include <Columns/ColumnsNumber.h> |
| 16 | #include <Columns/ColumnConst.h> |
| 17 | #include <Columns/ColumnArray.h> |
| 18 | #include <Columns/ColumnString.h> |
| 19 | #include <Columns/ColumnTuple.h> |
| 20 | |
| 21 | #include <Interpreters/Context.h> |
| 22 | #include <Interpreters/ExternalDictionariesLoader.h> |
| 23 | |
| 24 | #include <Functions/IFunctionImpl.h> |
| 25 | #include <Functions/FunctionHelpers.h> |
| 26 | |
| 27 | #include <Dictionaries/FlatDictionary.h> |
| 28 | #include <Dictionaries/HashedDictionary.h> |
| 29 | #include <Dictionaries/CacheDictionary.h> |
| 30 | #include <Dictionaries/ComplexKeyHashedDictionary.h> |
| 31 | #include <Dictionaries/ComplexKeyCacheDictionary.h> |
| 32 | #include <Dictionaries/RangeHashedDictionary.h> |
| 33 | #include <Dictionaries/TrieDictionary.h> |
| 34 | |
| 35 | #include <ext/range.h> |
| 36 | |
| 37 | #include <type_traits> |
| 38 | |
| 39 | namespace DB |
| 40 | { |
| 41 | |
| 42 | namespace ErrorCodes |
| 43 | { |
| 44 | extern const int DICTIONARIES_WAS_NOT_LOADED; |
| 45 | extern const int UNSUPPORTED_METHOD; |
| 46 | extern const int UNKNOWN_TYPE; |
| 47 | extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; |
| 48 | extern const int TYPE_MISMATCH; |
| 49 | extern const int ILLEGAL_COLUMN; |
| 50 | extern const int BAD_ARGUMENTS; |
| 51 | extern const int DICTIONARY_ACCESS_DENIED; |
| 52 | } |
| 53 | |
| 54 | /** Functions that use plug-ins (external) dictionaries_loader. |
| 55 | * |
| 56 | * Get the value of the attribute of the specified type. |
| 57 | * dictGetType(dictionary, attribute, id), |
| 58 | * Type - placeholder for the type name, any numeric and string types are currently supported. |
| 59 | * The type must match the actual attribute type with which it was declared in the dictionary structure. |
| 60 | * |
| 61 | * Get an array of identifiers, consisting of the source and parents chain. |
| 62 | * dictGetHierarchy(dictionary, id). |
| 63 | * |
| 64 | * Is the first identifier the child of the second. |
| 65 | * dictIsIn(dictionary, child_id, parent_id). |
| 66 | */ |
| 67 | |
| 68 | |
| 69 | class FunctionDictHas final : public IFunction |
| 70 | { |
| 71 | public: |
| 72 | static constexpr auto name = "dictHas" ; |
| 73 | |
| 74 | static FunctionPtr create(const Context & context) |
| 75 | { |
| 76 | return std::make_shared<FunctionDictHas>(context.getExternalDictionariesLoader(), context); |
| 77 | } |
| 78 | |
| 79 | FunctionDictHas(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) |
| 80 | : dictionaries_loader(dictionaries_loader_) |
| 81 | , context(context_) {} |
| 82 | |
| 83 | String getName() const override { return name; } |
| 84 | |
| 85 | private: |
| 86 | size_t getNumberOfArguments() const override { return 2; } |
| 87 | |
| 88 | bool useDefaultImplementationForConstants() const final { return true; } |
| 89 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } |
| 90 | |
| 91 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 92 | { |
| 93 | if (!isString(arguments[0])) |
| 94 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 95 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 96 | |
| 97 | if (!WhichDataType(arguments[1]).isUInt64() && |
| 98 | !isTuple(arguments[1])) |
| 99 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 100 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 101 | |
| 102 | return std::make_shared<DataTypeUInt8>(); |
| 103 | } |
| 104 | |
| 105 | bool isDeterministic() const override { return false; } |
| 106 | |
| 107 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 108 | { |
| 109 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 110 | if (!dict_name_col) |
| 111 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 112 | |
| 113 | /** Do not require existence of the dictionary if the function is called for empty block. |
| 114 | * This is needed to allow successful query analysis on a server, |
| 115 | * that is the initiator of a distributed query, |
| 116 | * in the case when the function will be invoked for real data only at the remote servers. |
| 117 | * This feature is controversial and implemented specially |
| 118 | * for backward compatibility with the case in Yandex Banner System. |
| 119 | */ |
| 120 | if (input_rows_count == 0) |
| 121 | { |
| 122 | auto & elem = block.getByPosition(result); |
| 123 | elem.column = elem.type->createColumn(); |
| 124 | return; |
| 125 | } |
| 126 | |
| 127 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 128 | const auto dict_ptr = dict.get(); |
| 129 | |
| 130 | if (!context.hasDictionaryAccessRights(dict_ptr->getFullName())) |
| 131 | { |
| 132 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 133 | + dict->getFullName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 134 | } |
| 135 | |
| 136 | if (!executeDispatchSimple<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 137 | !executeDispatchSimple<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 138 | !executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) && |
| 139 | !executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) && |
| 140 | !executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) && |
| 141 | !executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr)) |
| 142 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 143 | } |
| 144 | |
| 145 | template <typename DictionaryType> |
| 146 | bool executeDispatchSimple( |
| 147 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 148 | { |
| 149 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 150 | if (!dict) |
| 151 | return false; |
| 152 | |
| 153 | const auto id_col_untyped = block.getByPosition(arguments[1]).column.get(); |
| 154 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 155 | { |
| 156 | const auto & ids = id_col->getData(); |
| 157 | |
| 158 | auto out = ColumnUInt8::create(ext::size(ids)); |
| 159 | dict->has(ids, out->getData()); |
| 160 | block.getByPosition(result).column = std::move(out); |
| 161 | } |
| 162 | else |
| 163 | throw Exception{"Second argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 164 | |
| 165 | return true; |
| 166 | } |
| 167 | |
| 168 | template <typename DictionaryType> |
| 169 | bool executeDispatchComplex( |
| 170 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 171 | { |
| 172 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 173 | if (!dict) |
| 174 | return false; |
| 175 | |
| 176 | const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[1]); |
| 177 | const ColumnPtr & key_col = key_col_with_type.column; |
| 178 | |
| 179 | if (checkColumn<ColumnTuple>(key_col.get())) |
| 180 | { |
| 181 | const auto & key_columns = assert_cast<const ColumnTuple &>(*key_col).getColumnsCopy(); |
| 182 | const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements(); |
| 183 | |
| 184 | auto out = ColumnUInt8::create(key_col_with_type.column->size()); |
| 185 | dict->has(key_columns, key_types, out->getData()); |
| 186 | block.getByPosition(result).column = std::move(out); |
| 187 | } |
| 188 | else |
| 189 | throw Exception{"Second argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH}; |
| 190 | |
| 191 | return true; |
| 192 | } |
| 193 | |
| 194 | const ExternalDictionariesLoader & dictionaries_loader; |
| 195 | const Context & context; |
| 196 | }; |
| 197 | |
| 198 | |
| 199 | static bool isDictGetFunctionInjective(const ExternalDictionariesLoader & dictionaries_loader, const Block & sample_block) |
| 200 | { |
| 201 | if (sample_block.columns() != 3 && sample_block.columns() != 4) |
| 202 | throw Exception{"Function dictGet... takes 3 or 4 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 203 | |
| 204 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(0).column.get()); |
| 205 | if (!dict_name_col) |
| 206 | throw Exception{"First argument of function dictGet... must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 207 | |
| 208 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(1).column.get()); |
| 209 | if (!attr_name_col) |
| 210 | throw Exception{"Second argument of function dictGet... must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 211 | |
| 212 | return dictionaries_loader.getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>()); |
| 213 | } |
| 214 | |
| 215 | |
| 216 | /** For ColumnVector. Either returns a reference to internal data, |
| 217 | * or convert it to T type, stores the result in backup_storage and returns a reference to it. |
| 218 | */ |
| 219 | template <typename T> |
| 220 | static const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage); |
| 221 | |
| 222 | |
| 223 | class FunctionDictGetString final : public IFunction |
| 224 | { |
| 225 | public: |
| 226 | static constexpr auto name = "dictGetString" ; |
| 227 | |
| 228 | static FunctionPtr create(const Context & context) |
| 229 | { |
| 230 | return std::make_shared<FunctionDictGetString>(context.getExternalDictionariesLoader(), context); |
| 231 | } |
| 232 | |
| 233 | FunctionDictGetString(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) |
| 234 | : dictionaries_loader(dictionaries_loader_) |
| 235 | , context(context_) {} |
| 236 | |
| 237 | String getName() const override { return name; } |
| 238 | |
| 239 | private: |
| 240 | bool isVariadic() const override { return true; } |
| 241 | size_t getNumberOfArguments() const override { return 0; } |
| 242 | |
| 243 | bool useDefaultImplementationForConstants() const final { return true; } |
| 244 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 245 | |
| 246 | bool isInjective(const Block & sample_block) override |
| 247 | { |
| 248 | return isDictGetFunctionInjective(dictionaries_loader, sample_block); |
| 249 | } |
| 250 | |
| 251 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 252 | { |
| 253 | if (arguments.size() != 3 && arguments.size() != 4) |
| 254 | throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " |
| 255 | + toString(arguments.size()) + ", should be 3 or 4." , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 256 | |
| 257 | if (!isString(arguments[0])) |
| 258 | { |
| 259 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 260 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 261 | } |
| 262 | |
| 263 | if (!isString(arguments[1])) |
| 264 | { |
| 265 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 266 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 267 | } |
| 268 | |
| 269 | if (!WhichDataType(arguments[2]).isUInt64() && |
| 270 | !isTuple(arguments[2])) |
| 271 | { |
| 272 | throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() |
| 273 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 274 | } |
| 275 | |
| 276 | /// This is for the case of range dictionaries_loader. |
| 277 | if (arguments.size() == 4 && !arguments[3]->isValueRepresentedByInteger()) |
| 278 | { |
| 279 | throw Exception{"Illegal type " + arguments[3]->getName() + |
| 280 | " of fourth argument of function " + getName() + |
| 281 | " must be convertible to Int64." , ErrorCodes::ILLEGAL_COLUMN}; |
| 282 | } |
| 283 | |
| 284 | return std::make_shared<DataTypeString>(); |
| 285 | } |
| 286 | |
| 287 | bool isDeterministic() const override { return false; } |
| 288 | |
| 289 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 290 | { |
| 291 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 292 | if (!dict_name_col) |
| 293 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 294 | |
| 295 | if (input_rows_count == 0) |
| 296 | { |
| 297 | auto & elem = block.getByPosition(result); |
| 298 | elem.column = elem.type->createColumn(); |
| 299 | return; |
| 300 | } |
| 301 | |
| 302 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 303 | const auto dict_ptr = dict.get(); |
| 304 | |
| 305 | if (!context.hasDictionaryAccessRights(dict_ptr->getFullName())) |
| 306 | { |
| 307 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 308 | + dict->getFullName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 309 | } |
| 310 | |
| 311 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 312 | !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 313 | !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) && |
| 314 | !executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) && |
| 315 | !executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) && |
| 316 | !executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) && |
| 317 | !executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr)) |
| 318 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 319 | } |
| 320 | |
| 321 | template <typename DictionaryType> |
| 322 | bool executeDispatch( |
| 323 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 324 | { |
| 325 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 326 | if (!dict) |
| 327 | return false; |
| 328 | |
| 329 | if (arguments.size() != 3) |
| 330 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 331 | " requires exactly 3 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 332 | |
| 333 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 334 | if (!attr_name_col) |
| 335 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 336 | |
| 337 | String attr_name = attr_name_col->getValue<String>(); |
| 338 | |
| 339 | const auto id_col_untyped = block.getByPosition(arguments[2]).column.get(); |
| 340 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 341 | { |
| 342 | auto out = ColumnString::create(); |
| 343 | dict->getString(attr_name, id_col->getData(), out.get()); |
| 344 | block.getByPosition(result).column = std::move(out); |
| 345 | } |
| 346 | else |
| 347 | throw Exception{"Third argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 348 | |
| 349 | return true; |
| 350 | } |
| 351 | |
| 352 | template <typename DictionaryType> |
| 353 | bool executeDispatchComplex( |
| 354 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 355 | { |
| 356 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 357 | if (!dict) |
| 358 | return false; |
| 359 | |
| 360 | if (arguments.size() != 3) |
| 361 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 362 | " requires exactly 3 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 363 | |
| 364 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 365 | if (!attr_name_col) |
| 366 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 367 | |
| 368 | String attr_name = attr_name_col->getValue<String>(); |
| 369 | |
| 370 | const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); |
| 371 | /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. |
| 372 | ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); |
| 373 | |
| 374 | if (checkColumn<ColumnTuple>(key_col.get())) |
| 375 | { |
| 376 | const auto & key_columns = assert_cast<const ColumnTuple &>(*key_col).getColumnsCopy(); |
| 377 | const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements(); |
| 378 | |
| 379 | auto out = ColumnString::create(); |
| 380 | dict->getString(attr_name, key_columns, key_types, out.get()); |
| 381 | block.getByPosition(result).column = std::move(out); |
| 382 | } |
| 383 | else |
| 384 | throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH}; |
| 385 | |
| 386 | return true; |
| 387 | } |
| 388 | |
| 389 | template <typename DictionaryType> |
| 390 | bool executeDispatchRange( |
| 391 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 392 | { |
| 393 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 394 | if (!dict) |
| 395 | return false; |
| 396 | |
| 397 | if (arguments.size() != 4) |
| 398 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 399 | " requires exactly 4 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 400 | |
| 401 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 402 | if (!attr_name_col) |
| 403 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 404 | |
| 405 | String attr_name = attr_name_col->getValue<String>(); |
| 406 | |
| 407 | const auto & id_col_untyped = block.getByPosition(arguments[2]).column; |
| 408 | const auto & range_col_untyped = block.getByPosition(arguments[3]).column; |
| 409 | |
| 410 | PaddedPODArray<UInt64> id_col_values_storage; |
| 411 | PaddedPODArray<Int64> range_col_values_storage; |
| 412 | const auto & id_col_values = getColumnDataAsPaddedPODArray(*id_col_untyped, id_col_values_storage); |
| 413 | const auto & range_col_values = getColumnDataAsPaddedPODArray(*range_col_untyped, range_col_values_storage); |
| 414 | |
| 415 | auto out = ColumnString::create(); |
| 416 | dict->getString(attr_name, id_col_values, range_col_values, out.get()); |
| 417 | block.getByPosition(result).column = std::move(out); |
| 418 | |
| 419 | return true; |
| 420 | } |
| 421 | |
| 422 | const ExternalDictionariesLoader & dictionaries_loader; |
| 423 | const Context & context; |
| 424 | }; |
| 425 | |
| 426 | |
| 427 | class FunctionDictGetStringOrDefault final : public IFunction |
| 428 | { |
| 429 | public: |
| 430 | static constexpr auto name = "dictGetStringOrDefault" ; |
| 431 | |
| 432 | static FunctionPtr create(const Context & context) |
| 433 | { |
| 434 | return std::make_shared<FunctionDictGetStringOrDefault>(context.getExternalDictionariesLoader(), context); |
| 435 | } |
| 436 | |
| 437 | FunctionDictGetStringOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) |
| 438 | : dictionaries_loader(dictionaries_loader_) |
| 439 | , context(context_) {} |
| 440 | |
| 441 | String getName() const override { return name; } |
| 442 | |
| 443 | private: |
| 444 | size_t getNumberOfArguments() const override { return 4; } |
| 445 | |
| 446 | bool useDefaultImplementationForConstants() const final { return true; } |
| 447 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 448 | |
| 449 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 450 | { |
| 451 | if (!isString(arguments[0])) |
| 452 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + |
| 453 | ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 454 | |
| 455 | if (!isString(arguments[1])) |
| 456 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + |
| 457 | ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 458 | |
| 459 | if (!WhichDataType(arguments[2]).isUInt64() && |
| 460 | !isTuple(arguments[2])) |
| 461 | { |
| 462 | throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() |
| 463 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 464 | } |
| 465 | |
| 466 | if (!isString(arguments[3])) |
| 467 | throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() + |
| 468 | ", must be String." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 469 | |
| 470 | return std::make_shared<DataTypeString>(); |
| 471 | } |
| 472 | |
| 473 | bool isDeterministic() const override { return false; } |
| 474 | |
| 475 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 476 | { |
| 477 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 478 | if (!dict_name_col) |
| 479 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 480 | |
| 481 | if (input_rows_count == 0) |
| 482 | { |
| 483 | auto & elem = block.getByPosition(result); |
| 484 | elem.column = elem.type->createColumn(); |
| 485 | return; |
| 486 | } |
| 487 | |
| 488 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 489 | const auto dict_ptr = dict.get(); |
| 490 | |
| 491 | if (!context.hasDictionaryAccessRights(dict_ptr->getFullName())) |
| 492 | { |
| 493 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 494 | + dict->getFullName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 495 | } |
| 496 | |
| 497 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 498 | !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 499 | !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) && |
| 500 | !executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) && |
| 501 | !executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) && |
| 502 | !executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr)) |
| 503 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 504 | } |
| 505 | |
| 506 | template <typename DictionaryType> |
| 507 | bool executeDispatch( |
| 508 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 509 | { |
| 510 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 511 | if (!dict) |
| 512 | return false; |
| 513 | |
| 514 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 515 | if (!attr_name_col) |
| 516 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 517 | |
| 518 | String attr_name = attr_name_col->getValue<String>(); |
| 519 | |
| 520 | const auto id_col_untyped = block.getByPosition(arguments[2]).column.get(); |
| 521 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 522 | executeDispatch(block, arguments, result, dict, attr_name, id_col); |
| 523 | else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped)) |
| 524 | executeDispatch(block, arguments, result, dict, attr_name, id_col_const); |
| 525 | else |
| 526 | throw Exception{"Third argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 527 | |
| 528 | return true; |
| 529 | } |
| 530 | |
| 531 | template <typename DictionaryType> |
| 532 | void executeDispatch( |
| 533 | Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary, |
| 534 | const std::string & attr_name, const ColumnUInt64 * id_col) |
| 535 | { |
| 536 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 537 | |
| 538 | if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped)) |
| 539 | { |
| 540 | /// vector ids, vector defaults |
| 541 | auto out = ColumnString::create(); |
| 542 | const auto & ids = id_col->getData(); |
| 543 | dictionary->getString(attr_name, ids, default_col, out.get()); |
| 544 | block.getByPosition(result).column = std::move(out); |
| 545 | } |
| 546 | else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped)) |
| 547 | { |
| 548 | /// vector ids, const defaults |
| 549 | auto out = ColumnString::create(); |
| 550 | const auto & ids = id_col->getData(); |
| 551 | String def = default_col_const->getValue<String>(); |
| 552 | dictionary->getString(attr_name, ids, def, out.get()); |
| 553 | block.getByPosition(result).column = std::move(out); |
| 554 | } |
| 555 | else |
| 556 | throw Exception{"Fourth argument of function " + getName() + " must be String" , ErrorCodes::ILLEGAL_COLUMN}; |
| 557 | } |
| 558 | |
| 559 | template <typename DictionaryType> |
| 560 | void executeDispatch( |
| 561 | Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary, |
| 562 | const std::string & attr_name, const ColumnConst * id_col) |
| 563 | { |
| 564 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 565 | |
| 566 | if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped)) |
| 567 | { |
| 568 | /// const ids, vector defaults |
| 569 | const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>()); |
| 570 | PaddedPODArray<UInt8> flags(1); |
| 571 | dictionary->has(ids, flags); |
| 572 | if (flags.front()) |
| 573 | { |
| 574 | auto out = ColumnString::create(); |
| 575 | dictionary->getString(attr_name, ids, String(), out.get()); |
| 576 | block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString()); |
| 577 | } |
| 578 | else |
| 579 | block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column |
| 580 | } |
| 581 | else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped)) |
| 582 | { |
| 583 | /// const ids, const defaults |
| 584 | const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>()); |
| 585 | auto out = ColumnString::create(); |
| 586 | String def = default_col_const->getValue<String>(); |
| 587 | dictionary->getString(attr_name, ids, def, out.get()); |
| 588 | block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString()); |
| 589 | } |
| 590 | else |
| 591 | throw Exception{"Fourth argument of function " + getName() + " must be String" , ErrorCodes::ILLEGAL_COLUMN}; |
| 592 | } |
| 593 | |
| 594 | template <typename DictionaryType> |
| 595 | bool executeDispatchComplex( |
| 596 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 597 | { |
| 598 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 599 | if (!dict) |
| 600 | return false; |
| 601 | |
| 602 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 603 | if (!attr_name_col) |
| 604 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 605 | |
| 606 | String attr_name = attr_name_col->getValue<String>(); |
| 607 | |
| 608 | const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); |
| 609 | /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. |
| 610 | ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); |
| 611 | |
| 612 | const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_col).getColumnsCopy(); |
| 613 | const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements(); |
| 614 | |
| 615 | auto out = ColumnString::create(); |
| 616 | |
| 617 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 618 | if (const auto default_col = checkAndGetColumn<ColumnString>(default_col_untyped)) |
| 619 | { |
| 620 | dict->getString(attr_name, key_columns, key_types, default_col, out.get()); |
| 621 | } |
| 622 | else if (const auto default_col_const = checkAndGetColumnConstStringOrFixedString(default_col_untyped)) |
| 623 | { |
| 624 | String def = default_col_const->getValue<String>(); |
| 625 | dict->getString(attr_name, key_columns, key_types, def, out.get()); |
| 626 | } |
| 627 | else |
| 628 | throw Exception{"Fourth argument of function " + getName() + " must be String" , ErrorCodes::ILLEGAL_COLUMN}; |
| 629 | |
| 630 | block.getByPosition(result).column = std::move(out); |
| 631 | return true; |
| 632 | } |
| 633 | |
| 634 | const ExternalDictionariesLoader & dictionaries_loader; |
| 635 | const Context & context; |
| 636 | }; |
| 637 | |
| 638 | |
| 639 | template <typename DataType> struct DictGetTraits; |
| 640 | #define DECLARE_DICT_GET_TRAITS(TYPE, DATA_TYPE) \ |
| 641 | template <> struct DictGetTraits<DATA_TYPE>\ |
| 642 | {\ |
| 643 | template <typename DictionaryType>\ |
| 644 | static void get(\ |
| 645 | const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,\ |
| 646 | PaddedPODArray<TYPE> & out)\ |
| 647 | {\ |
| 648 | dict->get##TYPE(name, ids, out);\ |
| 649 | }\ |
| 650 | template <typename DictionaryType>\ |
| 651 | static void get(\ |
| 652 | const DictionaryType * dict, const std::string & name, const Columns & key_columns,\ |
| 653 | const DataTypes & key_types, PaddedPODArray<TYPE> & out)\ |
| 654 | {\ |
| 655 | dict->get##TYPE(name, key_columns, key_types, out);\ |
| 656 | }\ |
| 657 | template <typename DictionaryType>\ |
| 658 | static void get(\ |
| 659 | const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,\ |
| 660 | const PaddedPODArray<Int64> & dates, PaddedPODArray<TYPE> & out)\ |
| 661 | {\ |
| 662 | dict->get##TYPE(name, ids, dates, out);\ |
| 663 | }\ |
| 664 | template <typename DictionaryType, typename DefaultsType>\ |
| 665 | static void getOrDefault(\ |
| 666 | const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,\ |
| 667 | const DefaultsType & def, PaddedPODArray<TYPE> & out)\ |
| 668 | {\ |
| 669 | dict->get##TYPE(name, ids, def, out);\ |
| 670 | }\ |
| 671 | template <typename DictionaryType, typename DefaultsType>\ |
| 672 | static void getOrDefault(\ |
| 673 | const DictionaryType * dict, const std::string & name, const Columns & key_columns,\ |
| 674 | const DataTypes & key_types, const DefaultsType & def, PaddedPODArray<TYPE> & out)\ |
| 675 | {\ |
| 676 | dict->get##TYPE(name, key_columns, key_types, def, out);\ |
| 677 | }\ |
| 678 | }; |
| 679 | DECLARE_DICT_GET_TRAITS(UInt8, DataTypeUInt8) |
| 680 | DECLARE_DICT_GET_TRAITS(UInt16, DataTypeUInt16) |
| 681 | DECLARE_DICT_GET_TRAITS(UInt32, DataTypeUInt32) |
| 682 | DECLARE_DICT_GET_TRAITS(UInt64, DataTypeUInt64) |
| 683 | DECLARE_DICT_GET_TRAITS(Int8, DataTypeInt8) |
| 684 | DECLARE_DICT_GET_TRAITS(Int16, DataTypeInt16) |
| 685 | DECLARE_DICT_GET_TRAITS(Int32, DataTypeInt32) |
| 686 | DECLARE_DICT_GET_TRAITS(Int64, DataTypeInt64) |
| 687 | DECLARE_DICT_GET_TRAITS(Float32, DataTypeFloat32) |
| 688 | DECLARE_DICT_GET_TRAITS(Float64, DataTypeFloat64) |
| 689 | DECLARE_DICT_GET_TRAITS(UInt16, DataTypeDate) |
| 690 | DECLARE_DICT_GET_TRAITS(UInt32, DataTypeDateTime) |
| 691 | DECLARE_DICT_GET_TRAITS(UInt128, DataTypeUUID) |
| 692 | #undef DECLARE_DICT_GET_TRAITS |
| 693 | |
| 694 | template <typename T> struct DictGetTraits<DataTypeDecimal<T>> |
| 695 | { |
| 696 | static constexpr bool is_dec32 = std::is_same_v<T, Decimal32>; |
| 697 | static constexpr bool is_dec64 = std::is_same_v<T, Decimal64>; |
| 698 | static constexpr bool is_dec128 = std::is_same_v<T, Decimal128>; |
| 699 | |
| 700 | template <typename DictionaryType> |
| 701 | static void get(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids, |
| 702 | DecimalPaddedPODArray<T> & out) |
| 703 | { |
| 704 | if constexpr (is_dec32) dict->getDecimal32(name, ids, out); |
| 705 | if constexpr (is_dec64) dict->getDecimal64(name, ids, out); |
| 706 | if constexpr (is_dec128) dict->getDecimal128(name, ids, out); |
| 707 | } |
| 708 | |
| 709 | template <typename DictionaryType> |
| 710 | static void get(const DictionaryType * dict, const std::string & name, const Columns & key_columns, const DataTypes & key_types, |
| 711 | DecimalPaddedPODArray<T> & out) |
| 712 | { |
| 713 | if constexpr (is_dec32) dict->getDecimal32(name, key_columns, key_types, out); |
| 714 | if constexpr (is_dec64) dict->getDecimal64(name, key_columns, key_types, out); |
| 715 | if constexpr (is_dec128) dict->getDecimal128(name, key_columns, key_types, out); |
| 716 | } |
| 717 | |
| 718 | template <typename DictionaryType> |
| 719 | static void get(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids, |
| 720 | const PaddedPODArray<Int64> & dates, DecimalPaddedPODArray<T> & out) |
| 721 | { |
| 722 | if constexpr (is_dec32) dict->getDecimal32(name, ids, dates, out); |
| 723 | if constexpr (is_dec64) dict->getDecimal64(name, ids, dates, out); |
| 724 | if constexpr (is_dec128) dict->getDecimal128(name, ids, dates, out); |
| 725 | } |
| 726 | |
| 727 | template <typename DictionaryType, typename DefaultsType> |
| 728 | static void getOrDefault(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids, |
| 729 | const DefaultsType & def, DecimalPaddedPODArray<T> & out) |
| 730 | { |
| 731 | if constexpr (is_dec32) dict->getDecimal32(name, ids, def, out); |
| 732 | if constexpr (is_dec64) dict->getDecimal64(name, ids, def, out); |
| 733 | if constexpr (is_dec128) dict->getDecimal128(name, ids, def, out); |
| 734 | } |
| 735 | |
| 736 | template <typename DictionaryType, typename DefaultsType> |
| 737 | static void getOrDefault(const DictionaryType * dict, const std::string & name, const Columns & key_columns, |
| 738 | const DataTypes & key_types, const DefaultsType & def, DecimalPaddedPODArray<T> & out) |
| 739 | { |
| 740 | if constexpr (is_dec32) dict->getDecimal32(name, key_columns, key_types, def, out); |
| 741 | if constexpr (is_dec64) dict->getDecimal64(name, key_columns, key_types, def, out); |
| 742 | if constexpr (is_dec128) dict->getDecimal128(name, key_columns, key_types, def, out); |
| 743 | } |
| 744 | }; |
| 745 | |
| 746 | |
| 747 | template <typename DataType, typename Name> |
| 748 | class FunctionDictGet final : public IFunction |
| 749 | { |
| 750 | using Type = typename DataType::FieldType; |
| 751 | using ColVec = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>; |
| 752 | |
| 753 | public: |
| 754 | static constexpr auto name = Name::name; |
| 755 | |
| 756 | static FunctionPtr create(const Context & context, UInt32 dec_scale = 0) |
| 757 | { |
| 758 | return std::make_shared<FunctionDictGet>(context.getExternalDictionariesLoader(), context, dec_scale); |
| 759 | } |
| 760 | |
| 761 | FunctionDictGet(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_, UInt32 dec_scale = 0) |
| 762 | : dictionaries_loader(dictionaries_loader_) |
| 763 | , context(context_) |
| 764 | , decimal_scale(dec_scale) |
| 765 | {} |
| 766 | |
| 767 | String getName() const override { return name; } |
| 768 | |
| 769 | private: |
| 770 | bool isVariadic() const override { return true; } |
| 771 | size_t getNumberOfArguments() const override { return 0; } |
| 772 | |
| 773 | bool useDefaultImplementationForConstants() const final { return true; } |
| 774 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 775 | |
| 776 | bool isInjective(const Block & sample_block) override |
| 777 | { |
| 778 | return isDictGetFunctionInjective(dictionaries_loader, sample_block); |
| 779 | } |
| 780 | |
| 781 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 782 | { |
| 783 | if (arguments.size() != 3 && arguments.size() != 4) |
| 784 | throw Exception{"Function " + getName() + " takes 3 or 4 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 785 | |
| 786 | if (!isString(arguments[0])) |
| 787 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 788 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 789 | |
| 790 | if (!isString(arguments[1])) |
| 791 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 792 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 793 | |
| 794 | if (!WhichDataType(arguments[2]).isUInt64() && |
| 795 | !isTuple(arguments[2])) |
| 796 | throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() |
| 797 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 798 | |
| 799 | if (arguments.size() == 4) |
| 800 | { |
| 801 | const auto range_argument = arguments[3].get(); |
| 802 | if (!(range_argument->isValueRepresentedByInteger() && |
| 803 | range_argument->getSizeOfValueInMemory() <= sizeof(Int64))) |
| 804 | throw Exception{"Illegal type " + range_argument->getName() + " of fourth argument of function " + getName() |
| 805 | + ", must be convertible to " + TypeName<Int64>::get() + "." , |
| 806 | ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 807 | } |
| 808 | |
| 809 | if constexpr (IsDataTypeDecimal<DataType>) |
| 810 | return std::make_shared<DataType>(DataType::maxPrecision(), decimal_scale); |
| 811 | else |
| 812 | return std::make_shared<DataType>(); |
| 813 | } |
| 814 | |
| 815 | bool isDeterministic() const override { return false; } |
| 816 | |
| 817 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 818 | { |
| 819 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 820 | if (!dict_name_col) |
| 821 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 822 | |
| 823 | if (input_rows_count == 0) |
| 824 | { |
| 825 | auto & elem = block.getByPosition(result); |
| 826 | elem.column = elem.type->createColumn(); |
| 827 | return; |
| 828 | } |
| 829 | |
| 830 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 831 | const auto dict_ptr = dict.get(); |
| 832 | |
| 833 | if (!context.hasDictionaryAccessRights(dict_ptr->getName())) |
| 834 | { |
| 835 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 836 | + dict->getName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 837 | } |
| 838 | |
| 839 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 840 | !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 841 | !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) && |
| 842 | !executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) && |
| 843 | !executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) && |
| 844 | !executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) && |
| 845 | !executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr)) |
| 846 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 847 | } |
| 848 | |
| 849 | template <typename DictionaryType> |
| 850 | bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, |
| 851 | const IDictionaryBase * dictionary) |
| 852 | { |
| 853 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 854 | if (!dict) |
| 855 | return false; |
| 856 | |
| 857 | if (arguments.size() != 3) |
| 858 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 859 | " requires exactly 3 arguments." , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 860 | |
| 861 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 862 | if (!attr_name_col) |
| 863 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 864 | |
| 865 | String attr_name = attr_name_col->getValue<String>(); |
| 866 | |
| 867 | const auto id_col_untyped = block.getByPosition(arguments[2]).column.get(); |
| 868 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 869 | { |
| 870 | typename ColVec::MutablePtr out; |
| 871 | if constexpr (IsDataTypeDecimal<DataType>) |
| 872 | out = ColVec::create(id_col->size(), decimal_scale); |
| 873 | else |
| 874 | out = ColVec::create(id_col->size()); |
| 875 | const auto & ids = id_col->getData(); |
| 876 | auto & data = out->getData(); |
| 877 | DictGetTraits<DataType>::get(dict, attr_name, ids, data); |
| 878 | block.getByPosition(result).column = std::move(out); |
| 879 | } |
| 880 | else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped)) |
| 881 | { |
| 882 | const PaddedPODArray<UInt64> ids(1, id_col_const->getValue<UInt64>()); |
| 883 | |
| 884 | if constexpr (IsDataTypeDecimal<DataType>) |
| 885 | { |
| 886 | DecimalPaddedPODArray<Type> data(1, decimal_scale); |
| 887 | DictGetTraits<DataType>::get(dict, attr_name, ids, data); |
| 888 | block.getByPosition(result).column = |
| 889 | DataType(DataType::maxPrecision(), decimal_scale).createColumnConst( |
| 890 | id_col_const->size(), toField(data.front(), decimal_scale)); |
| 891 | } |
| 892 | else |
| 893 | { |
| 894 | PaddedPODArray<Type> data(1); |
| 895 | DictGetTraits<DataType>::get(dict, attr_name, ids, data); |
| 896 | block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front())); |
| 897 | } |
| 898 | } |
| 899 | else |
| 900 | throw Exception{"Third argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 901 | |
| 902 | return true; |
| 903 | } |
| 904 | |
| 905 | template <typename DictionaryType> |
| 906 | bool executeDispatchComplex( |
| 907 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 908 | { |
| 909 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 910 | if (!dict) |
| 911 | return false; |
| 912 | |
| 913 | if (arguments.size() != 3) |
| 914 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 915 | " requires exactly 3 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 916 | |
| 917 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 918 | if (!attr_name_col) |
| 919 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 920 | |
| 921 | String attr_name = attr_name_col->getValue<String>(); |
| 922 | |
| 923 | const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); |
| 924 | |
| 925 | /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. |
| 926 | ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); |
| 927 | |
| 928 | if (checkColumn<ColumnTuple>(key_col.get())) |
| 929 | { |
| 930 | const auto & key_columns = assert_cast<const ColumnTuple &>(*key_col).getColumnsCopy(); |
| 931 | const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements(); |
| 932 | |
| 933 | typename ColVec::MutablePtr out; |
| 934 | if constexpr (IsDataTypeDecimal<DataType>) |
| 935 | out = ColVec::create(key_columns.front()->size(), decimal_scale); |
| 936 | else |
| 937 | out = ColVec::create(key_columns.front()->size()); |
| 938 | auto & data = out->getData(); |
| 939 | DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data); |
| 940 | block.getByPosition(result).column = std::move(out); |
| 941 | } |
| 942 | else |
| 943 | throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH}; |
| 944 | |
| 945 | return true; |
| 946 | } |
| 947 | |
| 948 | template <typename DictionaryType> |
| 949 | bool executeDispatchRange( |
| 950 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 951 | { |
| 952 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 953 | if (!dict) |
| 954 | return false; |
| 955 | |
| 956 | if (arguments.size() != 4) |
| 957 | throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() + |
| 958 | " requires exactly 4 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 959 | |
| 960 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 961 | if (!attr_name_col) |
| 962 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 963 | |
| 964 | String attr_name = attr_name_col->getValue<String>(); |
| 965 | |
| 966 | const auto & id_col_untyped = block.getByPosition(arguments[2]).column; |
| 967 | const auto & range_col_untyped = block.getByPosition(arguments[3]).column; |
| 968 | |
| 969 | PaddedPODArray<UInt64> id_col_values_storage; |
| 970 | PaddedPODArray<Int64> range_col_values_storage; |
| 971 | const auto & id_col_values = getColumnDataAsPaddedPODArray(*id_col_untyped, id_col_values_storage); |
| 972 | const auto & range_col_values = getColumnDataAsPaddedPODArray(*range_col_untyped, range_col_values_storage); |
| 973 | |
| 974 | typename ColVec::MutablePtr out; |
| 975 | if constexpr (IsDataTypeDecimal<DataType>) |
| 976 | out = ColVec::create(id_col_untyped->size(), decimal_scale); |
| 977 | else |
| 978 | out = ColVec::create(id_col_untyped->size()); |
| 979 | auto & data = out->getData(); |
| 980 | DictGetTraits<DataType>::get(dict, attr_name, id_col_values, range_col_values, data); |
| 981 | block.getByPosition(result).column = std::move(out); |
| 982 | |
| 983 | return true; |
| 984 | } |
| 985 | |
| 986 | const ExternalDictionariesLoader & dictionaries_loader; |
| 987 | const Context & context; |
| 988 | UInt32 decimal_scale; |
| 989 | }; |
| 990 | |
| 991 | struct NameDictGetUInt8 { static constexpr auto name = "dictGetUInt8" ; }; |
| 992 | struct NameDictGetUInt16 { static constexpr auto name = "dictGetUInt16" ; }; |
| 993 | struct NameDictGetUInt32 { static constexpr auto name = "dictGetUInt32" ; }; |
| 994 | struct NameDictGetUInt64 { static constexpr auto name = "dictGetUInt64" ; }; |
| 995 | struct NameDictGetInt8 { static constexpr auto name = "dictGetInt8" ; }; |
| 996 | struct NameDictGetInt16 { static constexpr auto name = "dictGetInt16" ; }; |
| 997 | struct NameDictGetInt32 { static constexpr auto name = "dictGetInt32" ; }; |
| 998 | struct NameDictGetInt64 { static constexpr auto name = "dictGetInt64" ; }; |
| 999 | struct NameDictGetFloat32 { static constexpr auto name = "dictGetFloat32" ; }; |
| 1000 | struct NameDictGetFloat64 { static constexpr auto name = "dictGetFloat64" ; }; |
| 1001 | struct NameDictGetDate { static constexpr auto name = "dictGetDate" ; }; |
| 1002 | struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime" ; }; |
| 1003 | struct NameDictGetUUID { static constexpr auto name = "dictGetUUID" ; }; |
| 1004 | struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32" ; }; |
| 1005 | struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64" ; }; |
| 1006 | struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128" ; }; |
| 1007 | |
| 1008 | using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>; |
| 1009 | using FunctionDictGetUInt16 = FunctionDictGet<DataTypeUInt16, NameDictGetUInt16>; |
| 1010 | using FunctionDictGetUInt32 = FunctionDictGet<DataTypeUInt32, NameDictGetUInt32>; |
| 1011 | using FunctionDictGetUInt64 = FunctionDictGet<DataTypeUInt64, NameDictGetUInt64>; |
| 1012 | using FunctionDictGetInt8 = FunctionDictGet<DataTypeInt8, NameDictGetInt8>; |
| 1013 | using FunctionDictGetInt16 = FunctionDictGet<DataTypeInt16, NameDictGetInt16>; |
| 1014 | using FunctionDictGetInt32 = FunctionDictGet<DataTypeInt32, NameDictGetInt32>; |
| 1015 | using FunctionDictGetInt64 = FunctionDictGet<DataTypeInt64, NameDictGetInt64>; |
| 1016 | using FunctionDictGetFloat32 = FunctionDictGet<DataTypeFloat32, NameDictGetFloat32>; |
| 1017 | using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64, NameDictGetFloat64>; |
| 1018 | using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>; |
| 1019 | using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>; |
| 1020 | using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>; |
| 1021 | using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>; |
| 1022 | using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>; |
| 1023 | using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>; |
| 1024 | |
| 1025 | |
| 1026 | template <typename DataType, typename Name> |
| 1027 | class FunctionDictGetOrDefault final : public IFunction |
| 1028 | { |
| 1029 | using Type = typename DataType::FieldType; |
| 1030 | using ColVec = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>; |
| 1031 | |
| 1032 | public: |
| 1033 | static constexpr auto name = Name::name; |
| 1034 | |
| 1035 | static FunctionPtr create(const Context & context, UInt32 dec_scale = 0) |
| 1036 | { |
| 1037 | return std::make_shared<FunctionDictGetOrDefault>(context.getExternalDictionariesLoader(), context, dec_scale); |
| 1038 | } |
| 1039 | |
| 1040 | FunctionDictGetOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_, UInt32 dec_scale = 0) |
| 1041 | : dictionaries_loader(dictionaries_loader_) |
| 1042 | , context(context_) |
| 1043 | , decimal_scale(dec_scale) |
| 1044 | {} |
| 1045 | |
| 1046 | String getName() const override { return name; } |
| 1047 | |
| 1048 | private: |
| 1049 | size_t getNumberOfArguments() const override { return 4; } |
| 1050 | |
| 1051 | bool useDefaultImplementationForConstants() const final { return true; } |
| 1052 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 1053 | |
| 1054 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 1055 | { |
| 1056 | if (!isString(arguments[0])) |
| 1057 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 1058 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1059 | |
| 1060 | if (!isString(arguments[1])) |
| 1061 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 1062 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1063 | |
| 1064 | if (!WhichDataType(arguments[2]).isUInt64() && |
| 1065 | !isTuple(arguments[2])) |
| 1066 | throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() |
| 1067 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1068 | |
| 1069 | if (!checkAndGetDataType<DataType>(arguments[3].get())) |
| 1070 | throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() |
| 1071 | + ", must be " + TypeName<Type>::get() + "." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1072 | |
| 1073 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1074 | return std::make_shared<DataType>(DataType::maxPrecision(), decimal_scale); |
| 1075 | else |
| 1076 | return std::make_shared<DataType>(); |
| 1077 | } |
| 1078 | |
| 1079 | bool isDeterministic() const override { return false; } |
| 1080 | |
| 1081 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 1082 | { |
| 1083 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 1084 | if (!dict_name_col) |
| 1085 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1086 | |
| 1087 | if (input_rows_count == 0) |
| 1088 | { |
| 1089 | auto & elem = block.getByPosition(result); |
| 1090 | elem.column = elem.type->createColumn(); |
| 1091 | return; |
| 1092 | } |
| 1093 | |
| 1094 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 1095 | const auto dict_ptr = dict.get(); |
| 1096 | |
| 1097 | if (!context.hasDictionaryAccessRights(dict_ptr->getName())) |
| 1098 | { |
| 1099 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 1100 | + dict->getName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 1101 | } |
| 1102 | |
| 1103 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 1104 | !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 1105 | !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) && |
| 1106 | !executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) && |
| 1107 | !executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) && |
| 1108 | !executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr)) |
| 1109 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 1110 | } |
| 1111 | |
| 1112 | template <typename DictionaryType> |
| 1113 | bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, |
| 1114 | const IDictionaryBase * dictionary) |
| 1115 | { |
| 1116 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 1117 | if (!dict) |
| 1118 | return false; |
| 1119 | |
| 1120 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 1121 | if (!attr_name_col) |
| 1122 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1123 | |
| 1124 | String attr_name = attr_name_col->getValue<String>(); |
| 1125 | |
| 1126 | const auto id_col_untyped = block.getByPosition(arguments[2]).column.get(); |
| 1127 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 1128 | executeDispatch(block, arguments, result, dict, attr_name, id_col); |
| 1129 | else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped)) |
| 1130 | executeDispatch(block, arguments, result, dict, attr_name, id_col_const); |
| 1131 | else |
| 1132 | throw Exception{"Third argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1133 | |
| 1134 | return true; |
| 1135 | } |
| 1136 | |
| 1137 | template <typename DictionaryType> |
| 1138 | void executeDispatch( |
| 1139 | Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary, |
| 1140 | const std::string & attr_name, const ColumnUInt64 * id_col) |
| 1141 | { |
| 1142 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 1143 | |
| 1144 | if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped)) |
| 1145 | { |
| 1146 | /// vector ids, vector defaults |
| 1147 | typename ColVec::MutablePtr out; |
| 1148 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1149 | out = ColVec::create(id_col->size(), decimal_scale); |
| 1150 | else |
| 1151 | out = ColVec::create(id_col->size()); |
| 1152 | const auto & ids = id_col->getData(); |
| 1153 | auto & data = out->getData(); |
| 1154 | const auto & defs = default_col->getData(); |
| 1155 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data); |
| 1156 | block.getByPosition(result).column = std::move(out); |
| 1157 | } |
| 1158 | else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped)) |
| 1159 | { |
| 1160 | /// vector ids, const defaults |
| 1161 | typename ColVec::MutablePtr out; |
| 1162 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1163 | out = ColVec::create(id_col->size(), decimal_scale); |
| 1164 | else |
| 1165 | out = ColVec::create(id_col->size()); |
| 1166 | const auto & ids = id_col->getData(); |
| 1167 | auto & data = out->getData(); |
| 1168 | const auto def = default_col_const->template getValue<Type>(); |
| 1169 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data); |
| 1170 | block.getByPosition(result).column = std::move(out); |
| 1171 | } |
| 1172 | else |
| 1173 | throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1174 | } |
| 1175 | |
| 1176 | template <typename DictionaryType> |
| 1177 | void executeDispatch( |
| 1178 | Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * dictionary, |
| 1179 | const std::string & attr_name, const ColumnConst * id_col) |
| 1180 | { |
| 1181 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 1182 | |
| 1183 | if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped)) |
| 1184 | { |
| 1185 | /// const ids, vector defaults |
| 1186 | const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>()); |
| 1187 | PaddedPODArray<UInt8> flags(1); |
| 1188 | dictionary->has(ids, flags); |
| 1189 | if (flags.front()) |
| 1190 | { |
| 1191 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1192 | { |
| 1193 | DecimalPaddedPODArray<Type> data(1, decimal_scale); |
| 1194 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data); |
| 1195 | block.getByPosition(result).column = |
| 1196 | DataType(DataType::maxPrecision(), decimal_scale).createColumnConst( |
| 1197 | id_col->size(), toField(data.front(), decimal_scale)); |
| 1198 | } |
| 1199 | else |
| 1200 | { |
| 1201 | PaddedPODArray<Type> data(1); |
| 1202 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data); |
| 1203 | block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front())); |
| 1204 | } |
| 1205 | } |
| 1206 | else |
| 1207 | block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column |
| 1208 | } |
| 1209 | else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped)) |
| 1210 | { |
| 1211 | /// const ids, const defaults |
| 1212 | const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>()); |
| 1213 | |
| 1214 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1215 | { |
| 1216 | DecimalPaddedPODArray<Type> data(1, decimal_scale); |
| 1217 | const auto & def = default_col_const->template getValue<Type>(); |
| 1218 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data); |
| 1219 | block.getByPosition(result).column = |
| 1220 | DataType(DataType::maxPrecision(), decimal_scale).createColumnConst( |
| 1221 | id_col->size(), toField(data.front(), decimal_scale)); |
| 1222 | } |
| 1223 | else |
| 1224 | { |
| 1225 | PaddedPODArray<Type> data(1); |
| 1226 | const auto & def = default_col_const->template getValue<Type>(); |
| 1227 | DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data); |
| 1228 | block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front())); |
| 1229 | } |
| 1230 | } |
| 1231 | else |
| 1232 | throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1233 | } |
| 1234 | |
| 1235 | template <typename DictionaryType> |
| 1236 | bool executeDispatchComplex( |
| 1237 | Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * dictionary) |
| 1238 | { |
| 1239 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 1240 | if (!dict) |
| 1241 | return false; |
| 1242 | |
| 1243 | const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get()); |
| 1244 | if (!attr_name_col) |
| 1245 | throw Exception{"Second argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1246 | |
| 1247 | String attr_name = attr_name_col->getValue<String>(); |
| 1248 | |
| 1249 | const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); |
| 1250 | |
| 1251 | /// Functions in external dictionaries_loader only support full-value (not constant) columns with keys. |
| 1252 | ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); |
| 1253 | |
| 1254 | const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_col).getColumnsCopy(); |
| 1255 | const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements(); |
| 1256 | |
| 1257 | /// @todo detect when all key columns are constant |
| 1258 | const auto rows = key_col->size(); |
| 1259 | typename ColVec::MutablePtr out; |
| 1260 | if constexpr (IsDataTypeDecimal<DataType>) |
| 1261 | out = ColVec::create(rows, decimal_scale); |
| 1262 | else |
| 1263 | out = ColVec::create(rows); |
| 1264 | auto & data = out->getData(); |
| 1265 | |
| 1266 | const auto default_col_untyped = block.getByPosition(arguments[3]).column.get(); |
| 1267 | if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped)) |
| 1268 | { |
| 1269 | /// const defaults |
| 1270 | const auto & defs = default_col->getData(); |
| 1271 | |
| 1272 | DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data); |
| 1273 | } |
| 1274 | else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped)) |
| 1275 | { |
| 1276 | const auto def = default_col_const->template getValue<Type>(); |
| 1277 | |
| 1278 | DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data); |
| 1279 | } |
| 1280 | else |
| 1281 | throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1282 | |
| 1283 | block.getByPosition(result).column = std::move(out); |
| 1284 | return true; |
| 1285 | } |
| 1286 | |
| 1287 | const ExternalDictionariesLoader & dictionaries_loader; |
| 1288 | const Context & context; |
| 1289 | UInt32 decimal_scale; |
| 1290 | }; |
| 1291 | |
| 1292 | struct NameDictGetUInt8OrDefault { static constexpr auto name = "dictGetUInt8OrDefault" ; }; |
| 1293 | struct NameDictGetUInt16OrDefault { static constexpr auto name = "dictGetUInt16OrDefault" ; }; |
| 1294 | struct NameDictGetUInt32OrDefault { static constexpr auto name = "dictGetUInt32OrDefault" ; }; |
| 1295 | struct NameDictGetUInt64OrDefault { static constexpr auto name = "dictGetUInt64OrDefault" ; }; |
| 1296 | struct NameDictGetInt8OrDefault { static constexpr auto name = "dictGetInt8OrDefault" ; }; |
| 1297 | struct NameDictGetInt16OrDefault { static constexpr auto name = "dictGetInt16OrDefault" ; }; |
| 1298 | struct NameDictGetInt32OrDefault { static constexpr auto name = "dictGetInt32OrDefault" ; }; |
| 1299 | struct NameDictGetInt64OrDefault { static constexpr auto name = "dictGetInt64OrDefault" ; }; |
| 1300 | struct NameDictGetFloat32OrDefault { static constexpr auto name = "dictGetFloat32OrDefault" ; }; |
| 1301 | struct NameDictGetFloat64OrDefault { static constexpr auto name = "dictGetFloat64OrDefault" ; }; |
| 1302 | struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault" ; }; |
| 1303 | struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault" ; }; |
| 1304 | struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault" ; }; |
| 1305 | struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault" ; }; |
| 1306 | struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault" ; }; |
| 1307 | struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault" ; }; |
| 1308 | |
| 1309 | using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>; |
| 1310 | using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16, NameDictGetUInt16OrDefault>; |
| 1311 | using FunctionDictGetUInt32OrDefault = FunctionDictGetOrDefault<DataTypeUInt32, NameDictGetUInt32OrDefault>; |
| 1312 | using FunctionDictGetUInt64OrDefault = FunctionDictGetOrDefault<DataTypeUInt64, NameDictGetUInt64OrDefault>; |
| 1313 | using FunctionDictGetInt8OrDefault = FunctionDictGetOrDefault<DataTypeInt8, NameDictGetInt8OrDefault>; |
| 1314 | using FunctionDictGetInt16OrDefault = FunctionDictGetOrDefault<DataTypeInt16, NameDictGetInt16OrDefault>; |
| 1315 | using FunctionDictGetInt32OrDefault = FunctionDictGetOrDefault<DataTypeInt32, NameDictGetInt32OrDefault>; |
| 1316 | using FunctionDictGetInt64OrDefault = FunctionDictGetOrDefault<DataTypeInt64, NameDictGetInt64OrDefault>; |
| 1317 | using FunctionDictGetFloat32OrDefault = FunctionDictGetOrDefault<DataTypeFloat32, NameDictGetFloat32OrDefault>; |
| 1318 | using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64, NameDictGetFloat64OrDefault>; |
| 1319 | using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>; |
| 1320 | using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>; |
| 1321 | using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>; |
| 1322 | using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>; |
| 1323 | using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>; |
| 1324 | using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>; |
| 1325 | |
| 1326 | |
| 1327 | /// This variant of function derives the result type automatically. |
| 1328 | class FunctionDictGetNoType final : public IFunction |
| 1329 | { |
| 1330 | public: |
| 1331 | static constexpr auto name = "dictGet" ; |
| 1332 | |
| 1333 | static FunctionPtr create(const Context & context) |
| 1334 | { |
| 1335 | return std::make_shared<FunctionDictGetNoType>(context.getExternalDictionariesLoader(), context); |
| 1336 | } |
| 1337 | |
| 1338 | FunctionDictGetNoType(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) : dictionaries_loader(dictionaries_loader_), context(context_) {} |
| 1339 | |
| 1340 | String getName() const override { return name; } |
| 1341 | |
| 1342 | private: |
| 1343 | bool isVariadic() const override { return true; } |
| 1344 | size_t getNumberOfArguments() const override { return 0; } |
| 1345 | |
| 1346 | bool useDefaultImplementationForConstants() const final { return true; } |
| 1347 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 1348 | |
| 1349 | bool isInjective(const Block & sample_block) override |
| 1350 | { |
| 1351 | return isDictGetFunctionInjective(dictionaries_loader, sample_block); |
| 1352 | } |
| 1353 | |
| 1354 | DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override |
| 1355 | { |
| 1356 | if (arguments.size() != 3 && arguments.size() != 4) |
| 1357 | throw Exception{"Function " + getName() + " takes 3 or 4 arguments" , ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; |
| 1358 | |
| 1359 | String dict_name; |
| 1360 | if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get())) |
| 1361 | { |
| 1362 | dict_name = name_col->getValue<String>(); |
| 1363 | } |
| 1364 | else |
| 1365 | throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() |
| 1366 | + ", expected a const string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1367 | |
| 1368 | String attr_name; |
| 1369 | if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get())) |
| 1370 | { |
| 1371 | attr_name = name_col->getValue<String>(); |
| 1372 | } |
| 1373 | else |
| 1374 | throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() |
| 1375 | + ", expected a const string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1376 | |
| 1377 | if (!WhichDataType(arguments[2].type).isUInt64() && |
| 1378 | !isTuple(arguments[2].type)) |
| 1379 | throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() |
| 1380 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1381 | |
| 1382 | if (arguments.size() == 4) |
| 1383 | { |
| 1384 | const auto range_argument = arguments[3].type.get(); |
| 1385 | if (!(range_argument->isValueRepresentedByInteger() && |
| 1386 | range_argument->getSizeOfValueInMemory() <= sizeof(Int64))) |
| 1387 | throw Exception{"Illegal type " + range_argument->getName() + " of fourth argument of function " + getName() |
| 1388 | + ", must be convertible to " + TypeName<Int64>::get() + "." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1389 | } |
| 1390 | |
| 1391 | auto dict = dictionaries_loader.getDictionary(dict_name); |
| 1392 | const DictionaryStructure & structure = dict->getStructure(); |
| 1393 | |
| 1394 | for (const auto idx : ext::range(0, structure.attributes.size())) |
| 1395 | { |
| 1396 | const DictionaryAttribute & attribute = structure.attributes[idx]; |
| 1397 | if (attribute.name == attr_name) |
| 1398 | { |
| 1399 | WhichDataType dt = attribute.type; |
| 1400 | switch (dt.idx) |
| 1401 | { |
| 1402 | case TypeIndex::String: |
| 1403 | case TypeIndex::FixedString: |
| 1404 | impl = FunctionDictGetString::create(context); |
| 1405 | break; |
| 1406 | case TypeIndex::UInt8: |
| 1407 | impl = FunctionDictGetUInt8::create(context); |
| 1408 | break; |
| 1409 | case TypeIndex::UInt16: |
| 1410 | impl = FunctionDictGetUInt16::create(context); |
| 1411 | break; |
| 1412 | case TypeIndex::UInt32: |
| 1413 | impl = FunctionDictGetUInt32::create(context); |
| 1414 | break; |
| 1415 | case TypeIndex::UInt64: |
| 1416 | impl = FunctionDictGetUInt64::create(context); |
| 1417 | break; |
| 1418 | case TypeIndex::Int8: |
| 1419 | impl = FunctionDictGetInt8::create(context); |
| 1420 | break; |
| 1421 | case TypeIndex::Int16: |
| 1422 | impl = FunctionDictGetInt16::create(context); |
| 1423 | break; |
| 1424 | case TypeIndex::Int32: |
| 1425 | impl = FunctionDictGetInt32::create(context); |
| 1426 | break; |
| 1427 | case TypeIndex::Int64: |
| 1428 | impl = FunctionDictGetInt64::create(context); |
| 1429 | break; |
| 1430 | case TypeIndex::Float32: |
| 1431 | impl = FunctionDictGetFloat32::create(context); |
| 1432 | break; |
| 1433 | case TypeIndex::Float64: |
| 1434 | impl = FunctionDictGetFloat64::create(context); |
| 1435 | break; |
| 1436 | case TypeIndex::Date: |
| 1437 | impl = FunctionDictGetDate::create(context); |
| 1438 | break; |
| 1439 | case TypeIndex::DateTime: |
| 1440 | impl = FunctionDictGetDateTime::create(context); |
| 1441 | break; |
| 1442 | case TypeIndex::UUID: |
| 1443 | impl = FunctionDictGetUUID::create(context); |
| 1444 | break; |
| 1445 | case TypeIndex::Decimal32: |
| 1446 | impl = FunctionDictGetDecimal32::create(context, getDecimalScale(*attribute.type)); |
| 1447 | break; |
| 1448 | case TypeIndex::Decimal64: |
| 1449 | impl = FunctionDictGetDecimal64::create(context, getDecimalScale(*attribute.type)); |
| 1450 | break; |
| 1451 | case TypeIndex::Decimal128: |
| 1452 | impl = FunctionDictGetDecimal128::create(context, getDecimalScale(*attribute.type)); |
| 1453 | break; |
| 1454 | default: |
| 1455 | throw Exception("Unknown dictGet type" , ErrorCodes::UNKNOWN_TYPE); |
| 1456 | } |
| 1457 | return attribute.type; |
| 1458 | } |
| 1459 | } |
| 1460 | throw Exception{"No such attribute '" + attr_name + "'" , ErrorCodes::BAD_ARGUMENTS}; |
| 1461 | } |
| 1462 | |
| 1463 | bool isDeterministic() const override { return false; } |
| 1464 | |
| 1465 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 1466 | { |
| 1467 | impl->executeImpl(block, arguments, result, input_rows_count); |
| 1468 | } |
| 1469 | |
| 1470 | private: |
| 1471 | const ExternalDictionariesLoader & dictionaries_loader; |
| 1472 | const Context & context; |
| 1473 | mutable FunctionPtr impl; // underlying function used by dictGet function without explicit type info |
| 1474 | }; |
| 1475 | |
| 1476 | |
| 1477 | class FunctionDictGetNoTypeOrDefault final : public IFunction |
| 1478 | { |
| 1479 | public: |
| 1480 | static constexpr auto name = "dictGetOrDefault" ; |
| 1481 | |
| 1482 | static FunctionPtr create(const Context & context) |
| 1483 | { |
| 1484 | return std::make_shared<FunctionDictGetNoTypeOrDefault>(context.getExternalDictionariesLoader(), context); |
| 1485 | } |
| 1486 | |
| 1487 | FunctionDictGetNoTypeOrDefault(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) : dictionaries_loader(dictionaries_loader_), context(context_) {} |
| 1488 | |
| 1489 | String getName() const override { return name; } |
| 1490 | |
| 1491 | private: |
| 1492 | size_t getNumberOfArguments() const override { return 4; } |
| 1493 | |
| 1494 | bool useDefaultImplementationForConstants() const final { return true; } |
| 1495 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; } |
| 1496 | |
| 1497 | bool isInjective(const Block & sample_block) override |
| 1498 | { |
| 1499 | return isDictGetFunctionInjective(dictionaries_loader, sample_block); |
| 1500 | } |
| 1501 | |
| 1502 | DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override |
| 1503 | { |
| 1504 | String dict_name; |
| 1505 | if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get())) |
| 1506 | { |
| 1507 | dict_name = name_col->getValue<String>(); |
| 1508 | } |
| 1509 | else |
| 1510 | throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() |
| 1511 | + ", expected a const string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1512 | |
| 1513 | String attr_name; |
| 1514 | if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get())) |
| 1515 | { |
| 1516 | attr_name = name_col->getValue<String>(); |
| 1517 | } |
| 1518 | else |
| 1519 | throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() |
| 1520 | + ", expected a const string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1521 | |
| 1522 | if (!WhichDataType(arguments[2].type).isUInt64() && |
| 1523 | !isTuple(arguments[2].type)) |
| 1524 | throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() |
| 1525 | + ", must be UInt64 or tuple(...)." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1526 | |
| 1527 | auto dict = dictionaries_loader.getDictionary(dict_name); |
| 1528 | const DictionaryStructure & structure = dict->getStructure(); |
| 1529 | |
| 1530 | for (const auto idx : ext::range(0, structure.attributes.size())) |
| 1531 | { |
| 1532 | const DictionaryAttribute & attribute = structure.attributes[idx]; |
| 1533 | if (attribute.name == attr_name) |
| 1534 | { |
| 1535 | auto arg_type = arguments[3].type; |
| 1536 | WhichDataType dt = attribute.type; |
| 1537 | |
| 1538 | if ((arg_type->getTypeId() != dt.idx) || (dt.isStringOrFixedString() && !isString(arg_type))) |
| 1539 | throw Exception{"Illegal type " + arg_type->getName() + " of fourth argument of function " + getName() + |
| 1540 | ", must be " + getTypeName(dt.idx) + "." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1541 | |
| 1542 | switch (dt.idx) |
| 1543 | { |
| 1544 | case TypeIndex::String: |
| 1545 | impl = FunctionDictGetStringOrDefault::create(context); |
| 1546 | break; |
| 1547 | case TypeIndex::UInt8: |
| 1548 | impl = FunctionDictGetUInt8OrDefault::create(context); |
| 1549 | break; |
| 1550 | case TypeIndex::UInt16: |
| 1551 | impl = FunctionDictGetUInt16OrDefault::create(context); |
| 1552 | break; |
| 1553 | case TypeIndex::UInt32: |
| 1554 | impl = FunctionDictGetUInt32OrDefault::create(context); |
| 1555 | break; |
| 1556 | case TypeIndex::UInt64: |
| 1557 | impl = FunctionDictGetUInt64OrDefault::create(context); |
| 1558 | break; |
| 1559 | case TypeIndex::Int8: |
| 1560 | impl = FunctionDictGetInt8OrDefault::create(context); |
| 1561 | break; |
| 1562 | case TypeIndex::Int16: |
| 1563 | impl = FunctionDictGetInt16OrDefault::create(context); |
| 1564 | break; |
| 1565 | case TypeIndex::Int32: |
| 1566 | impl = FunctionDictGetInt32OrDefault::create(context); |
| 1567 | break; |
| 1568 | case TypeIndex::Int64: |
| 1569 | impl = FunctionDictGetInt64OrDefault::create(context); |
| 1570 | break; |
| 1571 | case TypeIndex::Float32: |
| 1572 | impl = FunctionDictGetFloat32OrDefault::create(context); |
| 1573 | break; |
| 1574 | case TypeIndex::Float64: |
| 1575 | impl = FunctionDictGetFloat64OrDefault::create(context); |
| 1576 | break; |
| 1577 | case TypeIndex::Date: |
| 1578 | impl = FunctionDictGetDateOrDefault::create(context); |
| 1579 | break; |
| 1580 | case TypeIndex::DateTime: |
| 1581 | impl = FunctionDictGetDateTimeOrDefault::create(context); |
| 1582 | break; |
| 1583 | case TypeIndex::UUID: |
| 1584 | impl = FunctionDictGetUUIDOrDefault::create(context); |
| 1585 | break; |
| 1586 | case TypeIndex::Decimal32: |
| 1587 | impl = FunctionDictGetDecimal32OrDefault::create(context, getDecimalScale(*attribute.type)); |
| 1588 | break; |
| 1589 | case TypeIndex::Decimal64: |
| 1590 | impl = FunctionDictGetDecimal64OrDefault::create(context, getDecimalScale(*attribute.type)); |
| 1591 | break; |
| 1592 | case TypeIndex::Decimal128: |
| 1593 | impl = FunctionDictGetDecimal128OrDefault::create(context, getDecimalScale(*attribute.type)); |
| 1594 | break; |
| 1595 | default: |
| 1596 | throw Exception("Unknown dictGetOrDefault type" , ErrorCodes::UNKNOWN_TYPE); |
| 1597 | } |
| 1598 | |
| 1599 | return attribute.type; |
| 1600 | } |
| 1601 | } |
| 1602 | throw Exception{"No such attribute '" + attr_name + "'" , ErrorCodes::BAD_ARGUMENTS}; |
| 1603 | } |
| 1604 | |
| 1605 | bool isDeterministic() const override { return false; } |
| 1606 | |
| 1607 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 1608 | { |
| 1609 | impl->executeImpl(block, arguments, result, input_rows_count); |
| 1610 | } |
| 1611 | |
| 1612 | private: |
| 1613 | const ExternalDictionariesLoader & dictionaries_loader; |
| 1614 | const Context & context; |
| 1615 | mutable FunctionPtr impl; // underlying function used by dictGet function without explicit type info |
| 1616 | }; |
| 1617 | |
| 1618 | /// Functions to work with hierarchies. |
| 1619 | |
| 1620 | class FunctionDictGetHierarchy final : public IFunction |
| 1621 | { |
| 1622 | public: |
| 1623 | static constexpr auto name = "dictGetHierarchy" ; |
| 1624 | |
| 1625 | static FunctionPtr create(const Context & context) |
| 1626 | { |
| 1627 | return std::make_shared<FunctionDictGetHierarchy>(context.getExternalDictionariesLoader(), context); |
| 1628 | } |
| 1629 | |
| 1630 | FunctionDictGetHierarchy(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) |
| 1631 | : dictionaries_loader(dictionaries_loader_) |
| 1632 | , context(context_) {} |
| 1633 | |
| 1634 | String getName() const override { return name; } |
| 1635 | |
| 1636 | private: |
| 1637 | size_t getNumberOfArguments() const override { return 2; } |
| 1638 | bool isInjective(const Block & /*sample_block*/) override { return true; } |
| 1639 | |
| 1640 | bool useDefaultImplementationForConstants() const final { return true; } |
| 1641 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } |
| 1642 | |
| 1643 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 1644 | { |
| 1645 | if (!isString(arguments[0])) |
| 1646 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 1647 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1648 | |
| 1649 | if (!WhichDataType(arguments[1]).isUInt64()) |
| 1650 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 1651 | + ", must be UInt64." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1652 | |
| 1653 | return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()); |
| 1654 | } |
| 1655 | |
| 1656 | bool isDeterministic() const override { return false; } |
| 1657 | |
| 1658 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 1659 | { |
| 1660 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 1661 | if (!dict_name_col) |
| 1662 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1663 | |
| 1664 | if (input_rows_count == 0) |
| 1665 | { |
| 1666 | auto & elem = block.getByPosition(result); |
| 1667 | elem.column = elem.type->createColumn(); |
| 1668 | return; |
| 1669 | } |
| 1670 | |
| 1671 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 1672 | const auto dict_ptr = dict.get(); |
| 1673 | |
| 1674 | if (!context.hasDictionaryAccessRights(dict_ptr->getName())) |
| 1675 | { |
| 1676 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 1677 | + dict->getName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 1678 | } |
| 1679 | |
| 1680 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) && |
| 1681 | !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) && |
| 1682 | !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) |
| 1683 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 1684 | } |
| 1685 | |
| 1686 | template <typename DictionaryType> |
| 1687 | bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, |
| 1688 | const IDictionaryBase * dictionary) |
| 1689 | { |
| 1690 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 1691 | if (!dict) |
| 1692 | return false; |
| 1693 | |
| 1694 | if (!dict->hasHierarchy()) |
| 1695 | throw Exception{"Dictionary does not have a hierarchy" , ErrorCodes::UNSUPPORTED_METHOD}; |
| 1696 | |
| 1697 | const auto get_hierarchies = [&] (const PaddedPODArray<UInt64> & in, PaddedPODArray<UInt64> & out, PaddedPODArray<UInt64> & offsets) |
| 1698 | { |
| 1699 | const auto size = in.size(); |
| 1700 | |
| 1701 | /// copy of `in` array |
| 1702 | auto in_array = std::make_unique<PaddedPODArray<UInt64>>(std::begin(in), std::end(in)); |
| 1703 | /// used for storing and handling result of ::toParent call |
| 1704 | auto out_array = std::make_unique<PaddedPODArray<UInt64>>(size); |
| 1705 | /// resulting hierarchies |
| 1706 | std::vector<std::vector<IDictionary::Key>> hierarchies(size); /// TODO Bad code, poor performance. |
| 1707 | |
| 1708 | /// total number of non-zero elements, used for allocating all the required memory upfront |
| 1709 | size_t total_count = 0; |
| 1710 | |
| 1711 | while (true) |
| 1712 | { |
| 1713 | auto all_zeroes = true; |
| 1714 | |
| 1715 | /// erase zeroed identifiers, store non-zeroed ones |
| 1716 | for (const auto i : ext::range(0, size)) |
| 1717 | { |
| 1718 | const auto id = (*in_array)[i]; |
| 1719 | if (0 == id) |
| 1720 | continue; |
| 1721 | |
| 1722 | |
| 1723 | auto & hierarchy = hierarchies[i]; |
| 1724 | |
| 1725 | /// Checking for loop |
| 1726 | if (std::find(std::begin(hierarchy), std::end(hierarchy), id) != std::end(hierarchy)) |
| 1727 | continue; |
| 1728 | |
| 1729 | all_zeroes = false; |
| 1730 | /// place id at it's corresponding place |
| 1731 | hierarchy.push_back(id); |
| 1732 | |
| 1733 | ++total_count; |
| 1734 | } |
| 1735 | |
| 1736 | if (all_zeroes) |
| 1737 | break; |
| 1738 | |
| 1739 | /// translate all non-zero identifiers at once |
| 1740 | dict->toParent(*in_array, *out_array); |
| 1741 | |
| 1742 | /// we're going to use the `in_array` from this iteration as `out_array` on the next one |
| 1743 | std::swap(in_array, out_array); |
| 1744 | } |
| 1745 | |
| 1746 | out.reserve(total_count); |
| 1747 | offsets.resize(size); |
| 1748 | |
| 1749 | for (const auto i : ext::range(0, size)) |
| 1750 | { |
| 1751 | const auto & ids = hierarchies[i]; |
| 1752 | out.insert_assume_reserved(std::begin(ids), std::end(ids)); |
| 1753 | offsets[i] = out.size(); |
| 1754 | } |
| 1755 | }; |
| 1756 | |
| 1757 | const auto id_col_untyped = block.getByPosition(arguments[1]).column.get(); |
| 1758 | if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped)) |
| 1759 | { |
| 1760 | const auto & in = id_col->getData(); |
| 1761 | auto backend = ColumnUInt64::create(); |
| 1762 | auto offsets = ColumnArray::ColumnOffsets::create(); |
| 1763 | get_hierarchies(in, backend->getData(), offsets->getData()); |
| 1764 | block.getByPosition(result).column = ColumnArray::create(std::move(backend), std::move(offsets)); |
| 1765 | } |
| 1766 | else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped)) |
| 1767 | { |
| 1768 | const PaddedPODArray<UInt64> in(1, id_col_const->getValue<UInt64>()); |
| 1769 | auto backend = ColumnUInt64::create(); |
| 1770 | auto offsets = ColumnArray::ColumnOffsets::create(); |
| 1771 | get_hierarchies(in, backend->getData(), offsets->getData()); |
| 1772 | auto array = ColumnArray::create(std::move(backend), std::move(offsets)); |
| 1773 | block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>()); |
| 1774 | } |
| 1775 | else |
| 1776 | throw Exception{"Second argument of function " + getName() + " must be UInt64" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1777 | |
| 1778 | return true; |
| 1779 | } |
| 1780 | |
| 1781 | const ExternalDictionariesLoader & dictionaries_loader; |
| 1782 | const Context & context; |
| 1783 | }; |
| 1784 | |
| 1785 | |
| 1786 | class FunctionDictIsIn final : public IFunction |
| 1787 | { |
| 1788 | public: |
| 1789 | static constexpr auto name = "dictIsIn" ; |
| 1790 | |
| 1791 | static FunctionPtr create(const Context & context) |
| 1792 | { |
| 1793 | return std::make_shared<FunctionDictIsIn>(context.getExternalDictionariesLoader(), context); |
| 1794 | } |
| 1795 | |
| 1796 | FunctionDictIsIn(const ExternalDictionariesLoader & dictionaries_loader_, const Context & context_) |
| 1797 | : dictionaries_loader(dictionaries_loader_) |
| 1798 | , context(context_) {} |
| 1799 | |
| 1800 | String getName() const override { return name; } |
| 1801 | |
| 1802 | private: |
| 1803 | size_t getNumberOfArguments() const override { return 3; } |
| 1804 | |
| 1805 | bool useDefaultImplementationForConstants() const final { return true; } |
| 1806 | ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } |
| 1807 | |
| 1808 | DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override |
| 1809 | { |
| 1810 | if (!isString(arguments[0])) |
| 1811 | throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() |
| 1812 | + ", expected a string." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1813 | |
| 1814 | if (!WhichDataType(arguments[1]).isUInt64()) |
| 1815 | throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() |
| 1816 | + ", must be UInt64." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1817 | |
| 1818 | if (!WhichDataType(arguments[2]).isUInt64()) |
| 1819 | throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() |
| 1820 | + ", must be UInt64." , ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; |
| 1821 | |
| 1822 | return std::make_shared<DataTypeUInt8>(); |
| 1823 | } |
| 1824 | |
| 1825 | bool isDeterministic() const override { return false; } |
| 1826 | |
| 1827 | void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override |
| 1828 | { |
| 1829 | const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get()); |
| 1830 | if (!dict_name_col) |
| 1831 | throw Exception{"First argument of function " + getName() + " must be a constant string" , ErrorCodes::ILLEGAL_COLUMN}; |
| 1832 | |
| 1833 | if (input_rows_count == 0) |
| 1834 | { |
| 1835 | auto & elem = block.getByPosition(result); |
| 1836 | elem.column = elem.type->createColumn(); |
| 1837 | return; |
| 1838 | } |
| 1839 | |
| 1840 | auto dict = dictionaries_loader.getDictionary(dict_name_col->getValue<String>()); |
| 1841 | const auto dict_ptr = dict.get(); |
| 1842 | |
| 1843 | if (!context.hasDictionaryAccessRights(dict_ptr->getName())) |
| 1844 | { |
| 1845 | throw Exception{"For function " + getName() + ", cannot access dictionary " |
| 1846 | + dict->getName() + " on database " + context.getCurrentDatabase(), ErrorCodes::DICTIONARY_ACCESS_DENIED}; |
| 1847 | } |
| 1848 | |
| 1849 | if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) |
| 1850 | && !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) |
| 1851 | && !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr)) |
| 1852 | throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; |
| 1853 | } |
| 1854 | |
| 1855 | template <typename DictionaryType> |
| 1856 | bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result, |
| 1857 | const IDictionaryBase * dictionary) |
| 1858 | { |
| 1859 | const auto dict = typeid_cast<const DictionaryType *>(dictionary); |
| 1860 | if (!dict) |
| 1861 | return false; |
| 1862 | |
| 1863 | if (!dict->hasHierarchy()) |
| 1864 | throw Exception{"Dictionary does not have a hierarchy" , ErrorCodes::UNSUPPORTED_METHOD}; |
| 1865 | |
| 1866 | const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get(); |
| 1867 | const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get(); |
| 1868 | |
| 1869 | if (const auto child_id_col = checkAndGetColumn<ColumnUInt64>(child_id_col_untyped)) |
| 1870 | execute(block, result, dict, child_id_col, ancestor_id_col_untyped); |
| 1871 | else if (const auto child_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped)) |
| 1872 | execute(block, result, dict, child_id_col_const, ancestor_id_col_untyped); |
| 1873 | else |
| 1874 | throw Exception{"Illegal column " + child_id_col_untyped->getName() |
| 1875 | + " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1876 | |
| 1877 | return true; |
| 1878 | } |
| 1879 | |
| 1880 | template <typename DictionaryType> |
| 1881 | bool execute(Block & block, const size_t result, const DictionaryType * dictionary, |
| 1882 | const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) |
| 1883 | { |
| 1884 | if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped)) |
| 1885 | { |
| 1886 | auto out = ColumnUInt8::create(); |
| 1887 | |
| 1888 | const auto & child_ids = child_id_col->getData(); |
| 1889 | const auto & ancestor_ids = ancestor_id_col->getData(); |
| 1890 | auto & data = out->getData(); |
| 1891 | const auto size = child_id_col->size(); |
| 1892 | data.resize(size); |
| 1893 | |
| 1894 | dictionary->isInVectorVector(child_ids, ancestor_ids, data); |
| 1895 | block.getByPosition(result).column = std::move(out); |
| 1896 | } |
| 1897 | else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped)) |
| 1898 | { |
| 1899 | auto out = ColumnUInt8::create(); |
| 1900 | |
| 1901 | const auto & child_ids = child_id_col->getData(); |
| 1902 | const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>(); |
| 1903 | auto & data = out->getData(); |
| 1904 | const auto size = child_id_col->size(); |
| 1905 | data.resize(size); |
| 1906 | |
| 1907 | dictionary->isInVectorConstant(child_ids, ancestor_id, data); |
| 1908 | block.getByPosition(result).column = std::move(out); |
| 1909 | } |
| 1910 | else |
| 1911 | { |
| 1912 | throw Exception{"Illegal column " + ancestor_id_col_untyped->getName() |
| 1913 | + " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1914 | } |
| 1915 | |
| 1916 | return true; |
| 1917 | } |
| 1918 | |
| 1919 | template <typename DictionaryType> |
| 1920 | bool execute(Block & block, const size_t result, const DictionaryType * dictionary, |
| 1921 | const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped) |
| 1922 | { |
| 1923 | if (const auto ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped)) |
| 1924 | { |
| 1925 | auto out = ColumnUInt8::create(); |
| 1926 | |
| 1927 | const auto child_id = child_id_col->getValue<UInt64>(); |
| 1928 | const auto & ancestor_ids = ancestor_id_col->getData(); |
| 1929 | auto & data = out->getData(); |
| 1930 | const auto size = child_id_col->size(); |
| 1931 | data.resize(size); |
| 1932 | |
| 1933 | dictionary->isInConstantVector(child_id, ancestor_ids, data); |
| 1934 | block.getByPosition(result).column = std::move(out); |
| 1935 | } |
| 1936 | else if (const auto ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped)) |
| 1937 | { |
| 1938 | const auto child_id = child_id_col->getValue<UInt64>(); |
| 1939 | const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>(); |
| 1940 | UInt8 res = 0; |
| 1941 | |
| 1942 | dictionary->isInConstantConstant(child_id, ancestor_id, res); |
| 1943 | block.getByPosition(result).column = DataTypeUInt8().createColumnConst(child_id_col->size(), res); |
| 1944 | } |
| 1945 | else |
| 1946 | throw Exception{"Illegal column " + ancestor_id_col_untyped->getName() |
| 1947 | + " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; |
| 1948 | |
| 1949 | return true; |
| 1950 | } |
| 1951 | |
| 1952 | const ExternalDictionariesLoader & dictionaries_loader; |
| 1953 | const Context & context; |
| 1954 | }; |
| 1955 | |
| 1956 | |
| 1957 | template <typename T> |
| 1958 | static const PaddedPODArray<T> & getColumnDataAsPaddedPODArray(const IColumn & column, PaddedPODArray<T> & backup_storage) |
| 1959 | { |
| 1960 | if (!isColumnConst(column)) |
| 1961 | { |
| 1962 | if (const auto vector_col = checkAndGetColumn<ColumnVector<T>>(&column)) |
| 1963 | { |
| 1964 | return vector_col->getData(); |
| 1965 | } |
| 1966 | } |
| 1967 | |
| 1968 | const auto full_column = column.convertToFullColumnIfConst(); |
| 1969 | |
| 1970 | // With type conversion and const columns we need to use backup storage here |
| 1971 | const auto size = full_column->size(); |
| 1972 | backup_storage.resize(size); |
| 1973 | for (size_t i = 0; i < size; ++i) |
| 1974 | backup_storage[i] = full_column->getUInt(i); |
| 1975 | |
| 1976 | return backup_storage; |
| 1977 | } |
| 1978 | |
| 1979 | } |
| 1980 | |