diff --git a/dbms/src/Functions/FunctionsMultiStringPosition.h b/dbms/src/Functions/FunctionsMultiStringPosition.h new file mode 100644 index 00000000000..40a38a2a0ad --- /dev/null +++ b/dbms/src/Functions/FunctionsMultiStringPosition.h @@ -0,0 +1,129 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack + * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found + * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 + * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + + * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found + * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +template +class FunctionsMultiStringPosition : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); + if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + using ResultType = typename Impl::ResultType; + + const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; + + const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + + const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; + const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); + + if (!col_const_arr) + throw Exception( + "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", + ErrorCodes::ILLEGAL_COLUMN); + + Array src_arr = col_const_arr->getValue(); + + if (src_arr.size() > std::numeric_limits::max()) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + + ", should be at most 255", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector refs; + for (const auto & el : src_arr) + refs.emplace_back(el.get()); + + const size_t column_haystack_size = column_haystack->size(); + + auto col_res = ColumnVector::create(); + auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); + + auto & vec_res = col_res->getData(); + auto & offsets_res = col_offsets->getData(); + + vec_res.resize(column_haystack_size * refs.size()); + + if (col_haystack_vector) + Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); + + size_t refs_size = refs.size(); + size_t accum = refs_size; + + for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) + offsets_res[i] = accum; + + block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsMultiStringSearch.h b/dbms/src/Functions/FunctionsMultiStringSearch.h new file mode 100644 index 00000000000..ccaad0eaa9a --- /dev/null +++ b/dbms/src/Functions/FunctionsMultiStringSearch.h @@ -0,0 +1,127 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** + * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 + * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + + * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found + * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + + +/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. +/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions +template ::max()> +class FunctionsMultiStringSearch : public IFunction +{ + static_assert(LimitArgs > 0); + +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context & context) + { + if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) + throw Exception( + "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); + if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return Impl::getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + using ResultType = typename Impl::ResultType; + + const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; + + const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + + const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; + const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); + + if (!col_const_arr) + throw Exception( + "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", + ErrorCodes::ILLEGAL_COLUMN); + + Array src_arr = col_const_arr->getValue(); + + if (src_arr.size() > LimitArgs) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + + ", should be at most " + std::to_string(LimitArgs), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector refs; + refs.reserve(src_arr.size()); + + for (const auto & el : src_arr) + refs.emplace_back(el.get()); + + auto col_res = ColumnVector::create(); + auto col_offsets = ColumnArray::ColumnOffsets::create(); + + auto & vec_res = col_res->getData(); + auto & offsets_res = col_offsets->getData(); + + /// The blame for resizing output is for the callee. + if (col_haystack_vector) + Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res); + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); + + if constexpr (Impl::is_column_array) + block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); + else + block.getByPosition(result).column = std::move(col_res); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsStringRegex.cpp b/dbms/src/Functions/FunctionsStringRegex.cpp index 40a42fd2643..856852ae197 100644 --- a/dbms/src/Functions/FunctionsStringRegex.cpp +++ b/dbms/src/Functions/FunctionsStringRegex.cpp @@ -1,5 +1,7 @@ #include "FunctionsStringRegex.h" #include "FunctionsStringSearch.h" +#include "FunctionsMultiStringSearch.h" +#include "FunctionsStringSearchToString.h" #include #include #include diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp deleted file mode 100644 index 8279ded5f81..00000000000 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ /dev/null @@ -1,707 +0,0 @@ -#include "FunctionsStringSearch.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} -/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness. - */ -struct PositionCaseSensitiveASCII -{ - /// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization. - using SearcherInBigHaystack = Volnitsky; - - /// For search many substrings in one string - using MultiSearcherInBigHaystack = MultiVolnitsky; - - /// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization. - using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - /// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8). - static size_t countChars(const char * begin, const char * end) { return end - begin; } - - /// Convert string to lowercase. Only for case-insensitive search. - /// Implementation is permitted to be inefficient because it is called for single string. - static void toLowerIfNeed(std::string &) { } -}; - -struct PositionCaseInsensitiveASCII -{ - /// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it. - using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher; - using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive; - using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/) - { - return SearcherInBigHaystack(needle_data, needle_size); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) { return end - begin; } - - static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } -}; - -struct PositionCaseSensitiveUTF8 -{ - using SearcherInBigHaystack = VolnitskyUTF8; - using MultiSearcherInBigHaystack = MultiVolnitskyUTF8; - using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) - { - size_t res = 0; - for (auto it = begin; it != end; ++it) - if (!UTF8::isContinuationOctet(static_cast(*it))) - ++res; - return res; - } - - static void toLowerIfNeed(std::string &) { } -}; - -struct PositionCaseInsensitiveUTF8 -{ - using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8; - using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8; - using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal. - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) - { - size_t res = 0; - for (auto it = begin; it != end; ++it) - if (!UTF8::isContinuationOctet(static_cast(*it))) - ++res; - return res; - } - - static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); } -}; - -template -struct PositionImpl -{ - static constexpr bool use_default_implementation_for_constants = false; - - using ResultType = UInt64; - - /// Find one substring in many strings. - static void vectorConstant( - const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray & res) - { - const UInt8 * begin = data.data(); - const UInt8 * pos = begin; - const UInt8 * end = pos + data.size(); - - /// Current index in the array of strings. - size_t i = 0; - - typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos); - - /// We will search for the next occurrence in all strings at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Determine which index it refers to. - while (begin + offsets[i] <= pos) - { - res[i] = 0; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + needle.size() < begin + offsets[i]) - res[i] = 1 + Impl::countChars(reinterpret_cast(begin + offsets[i - 1]), reinterpret_cast(pos)); - else - res[i] = 0; - - pos = begin + offsets[i]; - ++i; - } - - if (i < res.size()) - memset(&res[i], 0, (res.size() - i) * sizeof(res[0])); - } - - /// Search for substring in string. - static void constantConstant(std::string data, std::string needle, UInt64 & res) - { - Impl::toLowerIfNeed(data); - Impl::toLowerIfNeed(needle); - - res = data.find(needle); - if (res == std::string::npos) - res = 0; - else - res = 1 + Impl::countChars(data.data(), data.data() + res); - } - - /// Search each time for a different single substring inside each time different string. - static void vectorVector( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - PaddedPODArray & res) - { - ColumnString::Offset prev_haystack_offset = 0; - ColumnString::Offset prev_needle_offset = 0; - - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - - if (0 == needle_size) - { - /// An empty string is always at the very beginning of `haystack`. - res[i] = 1; - } - else - { - /// It is assumed that the StringSearcher is not very difficult to initialize. - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( - reinterpret_cast(&needle_data[prev_needle_offset]), - needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end - - /// searcher returns a pointer to the found substring or to the end of `haystack`. - size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1]) - - &haystack_data[prev_haystack_offset]; - - if (pos != haystack_size) - { - res[i] = 1 - + Impl::countChars( - reinterpret_cast(&haystack_data[prev_haystack_offset]), - reinterpret_cast(&haystack_data[prev_haystack_offset + pos])); - } - else - res[i] = 0; - } - - prev_haystack_offset = haystack_offsets[i]; - prev_needle_offset = needle_offsets[i]; - } - } - - /// Find many substrings in single string. - static void constantVector( - const String & haystack, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - PaddedPODArray & res) - { - // NOTE You could use haystack indexing. But this is a rare case. - - ColumnString::Offset prev_needle_offset = 0; - - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - - if (0 == needle_size) - { - res[i] = 1; - } - else - { - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( - reinterpret_cast(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1); - - size_t pos = searcher.search( - reinterpret_cast(haystack.data()), - reinterpret_cast(haystack.data()) + haystack.size()) - - reinterpret_cast(haystack.data()); - - if (pos != haystack.size()) - { - res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos); - } - else - res[i] = 0; - } - - prev_needle_offset = needle_offsets[i]; - } - } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template -struct MultiSearchAllPositionsImpl -{ - using ResultType = UInt64; - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res) - { - auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 - { - return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); - }; - - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - - const size_t haystack_string_size = haystack_offsets.size(); - const size_t needles_size = needles.size(); - - /// Something can be uninitialized after the search itself - std::fill(res.begin(), res.end(), 0); - - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback); - prev_offset = haystack_offsets[j]; - } - } - } -}; - -template -struct MultiSearchImpl -{ - using ResultType = UInt8; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - if (iteration == 0 || !res[j]) - res[j] = searcher.searchOne(haystack, haystack_end); - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -template -struct MultiSearchFirstPositionImpl -{ - using ResultType = UInt64; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 - { - return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); - }; - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - if (iteration == 0 || res[j] == 0) - res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); - else - { - UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); - if (result != 0) - res[j] = std::min(result, res[j]); - } - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -template -struct MultiSearchFirstIndexImpl -{ - using ResultType = UInt64; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - /// hasMoreToSearch traverse needles in increasing order - if (iteration == 0 || res[j] == 0) - res[j] = searcher.searchOneFirstIndex(haystack, haystack_end); - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. - */ -template -struct HasTokenImpl -{ - using ResultType = UInt8; - - static constexpr bool use_default_implementation_for_constants = true; - - static void vectorConstant( - const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) - { - if (offsets.empty()) - return; - - const UInt8 * begin = data.data(); - const UInt8 * pos = begin; - const UInt8 * end = pos + data.size(); - - /// The current index in the array of strings. - size_t i = 0; - - TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (begin + offsets[i] <= pos) - { - res[i] = negate_result; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + pattern.size() < begin + offsets[i]) - res[i] = !negate_result; - else - res[i] = negate_result; - - pos = begin + offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); - } - - template - static void vectorVector(Args &&...) - { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); - } - - /// Search different needles in single haystack. - template - static void constantVector(Args &&...) - { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); - } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -struct NamePosition -{ - static constexpr auto name = "position"; -}; -struct NamePositionUTF8 -{ - static constexpr auto name = "positionUTF8"; -}; -struct NamePositionCaseInsensitive -{ - static constexpr auto name = "positionCaseInsensitive"; -}; -struct NamePositionCaseInsensitiveUTF8 -{ - static constexpr auto name = "positionCaseInsensitiveUTF8"; -}; -struct NameMultiSearchAllPositions -{ - static constexpr auto name = "multiSearchAllPositions"; -}; -struct NameMultiSearchAllPositionsUTF8 -{ - static constexpr auto name = "multiSearchAllPositionsUTF8"; -}; -struct NameMultiSearchAllPositionsCaseInsensitive -{ - static constexpr auto name = "multiSearchAllPositionsCaseInsensitive"; -}; -struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8"; -}; -struct NameMultiSearchAny -{ - static constexpr auto name = "multiSearchAny"; -}; -struct NameMultiSearchAnyUTF8 -{ - static constexpr auto name = "multiSearchAnyUTF8"; -}; -struct NameMultiSearchAnyCaseInsensitive -{ - static constexpr auto name = "multiSearchAnyCaseInsensitive"; -}; -struct NameMultiSearchAnyCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8"; -}; -struct NameMultiSearchFirstIndex -{ - static constexpr auto name = "multiSearchFirstIndex"; -}; -struct NameMultiSearchFirstIndexUTF8 -{ - static constexpr auto name = "multiSearchFirstIndexUTF8"; -}; -struct NameMultiSearchFirstIndexCaseInsensitive -{ - static constexpr auto name = "multiSearchFirstIndexCaseInsensitive"; -}; -struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8"; -}; -struct NameMultiSearchFirstPosition -{ - static constexpr auto name = "multiSearchFirstPosition"; -}; -struct NameMultiSearchFirstPositionUTF8 -{ - static constexpr auto name = "multiSearchFirstPositionUTF8"; -}; -struct NameMultiSearchFirstPositionCaseInsensitive -{ - static constexpr auto name = "multiSearchFirstPositionCaseInsensitive"; -}; -struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; -}; - -struct NameHasToken -{ - static constexpr auto name = "hasToken"; -}; - -struct NameHasTokenCaseInsensitive -{ - static constexpr auto name = "hasTokenCaseInsensitive"; -}; - - -using FunctionPosition = FunctionsStringSearch, NamePosition>; -using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; -using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; -using FunctionPositionCaseInsensitiveUTF8 - = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; - -using FunctionMultiSearchAllPositions - = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; -using FunctionMultiSearchAllPositionsUTF8 - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; -using FunctionMultiSearchAllPositionsCaseInsensitive - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; -using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition< - MultiSearchAllPositionsImpl, - NameMultiSearchAllPositionsCaseInsensitiveUTF8>; - -using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; -using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; -using FunctionMultiSearchCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; -using FunctionMultiSearchCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; - -using FunctionMultiSearchFirstIndex - = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; -using FunctionMultiSearchFirstIndexUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; -using FunctionMultiSearchFirstIndexCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; -using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; - -using FunctionMultiSearchFirstPosition - = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; -using FunctionMultiSearchFirstPositionUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; -using FunctionMultiSearchFirstPositionCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; -using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch< - MultiSearchFirstPositionImpl, - NameMultiSearchFirstPositionCaseInsensitiveUTF8>; - -using FunctionHasToken = FunctionsStringSearch, NameHasToken>; -using FunctionHasTokenCaseInsensitive - = FunctionsStringSearch, NameHasTokenCaseInsensitive>; - -void registerFunctionsStringSearch(FunctionFactory & factory) -{ - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - - factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); -} -} diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index 2ae5f3598a8..9f4360cf935 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -10,9 +9,7 @@ #include #include #include -#include #include -#include namespace DB @@ -37,40 +34,12 @@ namespace DB * - the zero subpattern (the match part, otherwise); * - if not match - an empty string. * extract(haystack, pattern) - * - * replaceOne(haystack, pattern, replacement) - replacing the pattern with the specified rules, only the first occurrence. - * replaceAll(haystack, pattern, replacement) - replacing the pattern with the specified rules, all occurrences. - * - * replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence. - * replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences. - * - * multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack - * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found - * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 - * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - - * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found - * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) */ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int FUNCTION_NOT_ALLOWED; } template @@ -163,224 +132,4 @@ public: } }; - -template -class FunctionsStringSearchToString : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isString(arguments[1])) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; - - const ColumnConst * col_needle = typeid_cast(&*column_needle); - if (!col_needle) - throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN); - - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - - ColumnString::Chars & vec_res = col_res->getChars(); - ColumnString::Offsets & offsets_res = col_res->getOffsets(); - Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); - - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template -class FunctionsMultiStringPosition : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(std::make_shared()); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - using ResultType = typename Impl::ResultType; - - const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; - - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - - const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - - if (!col_const_arr) - throw Exception( - "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", - ErrorCodes::ILLEGAL_COLUMN); - - Array src_arr = col_const_arr->getValue(); - - if (src_arr.size() > std::numeric_limits::max()) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) - + ", should be at most 255", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - std::vector refs; - for (const auto & el : src_arr) - refs.emplace_back(el.get()); - - const size_t column_haystack_size = column_haystack->size(); - - auto col_res = ColumnVector::create(); - auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); - - auto & vec_res = col_res->getData(); - auto & offsets_res = col_offsets->getData(); - - vec_res.resize(column_haystack_size * refs.size()); - - if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); - - size_t refs_size = refs.size(); - size_t accum = refs_size; - - for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) - offsets_res[i] = accum; - - block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); - } -}; - -/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. -/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions -template ::max()> -class FunctionsMultiStringSearch : public IFunction -{ - static_assert(LimitArgs > 0); - -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context & context) - { - if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) - throw Exception( - "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return Impl::getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - using ResultType = typename Impl::ResultType; - - const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; - - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - - const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - - if (!col_const_arr) - throw Exception( - "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", - ErrorCodes::ILLEGAL_COLUMN); - - Array src_arr = col_const_arr->getValue(); - - if (src_arr.size() > LimitArgs) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) - + ", should be at most " + std::to_string(LimitArgs), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - std::vector refs; - refs.reserve(src_arr.size()); - - for (const auto & el : src_arr) - refs.emplace_back(el.get()); - - auto col_res = ColumnVector::create(); - auto col_offsets = ColumnArray::ColumnOffsets::create(); - - auto & vec_res = col_res->getData(); - auto & offsets_res = col_offsets->getData(); - - /// The blame for resizing output is for the callee. - if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res); - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); - - if constexpr (Impl::is_column_array) - block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); - else - block.getByPosition(result).column = std::move(col_res); - } -}; - } diff --git a/dbms/src/Functions/FunctionsStringSearchToString.h b/dbms/src/Functions/FunctionsStringSearchToString.h new file mode 100644 index 00000000000..b28b129f651 --- /dev/null +++ b/dbms/src/Functions/FunctionsStringSearchToString.h @@ -0,0 +1,87 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** Applies regexp re2 and extracts: + * - the first subpattern, if the regexp has a subpattern; + * - the zero subpattern (the match part, otherwise); + * - if not match - an empty string. + * extract(haystack, pattern) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + + +template +class FunctionsStringSearchToString : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isString(arguments[1])) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; + + const ColumnConst * col_needle = typeid_cast(&*column_needle); + if (!col_needle) + throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN); + + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + + ColumnString::Chars & vec_res = col_res->getChars(); + ColumnString::Offsets & offsets_res = col_res->getOffsets(); + Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); + + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/HasTokenImpl.h b/dbms/src/Functions/HasTokenImpl.h new file mode 100644 index 00000000000..b8f250be69f --- /dev/null +++ b/dbms/src/Functions/HasTokenImpl.h @@ -0,0 +1,83 @@ +#pragma once + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. + */ +template +struct HasTokenImpl +{ + using ResultType = UInt8; + + static constexpr bool use_default_implementation_for_constants = true; + + static void vectorConstant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) + { + if (offsets.empty()) + return; + + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// The current index in the array of strings. + size_t i = 0; + + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = negate_result; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + offsets[i]) + res[i] = !negate_result; + else + res[i] = negate_result; + + pos = begin + offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); + } + + template + static void vectorVector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + /// Search different needles in single haystack. + template + static void constantVector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + template + static void vectorFixedConstant(Args &&...) + { + throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchAllPositionsImpl.h b/dbms/src/Functions/MultiSearchAllPositionsImpl.h new file mode 100644 index 00000000000..f54fe41f20c --- /dev/null +++ b/dbms/src/Functions/MultiSearchAllPositionsImpl.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchAllPositionsImpl +{ + using ResultType = UInt64; + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res) + { + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + + const size_t haystack_string_size = haystack_offsets.size(); + const size_t needles_size = needles.size(); + + /// Something can be uninitialized after the search itself + std::fill(res.begin(), res.end(), 0); + + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback); + prev_offset = haystack_offsets[j]; + } + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchFirstIndexImpl.h b/dbms/src/Functions/MultiSearchFirstIndexImpl.h new file mode 100644 index 00000000000..bb0bb57b9db --- /dev/null +++ b/dbms/src/Functions/MultiSearchFirstIndexImpl.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchFirstIndexImpl +{ + using ResultType = UInt64; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + /// hasMoreToSearch traverse needles in increasing order + if (iteration == 0 || res[j] == 0) + res[j] = searcher.searchOneFirstIndex(haystack, haystack_end); + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchFirstPositionImpl.h b/dbms/src/Functions/MultiSearchFirstPositionImpl.h new file mode 100644 index 00000000000..4743518e61e --- /dev/null +++ b/dbms/src/Functions/MultiSearchFirstPositionImpl.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchFirstPositionImpl +{ + using ResultType = UInt64; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + if (iteration == 0 || res[j] == 0) + res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + else + { + UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + if (result != 0) + res[j] = std::min(result, res[j]); + } + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchImpl.h b/dbms/src/Functions/MultiSearchImpl.h new file mode 100644 index 00000000000..5b881e2fd55 --- /dev/null +++ b/dbms/src/Functions/MultiSearchImpl.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchImpl +{ + using ResultType = UInt8; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + if (iteration == 0 || !res[j]) + res[j] = searcher.searchOne(haystack, haystack_end); + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/PositionImpl.h b/dbms/src/Functions/PositionImpl.h new file mode 100644 index 00000000000..bc0b2c7bcfb --- /dev/null +++ b/dbms/src/Functions/PositionImpl.h @@ -0,0 +1,307 @@ +#include "FunctionsStringSearch.h" + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness. + */ +struct PositionCaseSensitiveASCII +{ + /// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization. + using SearcherInBigHaystack = Volnitsky; + + /// For search many substrings in one string + using MultiSearcherInBigHaystack = MultiVolnitsky; + + /// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization. + using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + /// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8). + static size_t countChars(const char * begin, const char * end) { return end - begin; } + + /// Convert string to lowercase. Only for case-insensitive search. + /// Implementation is permitted to be inefficient because it is called for single string. + static void toLowerIfNeed(std::string &) { } +}; + + +struct PositionCaseInsensitiveASCII +{ + /// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it. + using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher; + using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive; + using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/) + { + return SearcherInBigHaystack(needle_data, needle_size); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) { return end - begin; } + + static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + + +struct PositionCaseSensitiveUTF8 +{ + using SearcherInBigHaystack = VolnitskyUTF8; + using MultiSearcherInBigHaystack = MultiVolnitskyUTF8; + using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) + { + size_t res = 0; + for (auto it = begin; it != end; ++it) + if (!UTF8::isContinuationOctet(static_cast(*it))) + ++res; + return res; + } + + static void toLowerIfNeed(std::string &) { } +}; + + +struct PositionCaseInsensitiveUTF8 +{ + using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8; + using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8; + using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal. + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) + { + size_t res = 0; + for (auto it = begin; it != end; ++it) + if (!UTF8::isContinuationOctet(static_cast(*it))) + ++res; + return res; + } + + static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); } +}; + + +template +struct PositionImpl +{ + static constexpr bool use_default_implementation_for_constants = false; + + using ResultType = UInt64; + + /// Find one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray & res) + { + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// Current index in the array of strings. + size_t i = 0; + + typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos); + + /// We will search for the next occurrence in all strings at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = 0; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + needle.size() < begin + offsets[i]) + res[i] = 1 + Impl::countChars(reinterpret_cast(begin + offsets[i - 1]), reinterpret_cast(pos)); + else + res[i] = 0; + + pos = begin + offsets[i]; + ++i; + } + + if (i < res.size()) + memset(&res[i], 0, (res.size() - i) * sizeof(res[0])); + } + + /// Search for substring in string. + static void constantConstant(std::string data, std::string needle, UInt64 & res) + { + Impl::toLowerIfNeed(data); + Impl::toLowerIfNeed(needle); + + res = data.find(needle); + if (res == std::string::npos) + res = 0; + else + res = 1 + Impl::countChars(data.data(), data.data() + res); + } + + /// Search each time for a different single substring inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + PaddedPODArray & res) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + if (0 == needle_size) + { + /// An empty string is always at the very beginning of `haystack`. + res[i] = 1; + } + else + { + /// It is assumed that the StringSearcher is not very difficult to initialize. + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( + reinterpret_cast(&needle_data[prev_needle_offset]), + needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end + + /// searcher returns a pointer to the found substring or to the end of `haystack`. + size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1]) + - &haystack_data[prev_haystack_offset]; + + if (pos != haystack_size) + { + res[i] = 1 + + Impl::countChars( + reinterpret_cast(&haystack_data[prev_haystack_offset]), + reinterpret_cast(&haystack_data[prev_haystack_offset + pos])); + } + else + res[i] = 0; + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Find many substrings in single string. + static void constantVector( + const String & haystack, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + PaddedPODArray & res) + { + // NOTE You could use haystack indexing. But this is a rare case. + + ColumnString::Offset prev_needle_offset = 0; + + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( + reinterpret_cast(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1); + + size_t pos = searcher.search( + reinterpret_cast(haystack.data()), + reinterpret_cast(haystack.data()) + haystack.size()) + - reinterpret_cast(haystack.data()); + + if (pos != haystack.size()) + { + res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos); + } + else + res[i] = 0; + } + + prev_needle_offset = needle_offsets[i]; + } + } + + template + static void vectorFixedConstant(Args &&...) + { + throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/URL/cutURLParameter.cpp b/dbms/src/Functions/URL/cutURLParameter.cpp index b8f5c84fe83..f0103f42acd 100644 --- a/dbms/src/Functions/URL/cutURLParameter.cpp +++ b/dbms/src/Functions/URL/cutURLParameter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/dbms/src/Functions/URL/extractURLParameter.cpp b/dbms/src/Functions/URL/extractURLParameter.cpp index 8c5292bdc0a..c6234c66fc0 100644 --- a/dbms/src/Functions/URL/extractURLParameter.cpp +++ b/dbms/src/Functions/URL/extractURLParameter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/dbms/src/Functions/hasToken.cpp b/dbms/src/Functions/hasToken.cpp new file mode 100644 index 00000000000..ee04484ad54 --- /dev/null +++ b/dbms/src/Functions/hasToken.cpp @@ -0,0 +1,22 @@ +#include "FunctionsStringSearch.h" +#include +#include "HasTokenImpl.h" +#include + + +namespace DB +{ + +struct NameHasToken +{ + static constexpr auto name = "hasToken"; +}; + +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; + +void registerFunctionHasToken(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasTokenCaseInsensitive.cpp b/dbms/src/Functions/hasTokenCaseInsensitive.cpp new file mode 100644 index 00000000000..c58df05d239 --- /dev/null +++ b/dbms/src/Functions/hasTokenCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsStringSearch.h" +#include +#include "HasTokenImpl.h" +#include + + +namespace DB +{ + +struct NameHasTokenCaseInsensitive +{ + static constexpr auto name = "hasTokenCaseInsensitive"; +}; + +using FunctionHasTokenCaseInsensitive + = FunctionsStringSearch, NameHasTokenCaseInsensitive>; + +void registerFunctionHasTokenCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositions.cpp b/dbms/src/Functions/multiSearchAllPositions.cpp new file mode 100644 index 00000000000..c7aeb4d6245 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositions.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositions +{ + static constexpr auto name = "multiSearchAllPositions"; +}; + +using FunctionMultiSearchAllPositions + = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; + +void registerFunctionMultiSearchAllPositions(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp new file mode 100644 index 00000000000..4abcf7c8405 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsCaseInsensitive +{ + static constexpr auto name = "multiSearchAllPositionsCaseInsensitive"; +}; + +using FunctionMultiSearchAllPositionsCaseInsensitive + = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; + +void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..d9dbc1a7c8c --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp @@ -0,0 +1,24 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition< + MultiSearchAllPositionsImpl, + NameMultiSearchAllPositionsCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp b/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp new file mode 100644 index 00000000000..8f39c0eade9 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsUTF8 +{ + static constexpr auto name = "multiSearchAllPositionsUTF8"; +}; + +using FunctionMultiSearchAllPositionsUTF8 + = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; + +void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAny.cpp b/dbms/src/Functions/multiSearchAny.cpp new file mode 100644 index 00000000000..144dbdbfdc4 --- /dev/null +++ b/dbms/src/Functions/multiSearchAny.cpp @@ -0,0 +1,22 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAny +{ + static constexpr auto name = "multiSearchAny"; +}; + +using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; + +void registerFunctionMultiSearchAny(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp b/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp new file mode 100644 index 00000000000..8b33a61013b --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp @@ -0,0 +1,22 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyCaseInsensitive +{ + static constexpr auto name = "multiSearchAnyCaseInsensitive"; +}; +using FunctionMultiSearchCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; + +void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..49a8b95a0e0 --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchCaseInsensitiveUTF8 + = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyUTF8.cpp b/dbms/src/Functions/multiSearchAnyUTF8.cpp new file mode 100644 index 00000000000..55f2e449833 --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyUTF8.cpp @@ -0,0 +1,21 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyUTF8 +{ + static constexpr auto name = "multiSearchAnyUTF8"; +}; +using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; + +void registerFunctionMultiSearchAnyUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndex.cpp b/dbms/src/Functions/multiSearchFirstIndex.cpp new file mode 100644 index 00000000000..65a25004964 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndex.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndex +{ + static constexpr auto name = "multiSearchFirstIndex"; +}; + +using FunctionMultiSearchFirstIndex + = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; + +void registerFunctionMultiSearchFirstIndex(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp new file mode 100644 index 00000000000..bf643f0cf29 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexCaseInsensitive +{ + static constexpr auto name = "multiSearchFirstIndexCaseInsensitive"; +}; + +using FunctionMultiSearchFirstIndexCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; + +void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..005152388ab --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp b/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp new file mode 100644 index 00000000000..1158fda2a63 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexUTF8 +{ + static constexpr auto name = "multiSearchFirstIndexUTF8"; +}; + +using FunctionMultiSearchFirstIndexUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; + +void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPosition.cpp b/dbms/src/Functions/multiSearchFirstPosition.cpp new file mode 100644 index 00000000000..06ac396250e --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPosition.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPosition +{ + static constexpr auto name = "multiSearchFirstPosition"; +}; + +using FunctionMultiSearchFirstPosition + = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; + +void registerFunctionMultiSearchFirstPosition(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp new file mode 100644 index 00000000000..1d028ad4513 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionCaseInsensitive +{ + static constexpr auto name = "multiSearchFirstPositionCaseInsensitive"; +}; + +using FunctionMultiSearchFirstPositionCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; + +void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..0b355ddc446 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp @@ -0,0 +1,24 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch< + MultiSearchFirstPositionImpl, + NameMultiSearchFirstPositionCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp b/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp new file mode 100644 index 00000000000..26e16d17e8d --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionUTF8 +{ + static constexpr auto name = "multiSearchFirstPositionUTF8"; +}; + +using FunctionMultiSearchFirstPositionUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; + +void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/position.cpp b/dbms/src/Functions/position.cpp new file mode 100644 index 00000000000..df1405ae94a --- /dev/null +++ b/dbms/src/Functions/position.cpp @@ -0,0 +1,21 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePosition +{ + static constexpr auto name = "position"; +}; + +using FunctionPosition = FunctionsStringSearch, NamePosition>; + +void registerFunctionPosition(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); +} +} diff --git a/dbms/src/Functions/positionCaseInsensitive.cpp b/dbms/src/Functions/positionCaseInsensitive.cpp new file mode 100644 index 00000000000..00721dda212 --- /dev/null +++ b/dbms/src/Functions/positionCaseInsensitive.cpp @@ -0,0 +1,20 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionCaseInsensitive +{ + static constexpr auto name = "positionCaseInsensitive"; +}; + +using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; + +void registerFunctionPositionCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp b/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..196a5d67cd2 --- /dev/null +++ b/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp @@ -0,0 +1,22 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionCaseInsensitiveUTF8 +{ + static constexpr auto name = "positionCaseInsensitiveUTF8"; +}; + +using FunctionPositionCaseInsensitiveUTF8 + = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; + +void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/positionUTF8.cpp b/dbms/src/Functions/positionUTF8.cpp new file mode 100644 index 00000000000..944b55005f3 --- /dev/null +++ b/dbms/src/Functions/positionUTF8.cpp @@ -0,0 +1,21 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionUTF8 +{ + static constexpr auto name = "positionUTF8"; +}; + +using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; + +void registerFunctionPositionUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsStringSearch.cpp b/dbms/src/Functions/registerFunctionsStringSearch.cpp new file mode 100644 index 00000000000..e4f89201491 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsStringSearch.cpp @@ -0,0 +1,66 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionPosition(FunctionFactory &); +void registerFunctionPositionUTF8(FunctionFactory &); +void registerFunctionPositionCaseInsensitive(FunctionFactory &); +void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchAny(FunctionFactory &); +void registerFunctionMultiSearchAnyUTF8(FunctionFactory &); +void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchFirstIndex(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchFirstPosition(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchAllPositions(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionHasToken(FunctionFactory &); +void registerFunctionHasTokenCaseInsensitive(FunctionFactory &); + + +void registerFunctionsStringSearch(FunctionFactory & factory) +{ + registerFunctionPosition(factory); + registerFunctionPositionUTF8(factory); + registerFunctionPositionCaseInsensitive(factory); + registerFunctionPositionCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchAny(factory); + registerFunctionMultiSearchAnyUTF8(factory); + registerFunctionMultiSearchAnyCaseInsensitive(factory); + registerFunctionMultiSearchAnyCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchFirstIndex(factory); + registerFunctionMultiSearchFirstIndexUTF8(factory); + registerFunctionMultiSearchFirstIndexCaseInsensitive(factory); + registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchFirstPosition(factory); + registerFunctionMultiSearchFirstPositionUTF8(factory); + registerFunctionMultiSearchFirstPositionCaseInsensitive(factory); + registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchAllPositions(factory); + registerFunctionMultiSearchAllPositionsUTF8(factory); + registerFunctionMultiSearchAllPositionsCaseInsensitive(factory); + registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(factory); + + registerFunctionHasToken(factory); + registerFunctionHasTokenCaseInsensitive(factory); +} + +} diff --git a/dbms/src/Functions/visitParamExtractRaw.cpp b/dbms/src/Functions/visitParamExtractRaw.cpp index 5eeb36286a5..e6e89f1c7ba 100644 --- a/dbms/src/Functions/visitParamExtractRaw.cpp +++ b/dbms/src/Functions/visitParamExtractRaw.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Functions/visitParamExtractString.cpp b/dbms/src/Functions/visitParamExtractString.cpp index a6f4b98145d..b633a59807e 100644 --- a/dbms/src/Functions/visitParamExtractString.cpp +++ b/dbms/src/Functions/visitParamExtractString.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB