2020-10-10 18:37:02 +00:00
|
|
|
#pragma once
|
2020-03-29 17:04:16 +00:00
|
|
|
#include "FunctionsStringSearch.h"
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
#include <algorithm>
|
2020-03-29 17:04:16 +00:00
|
|
|
#include <string>
|
|
|
|
#include <vector>
|
|
|
|
#include <Poco/UTF8String.h>
|
|
|
|
#include <Common/Volnitsky.h>
|
|
|
|
|
|
|
|
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.
|
Fix countSubstrings() & position() on patterns with 0-bytes
SQL functions countSubstrings(), countSubstringsCaseInsensitive(),
countSubstringsUTF8(), position(), positionCaseInsensitive(),
positionUTF8() with non-const pattern argument use fallback sorters
LibCASCIICaseSensitiveStringSearcher and LibCASCIICaseInsensitiveStringSearcher
which call ::strstr(), resp. ::strcasestr(). These functions assume that
the haystack is 0-terminated and they even document that. However, the
callers did not check if the haystack contains 0-byte (perhaps because
its sort of expensive). As a consequence, if the haystack contained a
zero byte in it's payload, matches behind this zero byte were ignored.
create table t (id UInt32, pattern String) engine = MergeTree() order by id;
insert into t values (1, 'x');
select countSubstrings('aaaxxxaa\0xxx', pattern) from t;
We returned 3 before this commit, now we return 6
2022-06-29 15:08:16 +00:00
|
|
|
using SearcherInSmallHaystack = StdLibASCIIStringSearcher</*CaseInsensitive*/ false>;
|
2020-03-29 17:04:16 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2022-06-24 13:16:57 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<std::string_view> & needles)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
static const char * advancePos(const char * pos, const char * end, size_t n)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
return std::min(pos + n, end);
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +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; }
|
|
|
|
|
|
|
|
/// 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;
|
Fix countSubstrings() & position() on patterns with 0-bytes
SQL functions countSubstrings(), countSubstringsCaseInsensitive(),
countSubstringsUTF8(), position(), positionCaseInsensitive(),
positionUTF8() with non-const pattern argument use fallback sorters
LibCASCIICaseSensitiveStringSearcher and LibCASCIICaseInsensitiveStringSearcher
which call ::strstr(), resp. ::strcasestr(). These functions assume that
the haystack is 0-terminated and they even document that. However, the
callers did not check if the haystack contains 0-byte (perhaps because
its sort of expensive). As a consequence, if the haystack contained a
zero byte in it's payload, matches behind this zero byte were ignored.
create table t (id UInt32, pattern String) engine = MergeTree() order by id;
insert into t values (1, 'x');
select countSubstrings('aaaxxxaa\0xxx', pattern) from t;
We returned 3 before this commit, now we return 6
2022-06-29 15:08:16 +00:00
|
|
|
using SearcherInSmallHaystack = StdLibASCIIStringSearcher</*CaseInsensitive*/ true>;
|
2020-03-29 17:04:16 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2022-06-24 13:16:57 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<std::string_view> & needles)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
static const char * advancePos(const char * pos, const char * end, size_t n)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
return std::min(pos + n, end);
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +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); }
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
struct PositionCaseSensitiveUTF8
|
|
|
|
{
|
|
|
|
using SearcherInBigHaystack = VolnitskyUTF8;
|
|
|
|
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
|
Fix countSubstrings() & position() on patterns with 0-bytes
SQL functions countSubstrings(), countSubstringsCaseInsensitive(),
countSubstringsUTF8(), position(), positionCaseInsensitive(),
positionUTF8() with non-const pattern argument use fallback sorters
LibCASCIICaseSensitiveStringSearcher and LibCASCIICaseInsensitiveStringSearcher
which call ::strstr(), resp. ::strcasestr(). These functions assume that
the haystack is 0-terminated and they even document that. However, the
callers did not check if the haystack contains 0-byte (perhaps because
its sort of expensive). As a consequence, if the haystack contained a
zero byte in it's payload, matches behind this zero byte were ignored.
create table t (id UInt32, pattern String) engine = MergeTree() order by id;
insert into t values (1, 'x');
select countSubstrings('aaaxxxaa\0xxx', pattern) from t;
We returned 3 before this commit, now we return 6
2022-06-29 15:08:16 +00:00
|
|
|
using SearcherInSmallHaystack = StdLibASCIIStringSearcher</*CaseInsensitive*/ false>;
|
2020-03-29 17:04:16 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2022-06-24 13:16:57 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<std::string_view> & needles)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
static const char * advancePos(const char * pos, const char * end, size_t n)
|
|
|
|
{
|
2023-01-18 07:13:47 +00:00
|
|
|
for (const auto *it = pos; it != end; ++it)
|
2020-08-02 14:24:39 +00:00
|
|
|
{
|
|
|
|
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
|
|
|
{
|
|
|
|
if (n == 0)
|
2020-08-01 21:14:23 +00:00
|
|
|
return it;
|
|
|
|
n--;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return end;
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
static size_t countChars(const char * begin, const char * end)
|
|
|
|
{
|
|
|
|
size_t res = 0;
|
2023-01-18 07:13:47 +00:00
|
|
|
for (const auto *it = begin; it != end; ++it)
|
2020-03-29 17:04:16 +00:00
|
|
|
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*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);
|
|
|
|
}
|
|
|
|
|
2022-06-24 13:16:57 +00:00
|
|
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<std::string_view> & needles)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
|
|
|
return MultiSearcherInBigHaystack(needles);
|
|
|
|
}
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
static const char * advancePos(const char * pos, const char * end, size_t n)
|
|
|
|
{
|
|
|
|
// reuse implementation that doesn't depend on case
|
|
|
|
return PositionCaseSensitiveUTF8::advancePos(pos, end, n);
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
static size_t countChars(const char * begin, const char * end)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
// reuse implementation that doesn't depend on case
|
|
|
|
return PositionCaseSensitiveUTF8::countChars(begin, end);
|
2020-03-29 17:04:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2021-09-21 16:43:46 +00:00
|
|
|
template <typename Name, typename Impl>
|
2020-03-29 17:04:16 +00:00
|
|
|
struct PositionImpl
|
|
|
|
{
|
|
|
|
static constexpr bool use_default_implementation_for_constants = false;
|
2020-08-01 21:14:23 +00:00
|
|
|
static constexpr bool supports_start_pos = true;
|
2021-09-21 16:43:46 +00:00
|
|
|
static constexpr auto name = Name::name;
|
2020-03-29 17:04:16 +00:00
|
|
|
|
2022-05-16 20:23:51 +00:00
|
|
|
static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};}
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
using ResultType = UInt64;
|
|
|
|
|
|
|
|
/// Find one substring in many strings.
|
|
|
|
static void vectorConstant(
|
2022-05-13 08:52:25 +00:00
|
|
|
const ColumnString::Chars & haystack_data,
|
|
|
|
const ColumnString::Offsets & haystack_offsets,
|
2020-08-01 21:14:23 +00:00
|
|
|
const std::string & needle,
|
|
|
|
const ColumnPtr & start_pos,
|
2023-01-23 22:27:48 +00:00
|
|
|
PaddedPODArray<UInt64> & res,
|
|
|
|
[[maybe_unused]] ColumnUInt8 * res_null)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2023-01-23 22:27:48 +00:00
|
|
|
/// `res_null` serves as an output parameter for implementing an XYZOrNull variant.
|
|
|
|
assert(!res_null);
|
|
|
|
|
2022-05-13 08:52:25 +00:00
|
|
|
const UInt8 * const begin = haystack_data.data();
|
|
|
|
const UInt8 * const end = haystack_data.data() + haystack_data.size();
|
2020-03-29 17:04:16 +00:00
|
|
|
const UInt8 * pos = begin;
|
|
|
|
|
2023-01-31 10:25:18 +00:00
|
|
|
/// Fast path when needle is empty
|
2023-01-18 07:13:47 +00:00
|
|
|
if (needle.empty())
|
|
|
|
{
|
2023-01-31 10:25:18 +00:00
|
|
|
/// Needle is empty and start_pos doesn't exist --> always return 1
|
2023-01-18 07:13:47 +00:00
|
|
|
if (start_pos == nullptr)
|
|
|
|
{
|
2023-01-31 10:25:18 +00:00
|
|
|
for (auto & r : res)
|
|
|
|
r = 1;
|
2023-01-20 09:05:28 +00:00
|
|
|
return;
|
2023-01-18 07:13:47 +00:00
|
|
|
}
|
2023-01-20 09:05:28 +00:00
|
|
|
|
|
|
|
ColumnString::Offset prev_offset = 0;
|
2023-01-31 02:48:02 +00:00
|
|
|
size_t rows = haystack_offsets.size();
|
|
|
|
|
|
|
|
if (const ColumnConst * start_pos_const = typeid_cast<const ColumnConst *>(&*start_pos))
|
2023-01-18 07:13:47 +00:00
|
|
|
{
|
2023-01-31 10:25:18 +00:00
|
|
|
/// Needle is empty and start_pos is constant
|
|
|
|
UInt64 start = std::max(start_pos_const->getUInt(0), static_cast<UInt64>(1));
|
2023-01-20 09:05:28 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
2023-01-18 07:13:47 +00:00
|
|
|
{
|
|
|
|
size_t haystack_size = Impl::countChars(
|
|
|
|
reinterpret_cast<const char *>(pos), reinterpret_cast<const char *>(pos + haystack_offsets[i] - prev_offset - 1));
|
2023-01-31 10:25:18 +00:00
|
|
|
res[i] = (start <= haystack_size + 1) ? start : 0;
|
2023-01-18 07:13:47 +00:00
|
|
|
|
|
|
|
pos = begin + haystack_offsets[i];
|
|
|
|
prev_offset = haystack_offsets[i];
|
|
|
|
}
|
2023-01-20 09:05:28 +00:00
|
|
|
return;
|
|
|
|
}
|
2023-01-31 02:48:02 +00:00
|
|
|
else
|
2023-01-20 09:05:28 +00:00
|
|
|
{
|
2023-01-31 10:25:18 +00:00
|
|
|
/// Needle is empty and start_pos is not constant
|
2023-01-31 02:48:02 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
|
|
|
size_t haystack_size = Impl::countChars(
|
|
|
|
reinterpret_cast<const char *>(pos), reinterpret_cast<const char *>(pos + haystack_offsets[i] - prev_offset - 1));
|
|
|
|
UInt64 start = start_pos->getUInt(i);
|
2023-01-31 10:25:18 +00:00
|
|
|
start = std::max(static_cast<UInt64>(1), start);
|
|
|
|
res[i] = (start <= haystack_size + 1) ? start : 0;
|
2023-01-31 02:48:02 +00:00
|
|
|
|
|
|
|
pos = begin + haystack_offsets[i];
|
|
|
|
prev_offset = haystack_offsets[i];
|
|
|
|
}
|
|
|
|
return;
|
2023-01-18 07:13:47 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
/// 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.
|
2022-05-13 08:52:25 +00:00
|
|
|
while (begin + haystack_offsets[i] <= pos)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
|
|
|
res[i] = 0;
|
|
|
|
++i;
|
|
|
|
}
|
2020-08-01 21:14:23 +00:00
|
|
|
auto start = start_pos != nullptr ? start_pos->getUInt(i) : 0;
|
2020-03-29 17:04:16 +00:00
|
|
|
|
|
|
|
/// We check that the entry does not pass through the boundaries of strings.
|
2022-05-13 08:52:25 +00:00
|
|
|
if (pos + needle.size() < begin + haystack_offsets[i])
|
2020-08-02 14:24:39 +00:00
|
|
|
{
|
2022-05-13 08:52:25 +00:00
|
|
|
auto res_pos = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + haystack_offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
2020-08-02 14:24:39 +00:00
|
|
|
if (res_pos < start)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
pos = reinterpret_cast<const UInt8 *>(Impl::advancePos(
|
|
|
|
reinterpret_cast<const char *>(pos),
|
2022-05-13 08:52:25 +00:00
|
|
|
reinterpret_cast<const char *>(begin + haystack_offsets[i]),
|
2020-08-01 21:14:23 +00:00
|
|
|
start - res_pos));
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
res[i] = res_pos;
|
|
|
|
}
|
2020-08-02 14:24:39 +00:00
|
|
|
else
|
|
|
|
{
|
2020-03-29 17:04:16 +00:00
|
|
|
res[i] = 0;
|
2020-08-01 21:14:23 +00:00
|
|
|
}
|
2022-05-13 08:52:25 +00:00
|
|
|
pos = begin + haystack_offsets[i];
|
2020-03-29 17:04:16 +00:00
|
|
|
++i;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (i < res.size())
|
|
|
|
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Search for substring in string.
|
2020-08-01 21:14:23 +00:00
|
|
|
static void constantConstantScalar(
|
|
|
|
std::string data,
|
|
|
|
std::string needle,
|
|
|
|
UInt64 start_pos,
|
|
|
|
UInt64 & res)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2020-08-15 20:08:59 +00:00
|
|
|
auto start = std::max(start_pos, UInt64(1));
|
2020-08-01 21:14:23 +00:00
|
|
|
|
2023-01-18 07:13:47 +00:00
|
|
|
if (needle.empty())
|
2020-08-02 14:24:39 +00:00
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
size_t haystack_size = Impl::countChars(data.data(), data.data() + data.size());
|
|
|
|
res = start <= haystack_size + 1 ? start : 0;
|
|
|
|
return;
|
|
|
|
}
|
2020-03-29 17:04:16 +00:00
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
size_t start_byte = Impl::advancePos(data.data(), data.data() + data.size(), start - 1) - data.data();
|
|
|
|
res = data.find(needle, start_byte);
|
2020-03-29 17:04:16 +00:00
|
|
|
if (res == std::string::npos)
|
|
|
|
res = 0;
|
|
|
|
else
|
|
|
|
res = 1 + Impl::countChars(data.data(), data.data() + res);
|
|
|
|
}
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
/// Search for substring in string starting from different positions.
|
|
|
|
static void constantConstant(
|
|
|
|
std::string data,
|
|
|
|
std::string needle,
|
|
|
|
const ColumnPtr & start_pos,
|
2023-01-23 22:27:48 +00:00
|
|
|
PaddedPODArray<UInt64> & res,
|
|
|
|
[[maybe_unused]] ColumnUInt8 * res_null)
|
2020-08-01 21:14:23 +00:00
|
|
|
{
|
2023-01-23 22:27:48 +00:00
|
|
|
/// `res_null` serves as an output parameter for implementing an XYZOrNull variant.
|
|
|
|
assert(!res_null);
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
Impl::toLowerIfNeed(data);
|
|
|
|
Impl::toLowerIfNeed(needle);
|
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
if (start_pos == nullptr)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
constantConstantScalar(data, needle, 0, res[0]);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t haystack_size = Impl::countChars(data.data(), data.data() + data.size());
|
|
|
|
|
|
|
|
size_t size = start_pos != nullptr ? start_pos->size() : 0;
|
2020-08-02 14:24:39 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
auto start = start_pos->getUInt(i);
|
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
if (start > haystack_size + 1)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
res[i] = 0;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
constantConstantScalar(data, needle, start, res[i]);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
/// 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,
|
2020-08-01 21:14:23 +00:00
|
|
|
const ColumnPtr & start_pos,
|
2023-01-23 22:27:48 +00:00
|
|
|
PaddedPODArray<UInt64> & res,
|
|
|
|
[[maybe_unused]] ColumnUInt8 * res_null)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2023-01-23 22:27:48 +00:00
|
|
|
/// `res_null` serves as an output parameter for implementing an XYZOrNull variant.
|
|
|
|
assert(!res_null);
|
|
|
|
|
2020-03-29 17:04:16 +00:00
|
|
|
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;
|
|
|
|
|
2020-08-15 20:08:59 +00:00
|
|
|
auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1);
|
2020-08-01 21:14:23 +00:00
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
if (start > haystack_size + 1)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
res[i] = 0;
|
|
|
|
}
|
|
|
|
else if (0 == needle_size)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
/// An empty string is always at any position in `haystack`.
|
|
|
|
res[i] = start;
|
2020-03-29 17:04:16 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// It is assumed that the StringSearcher is not very difficult to initialize.
|
|
|
|
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
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
const char * beg = Impl::advancePos(
|
|
|
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
|
|
|
reinterpret_cast<const char *>(&haystack_data[haystack_offsets[i] - 1]),
|
|
|
|
start - 1);
|
2020-03-29 17:04:16 +00:00
|
|
|
/// searcher returns a pointer to the found substring or to the end of `haystack`.
|
2020-08-01 21:14:23 +00:00
|
|
|
size_t pos = searcher.search(reinterpret_cast<const UInt8 *>(beg), &haystack_data[haystack_offsets[i] - 1])
|
2020-03-29 17:04:16 +00:00
|
|
|
- &haystack_data[prev_haystack_offset];
|
|
|
|
|
|
|
|
if (pos != haystack_size)
|
|
|
|
{
|
|
|
|
res[i] = 1
|
|
|
|
+ Impl::countChars(
|
|
|
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
|
|
|
reinterpret_cast<const char *>(&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,
|
2020-08-01 21:14:23 +00:00
|
|
|
const ColumnPtr & start_pos,
|
2023-01-23 22:27:48 +00:00
|
|
|
PaddedPODArray<UInt64> & res,
|
|
|
|
[[maybe_unused]] ColumnUInt8 * res_null)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2023-01-23 22:27:48 +00:00
|
|
|
/// `res_null` serves as an output parameter for implementing an XYZOrNull variant.
|
|
|
|
assert(!res_null);
|
2020-03-29 17:04:16 +00:00
|
|
|
|
2023-01-23 22:27:48 +00:00
|
|
|
/// NOTE You could use haystack indexing. But this is a rare case.
|
2020-03-29 17:04:16 +00:00
|
|
|
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;
|
|
|
|
|
2020-08-15 20:08:59 +00:00
|
|
|
auto start = start_pos != nullptr ? std::max(start_pos->getUInt(i), UInt64(1)) : UInt64(1);
|
2020-08-01 21:14:23 +00:00
|
|
|
|
2020-08-02 14:24:39 +00:00
|
|
|
if (start > haystack.size() + 1)
|
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
res[i] = 0;
|
|
|
|
}
|
|
|
|
else if (0 == needle_size)
|
2020-03-29 17:04:16 +00:00
|
|
|
{
|
2020-08-01 21:14:23 +00:00
|
|
|
res[i] = start;
|
2020-03-29 17:04:16 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
|
|
|
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
|
|
|
|
2020-08-01 21:14:23 +00:00
|
|
|
const char * beg = Impl::advancePos(haystack.data(), haystack.data() + haystack.size(), start - 1);
|
2020-03-29 17:04:16 +00:00
|
|
|
size_t pos = searcher.search(
|
2020-08-01 21:14:23 +00:00
|
|
|
reinterpret_cast<const UInt8 *>(beg),
|
2020-03-29 17:04:16 +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];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename... Args>
|
|
|
|
static void vectorFixedConstant(Args &&...)
|
|
|
|
{
|
2021-09-21 16:43:46 +00:00
|
|
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name);
|
2020-03-29 17:04:16 +00:00
|
|
|
}
|
Implement SQL functions (NOT) (I)LIKE() + MATCH() with non-const needles
With this commit, SQL functions LIKE and MATCH and their variants can
work with non-const needle arguments. E.g.
create table tab
(id UInt32, haystack String, needle String)
engine = MergeTree()
order by id;
insert into tab values
(1, 'Hello', '%ell%')
(2, 'World', '%orl%')
select id, haystack, needle, like(haystack, needle)
from tab;
For that, methods vectorVector() and vectorFixedVector() were added to
MatchImpl. The existing code for const needles has an optimization where
the compiled regexp is cached. The new code expects a different needle
per row and consequently does not cache the regexp.
2022-05-16 20:37:31 +00:00
|
|
|
|
|
|
|
template <typename... Args>
|
|
|
|
static void vectorFixedVector(Args &&...)
|
|
|
|
{
|
|
|
|
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name);
|
|
|
|
}
|
2020-03-29 17:04:16 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|