2019-03-25 21:31:00 +00:00
|
|
|
#include "FunctionsStringSearch.h"
|
2017-03-10 17:52:36 +00:00
|
|
|
|
2019-03-29 01:02:05 +00:00
|
|
|
#include <algorithm>
|
|
|
|
#include <memory>
|
|
|
|
#include <string>
|
|
|
|
#include <vector>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnFixedString.h>
|
|
|
|
#include <DataTypes/DataTypeFixedString.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/Regexps.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2019-01-14 15:54:47 +00:00
|
|
|
#include <Poco/UTF8String.h>
|
|
|
|
#include <Common/Volnitsky.h>
|
2017-03-10 17:52:36 +00:00
|
|
|
|
2016-06-07 08:23:15 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2017-03-10 17:52:36 +00:00
|
|
|
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
|
|
|
|
*/
|
|
|
|
struct PositionCaseSensitiveASCII
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
|
2019-01-14 15:54:47 +00:00
|
|
|
using SearcherInBigHaystack = Volnitsky;
|
|
|
|
|
|
|
|
/// For search many substrings in one string
|
|
|
|
using MultiSearcherInBigHaystack = MultiVolnitsky;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// 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);
|
|
|
|
}
|
|
|
|
|
2019-01-16 17:41:08 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-14 15:54:47 +00:00
|
|
|
return MultiSearcherInBigHaystack(needles);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
/// 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; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Convert string to lowercase. Only for case-insensitive search.
|
|
|
|
/// Implementation is permitted to be inefficient because it is called for single string.
|
2019-01-14 15:54:47 +00:00
|
|
|
static void toLowerIfNeed(std::string &) {}
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
struct PositionCaseInsensitiveASCII
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// `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;
|
2019-01-14 15:54:47 +00:00
|
|
|
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
|
2017-04-01 07:20:54 +00:00
|
|
|
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
|
|
|
|
|
2017-12-02 02:47:12 +00:00
|
|
|
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return SearcherInBigHaystack(needle_data, needle_size);
|
|
|
|
}
|
|
|
|
|
|
|
|
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
|
|
|
{
|
|
|
|
return SearcherInSmallHaystack(needle_data, needle_size);
|
|
|
|
}
|
|
|
|
|
2019-01-16 17:41:08 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-14 15:54:47 +00:00
|
|
|
return MultiSearcherInBigHaystack(needles);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
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); }
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
struct PositionCaseSensitiveUTF8
|
|
|
|
{
|
2019-01-14 15:54:47 +00:00
|
|
|
using SearcherInBigHaystack = VolnitskyUTF8;
|
|
|
|
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
|
2017-04-01 07:20:54 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2019-01-16 17:41:08 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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<UInt8>(*it)))
|
|
|
|
++res;
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
static void toLowerIfNeed(std::string &) {}
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
struct PositionCaseInsensitiveUTF8
|
|
|
|
{
|
2019-01-14 15:54:47 +00:00
|
|
|
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
|
|
|
|
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
|
2017-04-01 07:20:54 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2019-01-16 17:41:08 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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<UInt8>(*it)))
|
|
|
|
++res;
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Impl>
|
|
|
|
struct PositionImpl
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
using ResultType = UInt64;
|
|
|
|
|
|
|
|
/// Find one substring in many strings.
|
2019-01-14 15:54:47 +00:00
|
|
|
static void vector_constant(
|
|
|
|
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-02 03:00:04 +00:00
|
|
|
const UInt8 * begin = data.data();
|
2017-04-01 07:20:54 +00:00
|
|
|
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])
|
2019-03-30 09:39:45 +00:00
|
|
|
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
|
|
|
res[i] = 0;
|
|
|
|
|
|
|
|
pos = begin + offsets[i];
|
|
|
|
++i;
|
|
|
|
}
|
|
|
|
|
2018-09-02 03:52:04 +00:00
|
|
|
if (i < res.size())
|
|
|
|
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Search for substring in string.
|
|
|
|
static void constant_constant(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.
|
2019-01-14 15:54:47 +00:00
|
|
|
static void vector_vector(
|
|
|
|
const ColumnString::Chars & haystack_data,
|
2017-12-15 21:32:25 +00:00
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnString::Chars & needle_data,
|
2017-12-15 21:32:25 +00:00
|
|
|
const ColumnString::Offsets & needle_offsets,
|
2017-04-01 07:20:54 +00:00
|
|
|
PaddedPODArray<UInt64> & res)
|
|
|
|
{
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offset prev_haystack_offset = 0;
|
|
|
|
ColumnString::Offset prev_needle_offset = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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.
|
2019-01-14 15:54:47 +00:00
|
|
|
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
|
|
|
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
|
|
|
|
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// 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)
|
|
|
|
{
|
2019-01-14 15:54:47 +00:00
|
|
|
res[i] = 1
|
|
|
|
+ Impl::countChars(
|
|
|
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
|
|
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
res[i] = 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
prev_haystack_offset = haystack_offsets[i];
|
|
|
|
prev_needle_offset = needle_offsets[i];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-01-16 13:57:11 +00:00
|
|
|
/// Find many substrings in single string.
|
2019-01-14 15:54:47 +00:00
|
|
|
static void constant_vector(
|
|
|
|
const String & haystack,
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnString::Chars & needle_data,
|
2017-12-15 21:32:25 +00:00
|
|
|
const ColumnString::Offsets & needle_offsets,
|
2017-04-01 07:20:54 +00:00
|
|
|
PaddedPODArray<UInt64> & res)
|
|
|
|
{
|
|
|
|
// NOTE You could use haystack indexing. But this is a rare case.
|
|
|
|
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offset prev_needle_offset = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
size_t pos = searcher.search(
|
|
|
|
reinterpret_cast<const UInt8 *>(haystack.data()),
|
2017-04-01 07:20:54 +00:00
|
|
|
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
|
|
|
|
- reinterpret_cast<const UInt8 *>(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];
|
|
|
|
}
|
|
|
|
}
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
|
2019-01-14 15:54:47 +00:00
|
|
|
template <typename Impl>
|
2019-03-23 22:49:38 +00:00
|
|
|
struct MultiSearchAllPositionsImpl
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
|
|
|
using ResultType = UInt64;
|
|
|
|
|
2019-01-16 13:57:11 +00:00
|
|
|
static void vector_constant(
|
2019-01-14 15:54:47 +00:00
|
|
|
const ColumnString::Chars & haystack_data,
|
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
2019-01-16 17:41:08 +00:00
|
|
|
const std::vector<StringRef> & needles,
|
2019-01-14 15:54:47 +00:00
|
|
|
PaddedPODArray<UInt64> & res)
|
|
|
|
{
|
2019-03-30 09:39:45 +00:00
|
|
|
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
|
|
|
{
|
2019-01-16 17:41:08 +00:00
|
|
|
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
Impl::createMultiSearcherInBigHaystack(needles).searchAllPositions(haystack_data, haystack_offsets, res_callback, res);
|
2019-01-14 15:54:47 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Impl>
|
|
|
|
struct MultiSearchImpl
|
|
|
|
{
|
2019-03-23 19:40:16 +00:00
|
|
|
using ResultType = UInt8;
|
2019-03-26 21:56:46 +00:00
|
|
|
static constexpr bool is_using_hyperscan = false;
|
2019-01-14 15:54:47 +00:00
|
|
|
|
2019-01-16 13:57:11 +00:00
|
|
|
static void vector_constant(
|
2019-01-14 15:54:47 +00:00
|
|
|
const ColumnString::Chars & haystack_data,
|
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
2019-01-16 17:41:08 +00:00
|
|
|
const std::vector<StringRef> & needles,
|
2019-03-23 19:40:16 +00:00
|
|
|
PaddedPODArray<UInt8> & res)
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-01-16 17:41:08 +00:00
|
|
|
Impl::createMultiSearcherInBigHaystack(needles).search(haystack_data, haystack_offsets, res);
|
2019-01-14 15:54:47 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Impl>
|
2019-03-23 22:49:38 +00:00
|
|
|
struct MultiSearchFirstPositionImpl
|
|
|
|
{
|
|
|
|
using ResultType = UInt64;
|
2019-03-26 21:56:46 +00:00
|
|
|
static constexpr bool is_using_hyperscan = false;
|
2019-03-23 22:49:38 +00:00
|
|
|
|
|
|
|
static void vector_constant(
|
|
|
|
const ColumnString::Chars & haystack_data,
|
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
|
|
|
const std::vector<StringRef> & needles,
|
|
|
|
PaddedPODArray<UInt64> & res)
|
|
|
|
{
|
2019-03-30 09:39:45 +00:00
|
|
|
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
|
|
|
};
|
|
|
|
Impl::createMultiSearcherInBigHaystack(needles).searchFirstPosition(haystack_data, haystack_offsets, res_callback, res);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Impl>
|
|
|
|
struct MultiSearchFirstIndexImpl
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
|
|
|
using ResultType = UInt64;
|
2019-03-26 21:56:46 +00:00
|
|
|
static constexpr bool is_using_hyperscan = false;
|
2019-01-14 15:54:47 +00:00
|
|
|
|
2019-01-16 13:57:11 +00:00
|
|
|
static void vector_constant(
|
2019-01-14 15:54:47 +00:00
|
|
|
const ColumnString::Chars & haystack_data,
|
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
2019-01-16 17:41:08 +00:00
|
|
|
const std::vector<StringRef> & needles,
|
2019-01-14 15:54:47 +00:00
|
|
|
PaddedPODArray<UInt64> & res)
|
|
|
|
{
|
2019-01-16 17:41:08 +00:00
|
|
|
Impl::createMultiSearcherInBigHaystack(needles).searchIndex(haystack_data, haystack_offsets, res);
|
2019-01-14 15:54:47 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-03-10 17:52:36 +00:00
|
|
|
|
|
|
|
struct NamePosition
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "position";
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
struct NamePositionUTF8
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "positionUTF8";
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
struct NamePositionCaseInsensitive
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "positionCaseInsensitive";
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
|
|
|
struct NamePositionCaseInsensitiveUTF8
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "positionCaseInsensitiveUTF8";
|
2017-03-10 17:52:36 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAllPositions
|
|
|
|
{
|
|
|
|
static constexpr auto name = "multiSearchAllPositions";
|
|
|
|
};
|
|
|
|
struct NameMultiSearchAllPositionsUTF8
|
|
|
|
{
|
|
|
|
static constexpr auto name = "multiSearchAllPositionsUTF8";
|
|
|
|
};
|
|
|
|
struct NameMultiSearchAllPositionsCaseInsensitive
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAny
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAny";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAnyUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAnyUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAnyCaseInsensitive
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAnyCaseInsensitive";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchAnyCaseInsensitiveUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstIndex
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstIndex";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstIndexUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstIndexUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstIndexCaseInsensitive
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstPosition
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstPosition";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-23 22:49:38 +00:00
|
|
|
struct NameMultiSearchFirstPositionUTF8
|
2019-01-14 15:54:47 +00:00
|
|
|
{
|
2019-03-23 22:49:38 +00:00
|
|
|
static constexpr auto name = "multiSearchFirstPositionUTF8";
|
|
|
|
};
|
|
|
|
struct NameMultiSearchFirstPositionCaseInsensitive
|
|
|
|
{
|
|
|
|
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
|
|
|
|
};
|
|
|
|
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
|
|
|
|
{
|
|
|
|
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
|
2019-01-14 15:54:47 +00:00
|
|
|
};
|
2019-03-29 01:02:05 +00:00
|
|
|
|
2017-03-10 17:52:36 +00:00
|
|
|
|
|
|
|
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
|
|
|
|
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
|
|
|
|
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
|
|
|
|
using FunctionPositionCaseInsensitiveUTF8
|
2017-04-01 07:20:54 +00:00
|
|
|
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
|
2017-03-10 17:52:36 +00:00
|
|
|
|
2019-03-29 01:02:05 +00:00
|
|
|
using FunctionMultiSearchAllPositions
|
|
|
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
|
|
|
|
using FunctionMultiSearchAllPositionsUTF8
|
|
|
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
|
2019-03-23 22:49:38 +00:00
|
|
|
using FunctionMultiSearchAllPositionsCaseInsensitive
|
|
|
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
|
2019-03-29 01:02:05 +00:00
|
|
|
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
|
|
|
|
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
|
|
|
|
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
|
2019-01-14 15:54:47 +00:00
|
|
|
|
2019-03-23 22:49:38 +00:00
|
|
|
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
|
|
|
|
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
|
2019-01-14 15:54:47 +00:00
|
|
|
using FunctionMultiSearchCaseInsensitive
|
2019-03-23 22:49:38 +00:00
|
|
|
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
|
2019-01-14 15:54:47 +00:00
|
|
|
using FunctionMultiSearchCaseInsensitiveUTF8
|
2019-03-23 22:49:38 +00:00
|
|
|
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
|
|
|
|
|
2019-03-29 01:02:05 +00:00
|
|
|
using FunctionMultiSearchFirstIndex
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
|
|
|
|
using FunctionMultiSearchFirstIndexUTF8
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
|
2019-03-23 22:49:38 +00:00
|
|
|
using FunctionMultiSearchFirstIndexCaseInsensitive
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
|
|
|
|
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
|
|
|
|
|
2019-03-29 01:02:05 +00:00
|
|
|
using FunctionMultiSearchFirstPosition
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
|
|
|
|
using FunctionMultiSearchFirstPositionUTF8
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
|
2019-03-23 22:49:38 +00:00
|
|
|
using FunctionMultiSearchFirstPositionCaseInsensitive
|
|
|
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
|
2019-03-29 01:02:05 +00:00
|
|
|
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
|
|
|
|
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
|
|
|
|
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
|
2017-03-10 17:52:36 +00:00
|
|
|
|
2016-06-07 08:23:15 +00:00
|
|
|
|
|
|
|
void registerFunctionsStringSearch(FunctionFactory & factory)
|
|
|
|
{
|
2018-12-19 14:46:18 +00:00
|
|
|
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
|
2017-04-01 07:20:54 +00:00
|
|
|
factory.registerFunction<FunctionPositionUTF8>();
|
|
|
|
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
|
|
|
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
2019-01-14 15:54:47 +00:00
|
|
|
|
2019-03-23 22:49:38 +00:00
|
|
|
factory.registerFunction<FunctionMultiSearchAllPositions>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
|
2019-01-14 15:54:47 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionMultiSearch>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchUTF8>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
|
|
|
|
|
2019-03-23 22:49:38 +00:00
|
|
|
factory.registerFunction<FunctionMultiSearchFirstIndex>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
|
|
|
|
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstPosition>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
|
|
|
|
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
|
2019-01-14 15:54:47 +00:00
|
|
|
|
2018-12-06 13:36:23 +00:00
|
|
|
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
|
2016-06-07 08:23:15 +00:00
|
|
|
}
|
|
|
|
}
|