| 1 | #pragma once | 
| 2 |  | 
| 3 | #include <Columns/ColumnArray.h> | 
| 4 | #include <Columns/ColumnConst.h> | 
| 5 | #include <Columns/ColumnString.h> | 
| 6 | #include <Columns/ColumnVector.h> | 
| 7 | #include <DataTypes/DataTypeArray.h> | 
| 8 | #include <DataTypes/DataTypeString.h> | 
| 9 | #include <DataTypes/DataTypesNumber.h> | 
| 10 | #include <Functions/FunctionHelpers.h> | 
| 11 | #include <Functions/IFunctionImpl.h> | 
| 12 | #include <IO/WriteHelpers.h> | 
| 13 | #include <Interpreters/Context.h> | 
| 14 | #include <common/StringRef.h> | 
| 15 |  | 
| 16 | namespace DB | 
| 17 | { | 
| 18 | /** Search and replace functions in strings: | 
| 19 |   * | 
| 20 |   * position(haystack, needle)     - the normal search for a substring in a string, returns the position (in bytes) of the found substring starting with 1, or 0 if no substring is found. | 
| 21 |   * positionUTF8(haystack, needle) - the same, but the position is calculated at code points, provided that the string is encoded in UTF-8. | 
| 22 |   * positionCaseInsensitive(haystack, needle) | 
| 23 |   * positionCaseInsensitiveUTF8(haystack, needle) | 
| 24 |   * | 
| 25 |   * like(haystack, pattern)        - search by the regular expression LIKE; Returns 0 or 1. Case-insensitive, but only for Latin. | 
| 26 |   * notLike(haystack, pattern) | 
| 27 |   * | 
| 28 |   * match(haystack, pattern)       - search by regular expression re2; Returns 0 or 1. | 
| 29 |   * multiMatchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns 0 or 1 if any pattern_i matches. | 
| 30 |   * multiMatchAnyIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns index of any match or zero if none; | 
| 31 |   * multiMatchAllIndices(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- search by re2 regular expressions pattern_i; Returns an array of matched indices in any order; | 
| 32 |   * | 
| 33 |   * Applies regexp re2 and pulls: | 
| 34 |   * - the first subpattern, if the regexp has a subpattern; | 
| 35 |   * - the zero subpattern (the match part, otherwise); | 
| 36 |   * - if not match - an empty string. | 
| 37 |   * extract(haystack, pattern) | 
| 38 |   * | 
| 39 |   * replaceOne(haystack, pattern, replacement) - replacing the pattern with the specified rules, only the first occurrence. | 
| 40 |   * replaceAll(haystack, pattern, replacement) - replacing the pattern with the specified rules, all occurrences. | 
| 41 |   * | 
| 42 |   * replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence. | 
| 43 |   * replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences. | 
| 44 |   * | 
| 45 |   * multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack | 
| 46 |   * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 47 |   * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 48 |   * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 49 |   * | 
| 50 |   * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found | 
| 51 |   * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 52 |   * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 53 |   * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 54 |   * | 
| 55 |   * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 | 
| 56 |   * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 57 |   * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 58 |   * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 59 |  | 
| 60 |   * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found | 
| 61 |   * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 62 |   * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 63 |   * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) | 
| 64 |   */ | 
| 65 |  | 
| 66 | namespace ErrorCodes | 
| 67 | { | 
| 68 |     extern const int ILLEGAL_TYPE_OF_ARGUMENT; | 
| 69 |     extern const int ILLEGAL_COLUMN; | 
| 70 |     extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; | 
| 71 |     extern const int FUNCTION_NOT_ALLOWED; | 
| 72 | } | 
| 73 |  | 
| 74 | template <typename Impl, typename Name> | 
| 75 | class FunctionsStringSearch : public IFunction | 
| 76 | { | 
| 77 | public: | 
| 78 |     static constexpr auto name = Name::name; | 
| 79 |     static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearch>(); } | 
| 80 |  | 
| 81 |     String getName() const override { return name; } | 
| 82 |  | 
| 83 |     size_t getNumberOfArguments() const override { return 2; } | 
| 84 |  | 
| 85 |     DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override | 
| 86 |     { | 
| 87 |         if (!isString(arguments[0])) | 
| 88 |             throw Exception( | 
| 89 |                 "Illegal type "  + arguments[0]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 90 |  | 
| 91 |         if (!isString(arguments[1])) | 
| 92 |             throw Exception( | 
| 93 |                 "Illegal type "  + arguments[1]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 94 |  | 
| 95 |         return std::make_shared<DataTypeNumber<typename Impl::ResultType>>(); | 
| 96 |     } | 
| 97 |  | 
| 98 |     void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override | 
| 99 |     { | 
| 100 |         using ResultType = typename Impl::ResultType; | 
| 101 |  | 
| 102 |         const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; | 
| 103 |         const ColumnPtr & column_needle = block.getByPosition(arguments[1]).column; | 
| 104 |  | 
| 105 |         const ColumnConst * col_haystack_const = typeid_cast<const ColumnConst *>(&*column_haystack); | 
| 106 |         const ColumnConst * col_needle_const = typeid_cast<const ColumnConst *>(&*column_needle); | 
| 107 |  | 
| 108 |         if (col_haystack_const && col_needle_const) | 
| 109 |         { | 
| 110 |             ResultType res{}; | 
| 111 |             Impl::constant_constant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res); | 
| 112 |             block.getByPosition(result).column | 
| 113 |                 = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res)); | 
| 114 |             return; | 
| 115 |         } | 
| 116 |  | 
| 117 |         auto col_res = ColumnVector<ResultType>::create(); | 
| 118 |  | 
| 119 |         typename ColumnVector<ResultType>::Container & vec_res = col_res->getData(); | 
| 120 |         vec_res.resize(column_haystack->size()); | 
| 121 |  | 
| 122 |         const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack); | 
| 123 |         const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle); | 
| 124 |  | 
| 125 |         if (col_haystack_vector && col_needle_vector) | 
| 126 |             Impl::vector_vector( | 
| 127 |                 col_haystack_vector->getChars(), | 
| 128 |                 col_haystack_vector->getOffsets(), | 
| 129 |                 col_needle_vector->getChars(), | 
| 130 |                 col_needle_vector->getOffsets(), | 
| 131 |                 vec_res); | 
| 132 |         else if (col_haystack_vector && col_needle_const) | 
| 133 |             Impl::vector_constant( | 
| 134 |                 col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res); | 
| 135 |         else if (col_haystack_const && col_needle_vector) | 
| 136 |             Impl::constant_vector( | 
| 137 |                 col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res); | 
| 138 |         else | 
| 139 |             throw Exception( | 
| 140 |                 "Illegal columns "  + block.getByPosition(arguments[0]).column->getName() + " and "  | 
| 141 |                     + block.getByPosition(arguments[1]).column->getName() + " of arguments of function "  + getName(), | 
| 142 |                 ErrorCodes::ILLEGAL_COLUMN); | 
| 143 |  | 
| 144 |         block.getByPosition(result).column = std::move(col_res); | 
| 145 |     } | 
| 146 | }; | 
| 147 |  | 
| 148 |  | 
| 149 | template <typename Impl, typename Name> | 
| 150 | class FunctionsStringSearchToString : public IFunction | 
| 151 | { | 
| 152 | public: | 
| 153 |     static constexpr auto name = Name::name; | 
| 154 |     static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); } | 
| 155 |  | 
| 156 |     String getName() const override { return name; } | 
| 157 |  | 
| 158 |     size_t getNumberOfArguments() const override { return 2; } | 
| 159 |  | 
| 160 |     bool useDefaultImplementationForConstants() const override { return true; } | 
| 161 |     ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } | 
| 162 |  | 
| 163 |     DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override | 
| 164 |     { | 
| 165 |         if (!isString(arguments[0])) | 
| 166 |             throw Exception( | 
| 167 |                 "Illegal type "  + arguments[0]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 168 |  | 
| 169 |         if (!isString(arguments[1])) | 
| 170 |             throw Exception( | 
| 171 |                 "Illegal type "  + arguments[1]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 172 |  | 
| 173 |         return std::make_shared<DataTypeString>(); | 
| 174 |     } | 
| 175 |  | 
| 176 |     void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override | 
| 177 |     { | 
| 178 |         const ColumnPtr column = block.getByPosition(arguments[0]).column; | 
| 179 |         const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; | 
| 180 |  | 
| 181 |         const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle); | 
| 182 |         if (!col_needle) | 
| 183 |             throw Exception("Second argument of function "  + getName() + " must be constant string" , ErrorCodes::ILLEGAL_COLUMN); | 
| 184 |  | 
| 185 |         if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get())) | 
| 186 |         { | 
| 187 |             auto col_res = ColumnString::create(); | 
| 188 |  | 
| 189 |             ColumnString::Chars & vec_res = col_res->getChars(); | 
| 190 |             ColumnString::Offsets & offsets_res = col_res->getOffsets(); | 
| 191 |             Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res); | 
| 192 |  | 
| 193 |             block.getByPosition(result).column = std::move(col_res); | 
| 194 |         } | 
| 195 |         else | 
| 196 |             throw Exception( | 
| 197 |                 "Illegal column "  + block.getByPosition(arguments[0]).column->getName() + " of argument of function "  + getName(), | 
| 198 |                 ErrorCodes::ILLEGAL_COLUMN); | 
| 199 |     } | 
| 200 | }; | 
| 201 |  | 
| 202 | template <typename Impl, typename Name> | 
| 203 | class FunctionsMultiStringPosition : public IFunction | 
| 204 | { | 
| 205 | public: | 
| 206 |     static constexpr auto name = Name::name; | 
| 207 |     static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringPosition>(); } | 
| 208 |  | 
| 209 |     String getName() const override { return name; } | 
| 210 |  | 
| 211 |     size_t getNumberOfArguments() const override { return 2; } | 
| 212 |     bool useDefaultImplementationForConstants() const override { return true; } | 
| 213 |     ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } | 
| 214 |  | 
| 215 |     DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override | 
| 216 |     { | 
| 217 |         if (!isString(arguments[0])) | 
| 218 |             throw Exception( | 
| 219 |                 "Illegal type "  + arguments[0]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 220 |  | 
| 221 |         const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get()); | 
| 222 |         if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get())) | 
| 223 |             throw Exception( | 
| 224 |                 "Illegal type "  + arguments[1]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 225 |  | 
| 226 |         return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()); | 
| 227 |     } | 
| 228 |  | 
| 229 |     void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override | 
| 230 |     { | 
| 231 |         using ResultType = typename Impl::ResultType; | 
| 232 |  | 
| 233 |         const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; | 
| 234 |  | 
| 235 |         const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack); | 
| 236 |  | 
| 237 |         const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; | 
| 238 |         const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get()); | 
| 239 |  | 
| 240 |         if (!col_const_arr) | 
| 241 |             throw Exception( | 
| 242 |                 "Illegal column "  + block.getByPosition(arguments[1]).column->getName() + ". The array is not const" , | 
| 243 |                 ErrorCodes::ILLEGAL_COLUMN); | 
| 244 |  | 
| 245 |         Array src_arr = col_const_arr->getValue<Array>(); | 
| 246 |  | 
| 247 |         if (src_arr.size() > std::numeric_limits<UInt8>::max()) | 
| 248 |             throw Exception( | 
| 249 |                 "Number of arguments for function "  + getName() + " doesn't match: passed "  + std::to_string(src_arr.size()) | 
| 250 |                     + ", should be at most 255" , | 
| 251 |                 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); | 
| 252 |  | 
| 253 |         std::vector<StringRef> refs; | 
| 254 |         for (const auto & el : src_arr) | 
| 255 |             refs.emplace_back(el.get<String>()); | 
| 256 |  | 
| 257 |         const size_t column_haystack_size = column_haystack->size(); | 
| 258 |  | 
| 259 |         auto col_res = ColumnVector<ResultType>::create(); | 
| 260 |         auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); | 
| 261 |  | 
| 262 |         auto & vec_res = col_res->getData(); | 
| 263 |         auto & offsets_res = col_offsets->getData(); | 
| 264 |  | 
| 265 |         vec_res.resize(column_haystack_size * refs.size()); | 
| 266 |  | 
| 267 |         if (col_haystack_vector) | 
| 268 |             Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); | 
| 269 |         else | 
| 270 |             throw Exception("Illegal column "  + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); | 
| 271 |  | 
| 272 |         size_t refs_size = refs.size(); | 
| 273 |         size_t accum = refs_size; | 
| 274 |  | 
| 275 |         for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) | 
| 276 |             offsets_res[i] = accum; | 
| 277 |  | 
| 278 |         block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); | 
| 279 |     } | 
| 280 | }; | 
| 281 |  | 
| 282 | /// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. | 
| 283 | /// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions | 
| 284 | template <typename Impl, typename Name, size_t LimitArgs = std::numeric_limits<UInt8>::max()> | 
| 285 | class FunctionsMultiStringSearch : public IFunction | 
| 286 | { | 
| 287 |     static_assert(LimitArgs > 0); | 
| 288 |  | 
| 289 | public: | 
| 290 |     static constexpr auto name = Name::name; | 
| 291 |     static FunctionPtr create(const Context & context) | 
| 292 |     { | 
| 293 |         if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) | 
| 294 |             throw Exception( | 
| 295 |                 "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0" , ErrorCodes::FUNCTION_NOT_ALLOWED); | 
| 296 |  | 
| 297 |         return std::make_shared<FunctionsMultiStringSearch>(); | 
| 298 |     } | 
| 299 |  | 
| 300 |     String getName() const override { return name; } | 
| 301 |  | 
| 302 |     size_t getNumberOfArguments() const override { return 2; } | 
| 303 |     bool useDefaultImplementationForConstants() const override { return true; } | 
| 304 |     ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } | 
| 305 |  | 
| 306 |     DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override | 
| 307 |     { | 
| 308 |         if (!isString(arguments[0])) | 
| 309 |             throw Exception( | 
| 310 |                 "Illegal type "  + arguments[0]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 311 |  | 
| 312 |         const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get()); | 
| 313 |         if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get())) | 
| 314 |             throw Exception( | 
| 315 |                 "Illegal type "  + arguments[1]->getName() + " of argument of function "  + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); | 
| 316 |         return Impl::ReturnType(); | 
| 317 |     } | 
| 318 |  | 
| 319 |     void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override | 
| 320 |     { | 
| 321 |         using ResultType = typename Impl::ResultType; | 
| 322 |  | 
| 323 |         const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; | 
| 324 |  | 
| 325 |         const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack); | 
| 326 |  | 
| 327 |         const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; | 
| 328 |         const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get()); | 
| 329 |  | 
| 330 |         if (!col_const_arr) | 
| 331 |             throw Exception( | 
| 332 |                 "Illegal column "  + block.getByPosition(arguments[1]).column->getName() + ". The array is not const" , | 
| 333 |                 ErrorCodes::ILLEGAL_COLUMN); | 
| 334 |  | 
| 335 |         Array src_arr = col_const_arr->getValue<Array>(); | 
| 336 |  | 
| 337 |         if (src_arr.size() > LimitArgs) | 
| 338 |             throw Exception( | 
| 339 |                 "Number of arguments for function "  + getName() + " doesn't match: passed "  + std::to_string(src_arr.size()) | 
| 340 |                     + ", should be at most "  + std::to_string(LimitArgs), | 
| 341 |                 ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); | 
| 342 |  | 
| 343 |         std::vector<StringRef> refs; | 
| 344 |         refs.reserve(src_arr.size()); | 
| 345 |  | 
| 346 |         for (const auto & el : src_arr) | 
| 347 |             refs.emplace_back(el.get<String>()); | 
| 348 |  | 
| 349 |         auto col_res = ColumnVector<ResultType>::create(); | 
| 350 |         auto col_offsets = ColumnArray::ColumnOffsets::create(); | 
| 351 |  | 
| 352 |         auto & vec_res = col_res->getData(); | 
| 353 |         auto & offsets_res = col_offsets->getData(); | 
| 354 |  | 
| 355 |         /// The blame for resizing output is for the callee. | 
| 356 |         if (col_haystack_vector) | 
| 357 |             Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res); | 
| 358 |         else | 
| 359 |             throw Exception("Illegal column "  + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); | 
| 360 |  | 
| 361 |         if constexpr (Impl::is_column_array) | 
| 362 |             block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); | 
| 363 |         else | 
| 364 |             block.getByPosition(result).column = std::move(col_res); | 
| 365 |     } | 
| 366 | }; | 
| 367 |  | 
| 368 | } | 
| 369 |  |