提交 45882dc0 编写于 作者: D Danila Kutenin

Merge branch 'danlark1-master'

#pragma once #pragma once
#include <algorithm>
#include <vector> #include <vector>
#include <stdint.h> #include <stdint.h>
#include <string.h> #include <string.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Core/Types.h> #include <Core/Types.h>
#include <Poco/UTF8Encoding.h> #include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h> #include <Poco/Unicode.h>
...@@ -13,7 +12,6 @@ ...@@ -13,7 +12,6 @@
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <common/StringRef.h> #include <common/StringRef.h>
#include <common/unaligned.h> #include <common/unaligned.h>
#include <ext/range.h>
/** Search for a substring in a string by Volnitsky's algorithm /** Search for a substring in a string by Volnitsky's algorithm
* http://volnitsky.com/project/str_search/ * http://volnitsky.com/project/str_search/
...@@ -32,7 +30,7 @@ ...@@ -32,7 +30,7 @@
* - if it did not match, we check the next cell of the hash table from the collision resolution chain; * - if it did not match, we check the next cell of the hash table from the collision resolution chain;
* - if not found, skip to haystack almost the size of the needle bytes; * - if not found, skip to haystack almost the size of the needle bytes;
* *
* MultiVersion: * MultiVolnitsky - search for multiple substrings in a string:
* - Add bigrams to hash table with string index. Then the usual Volnitsky search is used. * - Add bigrams to hash table with string index. Then the usual Volnitsky search is used.
* - We are adding while searching, limiting the number of fallback searchers and the total number of added bigrams * - We are adding while searching, limiting the number of fallback searchers and the total number of added bigrams
*/ */
...@@ -43,7 +41,7 @@ namespace DB ...@@ -43,7 +41,7 @@ namespace DB
namespace VolnitskyTraits namespace VolnitskyTraits
{ {
using Offset = UInt8; /// Offset in the needle. For the basic algorithm, the length of the needle must not be greater than 255. using Offset = UInt8; /// Offset in the needle. For the basic algorithm, the length of the needle must not be greater than 255.
using Id = UInt8; /// Index of the string, must not be greater than 255. using Id = UInt8; /// Index of the string (within the array of multiple needles), must not be greater than 255.
using Ngram = UInt16; /// n-gram (2 bytes). using Ngram = UInt16; /// n-gram (2 bytes).
/** Fits into the L2 cache (of common Intel CPUs). /** Fits into the L2 cache (of common Intel CPUs).
...@@ -62,7 +60,7 @@ namespace VolnitskyTraits ...@@ -62,7 +60,7 @@ namespace VolnitskyTraits
static inline Ngram toNGram(const UInt8 * const pos) { return unalignedLoad<Ngram>(pos); } static inline Ngram toNGram(const UInt8 * const pos) { return unalignedLoad<Ngram>(pos); }
template <class Callback> template <typename Callback>
static inline void putNGramASCIICaseInsensitive(const UInt8 * const pos, const int offset, const Callback & putNGramBase) static inline void putNGramASCIICaseInsensitive(const UInt8 * const pos, const int offset, const Callback & putNGramBase)
{ {
struct Chars struct Chars
...@@ -112,16 +110,19 @@ namespace VolnitskyTraits ...@@ -112,16 +110,19 @@ namespace VolnitskyTraits
putNGramBase(n, offset); putNGramBase(n, offset);
} }
template <bool CaseSensitive, bool ASCII, class Callback> template <bool CaseSensitive, bool ASCII, typename Callback>
static inline void static inline void putNGram(const UInt8 * const pos, const int offset, [[maybe_unused]] const UInt8 * const begin, const Callback & putNGramBase)
putNGram(const UInt8 * const pos, const int offset, [[maybe_unused]] const UInt8 * const begin, const Callback & putNGramBase)
{ {
if constexpr (CaseSensitive) if constexpr (CaseSensitive)
{
putNGramBase(toNGram(pos), offset); putNGramBase(toNGram(pos), offset);
}
else else
{ {
if constexpr (ASCII) if constexpr (ASCII)
{
putNGramASCIICaseInsensitive(pos, offset, putNGramBase); putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
}
else else
{ {
struct Chars struct Chars
...@@ -304,15 +305,107 @@ namespace VolnitskyTraits ...@@ -304,15 +305,107 @@ namespace VolnitskyTraits
} }
} }
} }
}
/// @todo store lowercase needle to speed up in case there are numerous occurrences of bigrams from needle in haystack
template <bool CaseSensitive, bool ASCII, typename FallbackSearcher>
class VolnitskyBase
{
protected:
const UInt8 * const needle;
const size_t needle_size;
const UInt8 * const needle_end = needle + needle_size;
/// For how long we move, if the n-gram from haystack is not found in the hash table.
const size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
VolnitskyTraits::Offset hash[VolnitskyTraits::hash_size]; /// Hash table.
const bool fallback; /// Do we need to use the fallback algorithm.
FallbackSearcher fallback_searcher;
public:
/** haystack_size_hint - the expected total size of the haystack for `search` calls. Optional (zero means unspecified).
* If you specify it small enough, the fallback algorithm will be used,
* since it is considered that it's useless to waste time initializing the hash table.
*/
VolnitskyBase(const char * const needle, const size_t needle_size, size_t haystack_size_hint = 0)
: needle{reinterpret_cast<const UInt8 *>(needle)}
, needle_size{needle_size}
, fallback{VolnitskyTraits::isFallbackNeedle(needle_size, haystack_size_hint)}
, fallback_searcher{needle, needle_size}
{
if (fallback)
return;
memset(hash, 0, sizeof(hash));
auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) { return this->putNGramBase(ngram, offset); };
/// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(this->needle + i, i + 1, this->needle, callback);
}
/// If not found, the end of the haystack is returned.
const UInt8 * search(const UInt8 * const haystack, const size_t haystack_size) const
{
if (needle_size == 0)
return haystack;
} // namespace VolnitskyTraits const auto haystack_end = haystack + haystack_size;
template <bool CaseSensitive, bool ASCII, class FallbackSearcher> if (fallback || haystack_size <= needle_size)
return fallback_searcher.search(haystack, haystack_end);
/// Let's "apply" the needle to the haystack and compare the n-gram from the end of the needle.
const auto * pos = haystack + needle_size - sizeof(VolnitskyTraits::Ngram);
for (; pos <= haystack_end - needle_size; pos += step)
{
/// We look at all the cells of the hash table that can correspond to the n-gram from haystack.
for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size; hash[cell_num];
cell_num = (cell_num + 1) % VolnitskyTraits::hash_size)
{
/// When found - compare bytewise, using the offset from the hash table.
const auto res = pos - (hash[cell_num] - 1);
/// pointer in the code is always padded array so we can use pagesafe semantics
if (fallback_searcher.compare(res))
return res;
}
}
return fallback_searcher.search(pos - step + 1, haystack_end);
}
const char * search(const char * haystack, size_t haystack_size) const
{
return reinterpret_cast<const char *>(search(reinterpret_cast<const UInt8 *>(haystack), haystack_size));
}
protected:
void putNGramBase(const VolnitskyTraits::Ngram ngram, const int offset)
{
/// Put the offset for the n-gram in the corresponding cell or the nearest free cell.
size_t cell_num = ngram % VolnitskyTraits::hash_size;
while (hash[cell_num])
cell_num = (cell_num + 1) % VolnitskyTraits::hash_size; /// Search for the next free cell.
hash[cell_num] = offset;
}
};
template <bool CaseSensitive, bool ASCII, typename FallbackSearcher>
class MultiVolnitskyBase class MultiVolnitskyBase
{ {
private: private:
/// needles and their offsets /// needles and their offsets
const std::vector<String> & needles; const std::vector<StringRef> & needles;
/// fallback searchers /// fallback searchers
...@@ -338,17 +431,24 @@ private: ...@@ -338,17 +431,24 @@ private:
static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8; static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8;
public: public:
MultiVolnitskyBase(const std::vector<String> & needles_) : needles{needles_}, step{0}, last{0} MultiVolnitskyBase(const std::vector<StringRef> & needles_) : needles{needles_}, step{0}, last{0}
{ {
fallback_searchers.reserve(needles.size()); fallback_searchers.reserve(needles.size());
} }
/// returns vector of the positions template <typename ResultType, typename AnsCallback>
std::vector<const char *> search_all(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets) void searchAll(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const AnsCallback & ansCallback,
ResultType & ans)
{ {
const size_t haystack_string_size = haystack_offsets.size(); const size_t haystack_string_size = haystack_offsets.size();
const size_t needles_size = needles.size(); const size_t needles_size = needles.size();
std::vector<const char *> ans(needles_size * haystack_string_size, nullptr);
/// something can be uninitialized after
std::fill(ans.begin(), ans.end(), 0);
while (!reset()) while (!reset())
{ {
size_t fallback_size = fallback_needles.size(); size_t fallback_size = fallback_needles.size();
...@@ -361,7 +461,7 @@ public: ...@@ -361,7 +461,7 @@ public:
{ {
const UInt8 * ptr = fallback_searchers[fallback_needles[i]].search(haystack, haystack_end); const UInt8 * ptr = fallback_searchers[fallback_needles[i]].search(haystack, haystack_end);
if (ptr != haystack_end) if (ptr != haystack_end)
ans[from + fallback_needles[i]] = reinterpret_cast<const char *>(ptr); ans[from + fallback_needles[i]] = ansCallback(haystack, ptr);
} }
/// check if we have one non empty volnitsky searcher /// check if we have one non empty volnitsky searcher
...@@ -375,13 +475,13 @@ public: ...@@ -375,13 +475,13 @@ public:
{ {
if (pos >= haystack + hash[cell_num].off - 1) if (pos >= haystack + hash[cell_num].off - 1)
{ {
const auto res = pos - (hash[cell_num].off - 1); const auto * res = pos - (hash[cell_num].off - 1);
const size_t ind = hash[cell_num].id; const size_t ind = hash[cell_num].id;
if (!ans[from + ind] && res + needles[ind].size() <= haystack_end) if (ans[from + ind] == 0 && res + needles[ind].size <= haystack_end)
{ {
if (fallback_searchers[ind].compare(res)) if (fallback_searchers[ind].compare(res))
{ {
ans[from + ind] = reinterpret_cast<const char *>(res); ans[from + ind] = ansCallback(haystack, res);
} }
} }
} }
...@@ -391,46 +491,42 @@ public: ...@@ -391,46 +491,42 @@ public:
prev_offset = haystack_offsets[j]; prev_offset = haystack_offsets[j];
} }
} }
return ans;
} }
std::vector<char> search(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets) template <typename ResultType>
void search(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, ResultType & ans)
{ {
const size_t haystack_string_size = haystack_offsets.size(); auto callback = [this](const UInt8 * haystack, const UInt8 * haystack_end) -> bool
std::vector<char> ans(haystack_string_size, 0);
while (!reset())
{ {
size_t prev_offset = 0; return this->searchOne(haystack, haystack_end);
for (size_t j = 0; j < haystack_string_size; ++j) };
{ searchInternal(haystack_data, haystack_offsets, callback, ans);
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
ans[j] = search_one(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
}
return ans;
} }
std::vector<size_t> search_index(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets) template <typename ResultType>
void searchIndex(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, ResultType & ans)
{ {
const size_t haystack_string_size = haystack_offsets.size(); auto callback = [this](const UInt8 * haystack, const UInt8 * haystack_end) -> size_t
std::vector<size_t> ans(haystack_string_size, 0);
while (!reset())
{ {
size_t prev_offset = 0; return this->searchOneIndex(haystack, haystack_end);
for (size_t j = 0; j < haystack_string_size; ++j) };
{ searchInternal(haystack_data, haystack_offsets, callback, ans);
const auto * haystack = &haystack_data[prev_offset];
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
ans[j] = search_one_index(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
}
return ans;
} }
private: private:
/**
* This function is needed to initialize hash table
* Returns `true` if there is nothing to initialize
* and `false` if we have something to initialize and initializes it.
* This function is a kind of fallback if there are many needles.
* We actually destroy the hash table and initialize it with uninitialized needles
* and search through the haystack again.
* The actual usage of this function is like this:
* while (!reset())
* {
* search inside the haystack with the known needles
* }
*/
bool reset() bool reset()
{ {
if (last == needles.size()) if (last == needles.size())
...@@ -445,8 +541,8 @@ private: ...@@ -445,8 +541,8 @@ private:
for (; last < size; ++last) for (; last < size; ++last)
{ {
const char * cur_needle_data = needles[last].data(); const char * cur_needle_data = needles[last].data;
const size_t cur_needle_size = needles[last].size(); const size_t cur_needle_size = needles[last].size;
/// save the indices of fallback searchers /// save the indices of fallback searchers
if (VolnitskyTraits::isFallbackNeedle(cur_needle_size)) if (VolnitskyTraits::isFallbackNeedle(cur_needle_size))
...@@ -463,10 +559,9 @@ private: ...@@ -463,10 +559,9 @@ private:
buf += cur_needle_size - sizeof(VolnitskyTraits::Ngram) + 1; buf += cur_needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
/// this is the condition when we actually need to stop and start searching with known needles
if (buf > small_limit) if (buf > small_limit)
{
break; break;
}
step = std::min(step, cur_needle_size - sizeof(VolnitskyTraits::Ngram) + 1); step = std::min(step, cur_needle_size - sizeof(VolnitskyTraits::Ngram) + 1);
for (auto i = static_cast<int>(cur_needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) for (auto i = static_cast<int>(cur_needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
...@@ -483,7 +578,28 @@ private: ...@@ -483,7 +578,28 @@ private:
return false; return false;
} }
inline bool search_one(const UInt8 * haystack, const UInt8 * haystack_end) template <typename OneSearcher, typename ResultType>
inline void searchInternal(
const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets,
const OneSearcher & searchFallback,
ResultType & ans)
{
const size_t haystack_string_size = haystack_offsets.size();
while (!reset())
{
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;
ans[j] = searchFallback(haystack, haystack_end);
prev_offset = haystack_offsets[j];
}
}
}
inline bool searchOne(const UInt8 * haystack, const UInt8 * haystack_end) const
{ {
const size_t fallback_size = fallback_needles.size(); const size_t fallback_size = fallback_needles.size();
for (size_t i = 0; i < fallback_size; ++i) for (size_t i = 0; i < fallback_size; ++i)
...@@ -503,7 +619,7 @@ private: ...@@ -503,7 +619,7 @@ private:
{ {
const auto res = pos - (hash[cell_num].off - 1); const auto res = pos - (hash[cell_num].off - 1);
const size_t ind = hash[cell_num].id; const size_t ind = hash[cell_num].id;
if (res + needles[ind].size() <= haystack_end && fallback_searchers[ind].compare(res)) if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res))
return true; return true;
} }
} }
...@@ -512,7 +628,7 @@ private: ...@@ -512,7 +628,7 @@ private:
return false; return false;
} }
inline size_t search_one_index(const UInt8 * haystack, const UInt8 * haystack_end) inline size_t searchOneIndex(const UInt8 * haystack, const UInt8 * haystack_end) const
{ {
const size_t fallback_size = fallback_needles.size(); const size_t fallback_size = fallback_needles.size();
...@@ -535,7 +651,7 @@ private: ...@@ -535,7 +651,7 @@ private:
{ {
const auto res = pos - (hash[cell_num].off - 1); const auto res = pos - (hash[cell_num].off - 1);
const size_t ind = hash[cell_num].id; const size_t ind = hash[cell_num].id;
if (res + needles[ind].size() <= haystack_end && fallback_searchers[ind].compare(res)) if (res + needles[ind].size <= haystack_end && fallback_searchers[ind].compare(res))
ans = std::min(ans, ind); ans = std::min(ans, ind);
} }
} }
...@@ -543,8 +659,8 @@ private: ...@@ -543,8 +659,8 @@ private:
} }
/* /*
* NOTE!!! if nothing was found, ans + 1 will be equal to zero and we can * if nothing was found, ans + 1 will be equal to zero and we can
* std::copy it into the result because we need to return the position starting with one * assign it into the result because we need to return the position starting with one
*/ */
return ans + 1; return ans + 1;
} }
...@@ -560,96 +676,6 @@ private: ...@@ -560,96 +676,6 @@ private:
} }
}; };
/// @todo store lowercase needle to speed up in case there are numerous occurrences of bigrams from needle in haystack
template <bool CaseSensitive, bool ASCII, class FallbackSearcher>
class VolnitskyBase
{
protected:
const UInt8 * const needle;
const size_t needle_size;
const UInt8 * const needle_end = needle + needle_size;
/// For how long we move, if the n-gram from haystack is not found in the hash table.
const size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
VolnitskyTraits::Offset hash[VolnitskyTraits::hash_size]; /// Hash table.
const bool fallback; /// Do we need to use the fallback algorithm.
FallbackSearcher fallback_searcher;
public:
/** haystack_size_hint - the expected total size of the haystack for `search` calls. Optional (zero means unspecified).
* If you specify it small enough, the fallback algorithm will be used,
* since it is considered that it's useless to waste time initializing the hash table.
*/
VolnitskyBase(const char * const needle, const size_t needle_size, size_t haystack_size_hint = 0)
: needle{reinterpret_cast<const UInt8 *>(needle)}
, needle_size{needle_size}
, fallback{VolnitskyTraits::isFallbackNeedle(needle_size, haystack_size_hint)}
, fallback_searcher{needle, needle_size}
{
if (fallback)
return;
memset(hash, 0, sizeof(hash));
auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) { return this->putNGramBase(ngram, offset); };
/// int is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
for (auto i = static_cast<int>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(this->needle + i, i + 1, this->needle, callback);
}
/// If not found, the end of the haystack is returned.
const UInt8 * search(const UInt8 * const haystack, const size_t haystack_size) const
{
if (needle_size == 0)
return haystack;
const auto haystack_end = haystack + haystack_size;
if (fallback || haystack_size <= needle_size)
return fallback_searcher.search(haystack, haystack_end);
/// Let's "apply" the needle to the haystack and compare the n-gram from the end of the needle.
const auto * pos = haystack + needle_size - sizeof(VolnitskyTraits::Ngram);
for (; pos <= haystack_end - needle_size; pos += step)
{
/// We look at all the cells of the hash table that can correspond to the n-gram from haystack.
for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size; hash[cell_num];
cell_num = (cell_num + 1) % VolnitskyTraits::hash_size)
{
/// When found - compare bytewise, using the offset from the hash table.
const auto res = pos - (hash[cell_num] - 1);
/// pointer in the code is always padded array so we can use pagesafe semantics
if (fallback_searcher.compare(res))
return res;
}
}
return fallback_searcher.search(pos - step + 1, haystack_end);
}
const char * search(const char * haystack, size_t haystack_size) const
{
return reinterpret_cast<const char *>(search(reinterpret_cast<const UInt8 *>(haystack), haystack_size));
}
protected:
void putNGramBase(const VolnitskyTraits::Ngram ngram, const int offset)
{
/// Put the offset for the n-gram in the corresponding cell or the nearest free cell.
size_t cell_num = ngram % VolnitskyTraits::hash_size;
while (hash[cell_num])
cell_num = (cell_num + 1) % VolnitskyTraits::hash_size; /// Search for the next free cell.
hash[cell_num] = offset;
}
};
using Volnitsky = VolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>; using Volnitsky = VolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
using VolnitskyUTF8 = VolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>; /// exactly same as Volnitsky using VolnitskyUTF8 = VolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>; /// exactly same as Volnitsky
......
...@@ -53,7 +53,7 @@ struct PositionCaseSensitiveASCII ...@@ -53,7 +53,7 @@ struct PositionCaseSensitiveASCII
return SearcherInSmallHaystack(needle_data, needle_size); return SearcherInSmallHaystack(needle_data, needle_size);
} }
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<String> & needles) static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{ {
return MultiSearcherInBigHaystack(needles); return MultiSearcherInBigHaystack(needles);
} }
...@@ -83,7 +83,7 @@ struct PositionCaseInsensitiveASCII ...@@ -83,7 +83,7 @@ struct PositionCaseInsensitiveASCII
return SearcherInSmallHaystack(needle_data, needle_size); return SearcherInSmallHaystack(needle_data, needle_size);
} }
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<String> & needles) static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{ {
return MultiSearcherInBigHaystack(needles); return MultiSearcherInBigHaystack(needles);
} }
...@@ -109,7 +109,7 @@ struct PositionCaseSensitiveUTF8 ...@@ -109,7 +109,7 @@ struct PositionCaseSensitiveUTF8
return SearcherInSmallHaystack(needle_data, needle_size); return SearcherInSmallHaystack(needle_data, needle_size);
} }
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<String> & needles) static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{ {
return MultiSearcherInBigHaystack(needles); return MultiSearcherInBigHaystack(needles);
} }
...@@ -142,7 +142,7 @@ struct PositionCaseInsensitiveUTF8 ...@@ -142,7 +142,7 @@ struct PositionCaseInsensitiveUTF8
return SearcherInSmallHaystack(needle_data, needle_size); return SearcherInSmallHaystack(needle_data, needle_size);
} }
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<String> & needles) static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
{ {
return MultiSearcherInBigHaystack(needles); return MultiSearcherInBigHaystack(needles);
} }
...@@ -267,7 +267,7 @@ struct PositionImpl ...@@ -267,7 +267,7 @@ struct PositionImpl
} }
} }
/// Find many substrings in one line. /// Find many substrings in single string.
static void constant_vector( static void constant_vector(
const String & haystack, const String & haystack,
const ColumnString::Chars & needle_data, const ColumnString::Chars & needle_data,
...@@ -316,31 +316,18 @@ struct MultiPositionImpl ...@@ -316,31 +316,18 @@ struct MultiPositionImpl
{ {
using ResultType = UInt64; using ResultType = UInt64;
static void multi_constant_vector( static void vector_constant(
const ColumnString::Chars & haystack_data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<String> & needles, const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res) PaddedPODArray<UInt64> & res)
{ {
const size_t needles_size = needles.size(); auto resCallback = [](const UInt8 * start, const UInt8 * end) -> UInt64
const size_t haystack_offsets_size = haystack_offsets.size();
size_t k = 0;
const auto result = Impl::createMultiSearcherInBigHaystack(needles).search_all(haystack_data, haystack_offsets);
for (size_t j = 0; j < haystack_offsets_size; ++j)
{ {
for (size_t i = 0; i < needles_size; ++i) return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
{ };
const char * ptr = result[k];
if (ptr) Impl::createMultiSearcherInBigHaystack(needles).searchAll(haystack_data, haystack_offsets, resCallback, res);
{
const char * start = reinterpret_cast<const char *>(&haystack_data[j == 0 ? 0 : haystack_offsets[j - 1]]);
res[k] = 1 + Impl::countChars(start, ptr);
}
else
res[k] = 0;
++k;
}
}
} }
}; };
...@@ -349,14 +336,13 @@ struct MultiSearchImpl ...@@ -349,14 +336,13 @@ struct MultiSearchImpl
{ {
using ResultType = UInt64; using ResultType = UInt64;
static void multi_constant_vector( static void vector_constant(
const ColumnString::Chars & haystack_data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<String> & needles, const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res) PaddedPODArray<UInt64> & res)
{ {
const auto result = Impl::createMultiSearcherInBigHaystack(needles).search(haystack_data, haystack_offsets); Impl::createMultiSearcherInBigHaystack(needles).search(haystack_data, haystack_offsets, res);
std::copy(result.begin(), result.end(), res.begin());
} }
}; };
...@@ -365,14 +351,13 @@ struct FirstMatchImpl ...@@ -365,14 +351,13 @@ struct FirstMatchImpl
{ {
using ResultType = UInt64; using ResultType = UInt64;
static void multi_constant_vector( static void vector_constant(
const ColumnString::Chars & haystack_data, const ColumnString::Chars & haystack_data,
const ColumnString::Offsets & haystack_offsets, const ColumnString::Offsets & haystack_offsets,
const std::vector<String> & needles, const std::vector<StringRef> & needles,
PaddedPODArray<UInt64> & res) PaddedPODArray<UInt64> & res)
{ {
const auto result = Impl::createMultiSearcherInBigHaystack(needles).search_index(haystack_data, haystack_offsets); Impl::createMultiSearcherInBigHaystack(needles).searchIndex(haystack_data, haystack_offsets, res);
std::copy(result.begin(), result.end(), res.begin());
} }
}; };
...@@ -543,7 +528,7 @@ struct MatchImpl ...@@ -543,7 +528,7 @@ struct MatchImpl
size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1; size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1;
/** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp,
* so that it can match when `required_substring` occurs into the line several times, * so that it can match when `required_substring` occurs into the string several times,
* and at the first occurrence, the regexp is not a match. * and at the first occurrence, the regexp is not a match.
*/ */
...@@ -875,7 +860,7 @@ struct ReplaceStringImpl ...@@ -875,7 +860,7 @@ struct ReplaceStringImpl
if (i == offsets.size()) if (i == offsets.size())
break; break;
/// Is it true that this line no longer needs to perform transformations. /// Is it true that this string no longer needs to perform transformations.
bool can_finish_current_string = false; bool can_finish_current_string = false;
/// We check that the entry does not go through the boundaries of strings. /// We check that the entry does not go through the boundaries of strings.
...@@ -964,7 +949,7 @@ struct ReplaceStringImpl ...@@ -964,7 +949,7 @@ struct ReplaceStringImpl
memcpy(&res_data[res_offset], pos, match - pos); memcpy(&res_data[res_offset], pos, match - pos);
res_offset += (match - pos); res_offset += (match - pos);
/// Is it true that this line no longer needs to perform conversions. /// Is it true that this string no longer needs to perform conversions.
bool can_finish_current_string = false; bool can_finish_current_string = false;
/// We check that the entry does not pass through the boundaries of strings. /// We check that the entry does not pass through the boundaries of strings.
......
...@@ -12,7 +12,6 @@ ...@@ -12,7 +12,6 @@
#include <Functions/IFunction.h> #include <Functions/IFunction.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <common/StringRef.h> #include <common/StringRef.h>
#include <ext/range.h>
namespace DB namespace DB
{ {
...@@ -241,11 +240,9 @@ public: ...@@ -241,11 +240,9 @@ public:
Array src_arr = col_const_arr->getValue<Array>(); Array src_arr = col_const_arr->getValue<Array>();
std::vector<String> refs; std::vector<StringRef> refs;
for (const auto & el : src_arr) for (const auto & el : src_arr)
{ refs.emplace_back(el.get<String>());
refs.push_back(el.get<String>());
}
const size_t column_haystack_size = column_haystack->size(); const size_t column_haystack_size = column_haystack->size();
...@@ -258,7 +255,7 @@ public: ...@@ -258,7 +255,7 @@ public:
vec_res.resize(column_haystack_size * refs.size()); vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector) if (col_haystack_vector)
Impl::multi_constant_vector(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
...@@ -266,9 +263,7 @@ public: ...@@ -266,9 +263,7 @@ public:
size_t accum = refs_size; size_t accum = refs_size;
for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size)
{
offsets_res[i] = accum; offsets_res[i] = accum;
}
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
} }
...@@ -324,7 +319,7 @@ public: ...@@ -324,7 +319,7 @@ public:
Array src_arr = col_const_arr->getValue<Array>(); Array src_arr = col_const_arr->getValue<Array>();
std::vector<String> refs; std::vector<StringRef> refs;
refs.reserve(src_arr.size()); refs.reserve(src_arr.size());
for (const auto & el : src_arr) for (const auto & el : src_arr)
...@@ -339,7 +334,7 @@ public: ...@@ -339,7 +334,7 @@ public:
vec_res.resize(column_haystack_size); vec_res.resize(column_haystack_size);
if (col_haystack_vector) if (col_haystack_vector)
Impl::multi_constant_vector(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册