You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/06/24 00:57:37 UTC

[doris] branch master updated: [improvement](function) optimize substr performance (#10169)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 2335d233f1 [improvement](function) optimize substr performance (#10169)
2335d233f1 is described below

commit 2335d233f1f52eb64a380b4c9959becdf182b71b
Author: Kang <kx...@gmail.com>
AuthorDate: Fri Jun 24 08:57:31 2022 +0800

    [improvement](function) optimize substr performance (#10169)
    
    optimize substr performance about 1.5~2x speedup.
---
 be/src/exprs/like_predicate.h       |   2 +-
 be/src/runtime/string_search.hpp    |  29 +-
 be/src/vec/common/string_searcher.h | 860 ++++++++++++++++++++++++++++++++++++
 be/src/vec/common/volnitsky.h       | 713 ++++++++++++++++++++++++++++++
 be/src/vec/functions/like.cpp       |  39 +-
 be/src/vec/functions/like.h         |   2 +-
 6 files changed, 1635 insertions(+), 10 deletions(-)

diff --git a/be/src/exprs/like_predicate.h b/be/src/exprs/like_predicate.h
index c530e32f6f..7f0822ef07 100644
--- a/be/src/exprs/like_predicate.h
+++ b/be/src/exprs/like_predicate.h
@@ -73,7 +73,7 @@ private:
         void set_search_string(const std::string& search_string_arg) {
             search_string = search_string_arg;
             search_string_sv = StringValue(search_string);
-            substring_pattern = StringSearch(&search_string_sv);
+            substring_pattern.set_pattern(&search_string_sv);
         }
     };
 
diff --git a/be/src/runtime/string_search.hpp b/be/src/runtime/string_search.hpp
index 3f657b6fa4..463719f279 100644
--- a/be/src/runtime/string_search.hpp
+++ b/be/src/runtime/string_search.hpp
@@ -23,6 +23,7 @@
 
 #include "common/logging.h"
 #include "runtime/string_value.h"
+#include "vec/common/volnitsky.h"
 
 namespace doris {
 
@@ -31,18 +32,18 @@ public:
     virtual ~StringSearch() {}
     StringSearch() : _pattern(nullptr) {}
 
-    StringSearch(const StringValue* pattern) : _pattern(pattern) {}
+    StringSearch(const StringValue* pattern) { set_pattern(pattern); }
+
+    void set_pattern(const StringValue* pattern) {
+        _pattern = pattern;
+        _vol_searcher.reset(new Volnitsky(pattern->ptr, pattern->len));
+    }
 
     // search for this pattern in str.
     //   Returns the offset into str if the pattern exists
     //   Returns -1 if the pattern is not found
     int search(const StringValue* str) const {
-        if (!str || !_pattern || _pattern->len == 0) {
-            return -1;
-        }
-
-        auto it = std::search(str->ptr, str->ptr + str->len,
-                              std::default_searcher(_pattern->ptr, _pattern->ptr + _pattern->len));
+        auto it = search(str->ptr, str->len);
         if (it == str->ptr + str->len) {
             return -1;
         } else {
@@ -50,8 +51,22 @@ public:
         }
     }
 
+    // search for this pattern in str.
+    //   Returns the offset into str if the pattern exists
+    //   Returns str+len if the pattern is not found
+    const char* search(char* str, size_t len) const {
+        if (!str || !_pattern || _pattern->len == 0) {
+            return str + len;
+        }
+
+        return _vol_searcher->search(str, len);
+    }
+
+    inline size_t get_pattern_length() { return _pattern ? _pattern->len : 0; }
+
 private:
     const StringValue* _pattern;
+    std::unique_ptr<Volnitsky> _vol_searcher;
 };
 
 } // namespace doris
diff --git a/be/src/vec/common/string_searcher.h b/be/src/vec/common/string_searcher.h
new file mode 100644
index 0000000000..a1cc996771
--- /dev/null
+++ b/be/src/vec/common/string_searcher.h
@@ -0,0 +1,860 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+// This file is copied from
+// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/StringSearcher.h
+// and modified by Doris
+
+#pragma once
+
+#include <stdint.h>
+#include <string.h>
+
+#include <algorithm>
+#include <limits>
+#include <vector>
+
+#include "vec/common/string_utils/string_utils.h"
+
+#ifdef __SSE2__
+#include <emmintrin.h>
+#endif
+
+#ifdef __SSE4_1__
+#include <smmintrin.h>
+#endif
+
+namespace doris {
+
+// namespace ErrorCodes
+// {
+//     extern const int BAD_ARGUMENTS;
+// }
+
+/** Variants for searching a substring in a string.
+  * In most cases, performance is less than Volnitsky (see Volnitsky.h).
+  */
+
+class StringSearcherBase {
+public:
+    bool force_fallback = false;
+#ifdef __SSE2__
+protected:
+    static constexpr auto n = sizeof(__m128i);
+    const int page_size = sysconf(_SC_PAGESIZE); //::getPageSize();
+
+    bool page_safe(const void* const ptr) const {
+        return ((page_size - 1) & reinterpret_cast<std::uintptr_t>(ptr)) <= page_size - n;
+    }
+#endif
+};
+
+/// Performs case-sensitive and case-insensitive search of UTF-8 strings
+template <bool CaseSensitive, bool ASCII>
+class StringSearcher;
+
+// comment out since it's not used in doris and UTF8 dependency is not easy to meet
+// /// Case-insensitive UTF-8 searcher
+// template <>
+// class StringSearcher<false, false> : public StringSearcherBase
+// {
+// private:
+//     using UTF8SequenceBuffer = uint8_t[6];
+
+//     /// substring to be searched for
+//     const uint8_t * const needle;
+//     const size_t needle_size;
+//     const uint8_t * const needle_end = needle + needle_size;
+//     /// lower and uppercase variants of the first octet of the first character in `needle`
+//     bool first_needle_symbol_is_ascii{};
+//     uint8_t l{};
+//     uint8_t u{};
+
+// #ifdef __SSE4_1__
+//     /// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
+//     __m128i patl;
+//     __m128i patu;
+//     /// lower and uppercase vectors of first 16 characters of `needle`
+//     __m128i cachel = _mm_setzero_si128();
+//     __m128i cacheu = _mm_setzero_si128();
+//     int cachemask{};
+//     size_t cache_valid_len{};
+//     size_t cache_actual_len{};
+// #endif
+
+// public:
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     StringSearcher(const CharT * needle_, const size_t needle_size_)
+//         : needle{reinterpret_cast<const uint8_t *>(needle_)}, needle_size{needle_size_}
+//     {
+//         if (0 == needle_size)
+//             return;
+
+//         UTF8SequenceBuffer l_seq;
+//         UTF8SequenceBuffer u_seq;
+
+//         if (*needle < 0x80u)
+//         {
+//             first_needle_symbol_is_ascii = true;
+//             l = std::tolower(*needle);
+//             u = std::toupper(*needle);
+//         }
+//         else
+//         {
+//             auto first_u32 = UTF8::convertUTF8ToCodePoint(needle, needle_size);
+
+//             /// Invalid UTF-8
+//             if (!first_u32)
+//             {
+//                 /// Process it verbatim as a sequence of bytes.
+//                 size_t src_len = UTF8::seqLength(*needle);
+
+//                 memcpy(l_seq, needle, src_len);
+//                 memcpy(u_seq, needle, src_len);
+//             }
+//             else
+//             {
+//                 uint32_t first_l_u32 = Poco::Unicode::toLower(*first_u32);
+//                 uint32_t first_u_u32 = Poco::Unicode::toUpper(*first_u32);
+
+//                 /// lower and uppercase variants of the first octet of the first character in `needle`
+//                 size_t length_l = UTF8::convertCodePointToUTF8(first_l_u32, l_seq, sizeof(l_seq));
+//                 size_t length_u = UTF8::convertCodePointToUTF8(first_u_u32, u_seq, sizeof(u_seq));
+
+//                 if (length_l != length_u)
+//                     force_fallback = true;
+//             }
+
+//             l = l_seq[0];
+//             u = u_seq[0];
+
+//             if (force_fallback)
+//                 return;
+//         }
+
+// #ifdef __SSE4_1__
+//         /// for detecting leftmost position of the first symbol
+//         patl = _mm_set1_epi8(l);
+//         patu = _mm_set1_epi8(u);
+//         /// lower and uppercase vectors of first 16 octets of `needle`
+
+//         const auto * needle_pos = needle;
+
+//         for (size_t i = 0; i < n;)
+//         {
+//             if (needle_pos == needle_end)
+//             {
+//                 cachel = _mm_srli_si128(cachel, 1);
+//                 cacheu = _mm_srli_si128(cacheu, 1);
+//                 ++i;
+
+//                 continue;
+//             }
+
+//             size_t src_len = std::min<size_t>(needle_end - needle_pos, UTF8::seqLength(*needle_pos));
+//             auto c_u32 = UTF8::convertUTF8ToCodePoint(needle_pos, src_len);
+
+//             if (c_u32)
+//             {
+//                 int c_l_u32 = Poco::Unicode::toLower(*c_u32);
+//                 int c_u_u32 = Poco::Unicode::toUpper(*c_u32);
+
+//                 size_t dst_l_len = UTF8::convertCodePointToUTF8(c_l_u32, l_seq, sizeof(l_seq));
+//                 size_t dst_u_len = UTF8::convertCodePointToUTF8(c_u_u32, u_seq, sizeof(u_seq));
+
+//                 /// @note Unicode standard states it is a rare but possible occasion
+//                 if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
+//                 {
+//                     force_fallback = true;
+//                     return;
+//                 }
+//             }
+
+//             cache_actual_len += src_len;
+//             if (cache_actual_len < n)
+//                 cache_valid_len += src_len;
+
+//             for (size_t j = 0; j < src_len && i < n; ++j, ++i)
+//             {
+//                 cachel = _mm_srli_si128(cachel, 1);
+//                 cacheu = _mm_srli_si128(cacheu, 1);
+
+//                 if (needle_pos != needle_end)
+//                 {
+//                     cachel = _mm_insert_epi8(cachel, l_seq[j], n - 1);
+//                     cacheu = _mm_insert_epi8(cacheu, u_seq[j], n - 1);
+
+//                     cachemask |= 1 << i;
+//                     ++needle_pos;
+//                 }
+//             }
+//         }
+// #endif
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     ALWAYS_INLINE bool compareTrivial(const CharT * haystack_pos, const CharT * const haystack_end, const uint8_t * needle_pos) const
+//     {
+//         while (haystack_pos < haystack_end && needle_pos < needle_end)
+//         {
+//             auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos);
+//             auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
+
+//             /// Invalid UTF-8, should not compare equals
+//             if (!haystack_code_point || !needle_code_point)
+//                 break;
+
+//             /// Not equals case insensitive.
+//             if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
+//                 break;
+
+//             auto len = UTF8::seqLength(*haystack_pos);
+//             haystack_pos += len;
+
+//             len = UTF8::seqLength(*needle_pos);
+//             needle_pos += len;
+//         }
+
+//         return needle_pos == needle_end;
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * haystack_end, const CharT * pos) const
+//     {
+
+// #ifdef __SSE4_1__
+//         if (page_safe(pos) && !force_fallback)
+//         {
+//             const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
+//             const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
+//             const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
+//             const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
+//             const auto mask = _mm_movemask_epi8(v_against_l_or_u);
+
+//             if (0xffff == cachemask)
+//             {
+//                 if (mask == cachemask)
+//                 {
+//                     if (compareTrivial(pos, haystack_end, needle))
+//                         return true;
+//                 }
+//             }
+//             else if ((mask & cachemask) == cachemask)
+//             {
+//                 if (compareTrivial(pos, haystack_end, needle))
+//                     return true;
+//             }
+
+//             return false;
+//         }
+// #endif
+
+//         if (*pos == l || *pos == u)
+//         {
+//             pos += first_needle_symbol_is_ascii;
+//             const auto * needle_pos = needle + first_needle_symbol_is_ascii;
+
+//             if (compareTrivial(pos, haystack_end, needle_pos))
+//                 return true;
+//         }
+
+//         return false;
+//     }
+
+//     /** Returns haystack_end if not found.
+//       */
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     const CharT * search(const CharT * haystack, const CharT * const haystack_end) const
+//     {
+//         if (0 == needle_size)
+//             return haystack;
+
+//         while (haystack < haystack_end)
+//         {
+// #ifdef __SSE4_1__
+//             if (haystack + n <= haystack_end && page_safe(haystack) && !force_fallback)
+//             {
+//                 const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
+//                 const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
+//                 const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
+//                 const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
+
+//                 const auto mask = _mm_movemask_epi8(v_against_l_or_u);
+
+//                 if (mask == 0)
+//                 {
+//                     haystack += n;
+//                     UTF8::syncForward(haystack, haystack_end);
+//                     continue;
+//                 }
+
+//                 const auto offset = __builtin_ctz(mask);
+//                 haystack += offset;
+
+//                 if (haystack + n <= haystack_end && page_safe(haystack))
+//                 {
+//                     const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
+//                     const auto v_against_l_offset = _mm_cmpeq_epi8(v_haystack_offset, cachel);
+//                     const auto v_against_u_offset = _mm_cmpeq_epi8(v_haystack_offset, cacheu);
+//                     const auto v_against_l_or_u_offset = _mm_or_si128(v_against_l_offset, v_against_u_offset);
+//                     const auto mask_offset_both = _mm_movemask_epi8(v_against_l_or_u_offset);
+
+//                     if (0xffff == cachemask)
+//                     {
+//                         if (mask_offset_both == cachemask)
+//                         {
+//                             if (compareTrivial(haystack, haystack_end, needle))
+//                                 return haystack;
+//                         }
+//                     }
+//                     else if ((mask_offset_both & cachemask) == cachemask)
+//                     {
+//                         if (compareTrivial(haystack, haystack_end, needle))
+//                             return haystack;
+//                     }
+
+//                     /// first octet was ok, but not the first 16, move to start of next sequence and reapply
+//                     haystack += UTF8::seqLength(*haystack);
+//                     continue;
+//                 }
+//             }
+// #endif
+
+//             if (haystack == haystack_end)
+//                 return haystack_end;
+
+//             if (*haystack == l || *haystack == u)
+//             {
+//                 auto haystack_pos = haystack + first_needle_symbol_is_ascii;
+//                 const auto * needle_pos = needle + first_needle_symbol_is_ascii;
+
+//                 if (compareTrivial(haystack_pos, haystack_end, needle_pos))
+//                     return haystack;
+//             }
+
+//             /// advance to the start of the next sequence
+//             haystack += UTF8::seqLength(*haystack);
+//         }
+
+//         return haystack_end;
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     const CharT * search(const CharT * haystack, const size_t haystack_size) const
+//     {
+//         return search(haystack, haystack + haystack_size);
+//     }
+// };
+
+// /// Case-insensitive ASCII searcher
+// template <>
+// class StringSearcher<false, true> : public StringSearcherBase
+// {
+// private:
+//     /// string to be searched for
+//     const uint8_t * const needle;
+//     const uint8_t * const needle_end;
+//     /// lower and uppercase variants of the first character in `needle`
+//     uint8_t l{};
+//     uint8_t u{};
+
+// #ifdef __SSE4_1__
+//     /// vectors filled with `l` and `u`, for determining leftmost position of the first symbol
+//     __m128i patl, patu;
+//     /// lower and uppercase vectors of first 16 characters of `needle`
+//     __m128i cachel = _mm_setzero_si128(), cacheu = _mm_setzero_si128();
+//     int cachemask{};
+// #endif
+
+// public:
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     StringSearcher(const CharT * needle_, const size_t needle_size)
+//         : needle{reinterpret_cast<const uint8_t *>(needle_)}, needle_end{needle + needle_size}
+//     {
+//         if (0 == needle_size)
+//             return;
+
+//         l = static_cast<uint8_t>(std::tolower(*needle));
+//         u = static_cast<uint8_t>(std::toupper(*needle));
+
+// #ifdef __SSE4_1__
+//         patl = _mm_set1_epi8(l);
+//         patu = _mm_set1_epi8(u);
+
+//         const auto * needle_pos = needle;
+
+//         for (const auto i : collections::range(0, n))
+//         {
+//             cachel = _mm_srli_si128(cachel, 1);
+//             cacheu = _mm_srli_si128(cacheu, 1);
+
+//             if (needle_pos != needle_end)
+//             {
+//                 cachel = _mm_insert_epi8(cachel, std::tolower(*needle_pos), n - 1);
+//                 cacheu = _mm_insert_epi8(cacheu, std::toupper(*needle_pos), n - 1);
+//                 cachemask |= 1 << i;
+//                 ++needle_pos;
+//             }
+//         }
+// #endif
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * /*haystack_end*/, const CharT * pos) const
+//     {
+// #ifdef __SSE4_1__
+//         if (page_safe(pos))
+//         {
+//             const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(pos));
+//             const auto v_against_l = _mm_cmpeq_epi8(v_haystack, cachel);
+//             const auto v_against_u = _mm_cmpeq_epi8(v_haystack, cacheu);
+//             const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
+//             const auto mask = _mm_movemask_epi8(v_against_l_or_u);
+
+//             if (0xffff == cachemask)
+//             {
+//                 if (mask == cachemask)
+//                 {
+//                     pos += n;
+//                     const auto * needle_pos = needle + n;
+
+//                     while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
+//                     {
+//                         ++pos;
+//                         ++needle_pos;
+//                     }
+
+//                     if (needle_pos == needle_end)
+//                         return true;
+//                 }
+//             }
+//             else if ((mask & cachemask) == cachemask)
+//                 return true;
+
+//             return false;
+//         }
+// #endif
+
+//         if (*pos == l || *pos == u)
+//         {
+//             ++pos;
+//             const auto * needle_pos = needle + 1;
+
+//             while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
+//             {
+//                 ++pos;
+//                 ++needle_pos;
+//             }
+
+//             if (needle_pos == needle_end)
+//                 return true;
+//         }
+
+//         return false;
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     const CharT * search(const CharT * haystack, const CharT * const haystack_end) const
+//     {
+//         if (needle == needle_end)
+//             return haystack;
+
+//         while (haystack < haystack_end)
+//         {
+// #ifdef __SSE4_1__
+//             if (haystack + n <= haystack_end && page_safe(haystack))
+//             {
+//                 const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
+//                 const auto v_against_l = _mm_cmpeq_epi8(v_haystack, patl);
+//                 const auto v_against_u = _mm_cmpeq_epi8(v_haystack, patu);
+//                 const auto v_against_l_or_u = _mm_or_si128(v_against_l, v_against_u);
+
+//                 const auto mask = _mm_movemask_epi8(v_against_l_or_u);
+
+//                 if (mask == 0)
+//                 {
+//                     haystack += n;
+//                     continue;
+//                 }
+
+//                 const auto offset = __builtin_ctz(mask);
+//                 haystack += offset;
+
+//                 if (haystack + n <= haystack_end && page_safe(haystack))
+//                 {
+//                     const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast<const __m128i *>(haystack));
+//                     const auto v_against_l_offset = _mm_cmpeq_epi8(v_haystack_offset, cachel);
+//                     const auto v_against_u_offset = _mm_cmpeq_epi8(v_haystack_offset, cacheu);
+//                     const auto v_against_l_or_u_offset = _mm_or_si128(v_against_l_offset, v_against_u_offset);
+//                     const auto mask_offset = _mm_movemask_epi8(v_against_l_or_u_offset);
+
+//                     if (0xffff == cachemask)
+//                     {
+//                         if (mask_offset == cachemask)
+//                         {
+//                             const auto * haystack_pos = haystack + n;
+//                             const auto * needle_pos = needle + n;
+
+//                             while (haystack_pos < haystack_end && needle_pos < needle_end &&
+//                                    std::tolower(*haystack_pos) == std::tolower(*needle_pos))
+//                             {
+//                                 ++haystack_pos;
+//                                 ++needle_pos;
+//                             }
+
+//                             if (needle_pos == needle_end)
+//                                 return haystack;
+//                         }
+//                     }
+//                     else if ((mask_offset & cachemask) == cachemask)
+//                         return haystack;
+
+//                     ++haystack;
+//                     continue;
+//                 }
+//             }
+// #endif
+
+//             if (haystack == haystack_end)
+//                 return haystack_end;
+
+//             if (*haystack == l || *haystack == u)
+//             {
+//                 const auto * haystack_pos = haystack + 1;
+//                 const auto * needle_pos = needle + 1;
+
+//                 while (haystack_pos < haystack_end && needle_pos < needle_end &&
+//                        std::tolower(*haystack_pos) == std::tolower(*needle_pos))
+//                 {
+//                     ++haystack_pos;
+//                     ++needle_pos;
+//                 }
+
+//                 if (needle_pos == needle_end)
+//                     return haystack;
+//             }
+
+//             ++haystack;
+//         }
+
+//         return haystack_end;
+//     }
+
+//     template <typename CharT>
+//     // requires (sizeof(CharT) == 1)
+//     const CharT * search(const CharT * haystack, const size_t haystack_size) const
+//     {
+//         return search(haystack, haystack + haystack_size);
+//     }
+// };
+
+/// Case-sensitive searcher (both ASCII and UTF-8)
+template <bool ASCII>
+class StringSearcher<true, ASCII> : public StringSearcherBase {
+private:
+    /// string to be searched for
+    const uint8_t* const needle;
+    const uint8_t* const needle_end;
+    /// first character in `needle`
+    uint8_t first {};
+
+#ifdef __SSE4_1__
+    /// vector filled `first` for determining leftmost position of the first symbol
+    __m128i pattern;
+    /// vector of first 16 characters of `needle`
+    __m128i cache = _mm_setzero_si128();
+    int cachemask {};
+#endif
+
+public:
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    StringSearcher(const CharT* needle_, const size_t needle_size)
+            : needle {reinterpret_cast<const uint8_t*>(needle_)},
+              needle_end {needle + needle_size} {
+        if (0 == needle_size) return;
+
+        first = *needle;
+
+#ifdef __SSE4_1__
+        pattern = _mm_set1_epi8(first);
+
+        const auto* needle_pos = needle;
+
+        //for (const auto i : collections::range(0, n))
+        for (size_t i = 0; i < n; i++) {
+            cache = _mm_srli_si128(cache, 1);
+
+            if (needle_pos != needle_end) {
+                cache = _mm_insert_epi8(cache, *needle_pos, n - 1);
+                cachemask |= 1 << i;
+                ++needle_pos;
+            }
+        }
+#endif
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    ALWAYS_INLINE bool compare(const CharT* /*haystack*/, const CharT* /*haystack_end*/,
+                               const CharT* pos) const {
+#ifdef __SSE4_1__
+        if (page_safe(pos)) {
+            const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i*>(pos));
+            const auto v_against_cache = _mm_cmpeq_epi8(v_haystack, cache);
+            const auto mask = _mm_movemask_epi8(v_against_cache);
+
+            if (0xffff == cachemask) {
+                if (mask == cachemask) {
+                    pos += n;
+                    const auto* needle_pos = needle + n;
+
+                    while (needle_pos < needle_end && *pos == *needle_pos) ++pos, ++needle_pos;
+
+                    if (needle_pos == needle_end) return true;
+                }
+            } else if ((mask & cachemask) == cachemask)
+                return true;
+
+            return false;
+        }
+#endif
+
+        if (*pos == first) {
+            ++pos;
+            const auto* needle_pos = needle + 1;
+
+            while (needle_pos < needle_end && *pos == *needle_pos) ++pos, ++needle_pos;
+
+            if (needle_pos == needle_end) return true;
+        }
+
+        return false;
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const CharT* const haystack_end) const {
+        if (needle == needle_end) return haystack;
+
+        while (haystack < haystack_end) {
+#ifdef __SSE4_1__
+            if (haystack + n <= haystack_end && page_safe(haystack)) {
+                /// find first character
+                const auto v_haystack = _mm_loadu_si128(reinterpret_cast<const __m128i*>(haystack));
+                const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern);
+
+                const auto mask = _mm_movemask_epi8(v_against_pattern);
+
+                /// first character not present in 16 octets starting at `haystack`
+                if (mask == 0) {
+                    haystack += n;
+                    continue;
+                }
+
+                const auto offset = __builtin_ctz(mask);
+                haystack += offset;
+
+                if (haystack + n <= haystack_end && page_safe(haystack)) {
+                    /// check for first 16 octets
+                    const auto v_haystack_offset =
+                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(haystack));
+                    const auto v_against_cache = _mm_cmpeq_epi8(v_haystack_offset, cache);
+                    const auto mask_offset = _mm_movemask_epi8(v_against_cache);
+
+                    if (0xffff == cachemask) {
+                        if (mask_offset == cachemask) {
+                            const auto* haystack_pos = haystack + n;
+                            const auto* needle_pos = needle + n;
+
+                            while (haystack_pos < haystack_end && needle_pos < needle_end &&
+                                   *haystack_pos == *needle_pos)
+                                ++haystack_pos, ++needle_pos;
+
+                            if (needle_pos == needle_end) return haystack;
+                        }
+                    } else if ((mask_offset & cachemask) == cachemask)
+                        return haystack;
+
+                    ++haystack;
+                    continue;
+                }
+            }
+#endif
+
+            if (haystack == haystack_end) return haystack_end;
+
+            if (*haystack == first) {
+                const auto* haystack_pos = haystack + 1;
+                const auto* needle_pos = needle + 1;
+
+                while (haystack_pos < haystack_end && needle_pos < needle_end &&
+                       *haystack_pos == *needle_pos)
+                    ++haystack_pos, ++needle_pos;
+
+                if (needle_pos == needle_end) return haystack;
+            }
+
+            ++haystack;
+        }
+
+        return haystack_end;
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const size_t haystack_size) const {
+        return search(haystack, haystack + haystack_size);
+    }
+};
+
+// Searches for needle surrounded by token-separators.
+// Separators are anything inside ASCII (0-128) and not alphanum.
+// Any value outside of basic ASCII (>=128) is considered a non-separator symbol, hence UTF-8 strings
+// should work just fine. But any Unicode whitespace is not considered a token separtor.
+template <typename StringSearcher>
+class TokenSearcher : public StringSearcherBase {
+    StringSearcher searcher;
+    size_t needle_size;
+
+public:
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    TokenSearcher(const CharT* needle_, const size_t needle_size_)
+            : searcher {needle_, needle_size_}, needle_size(needle_size_) {
+        if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) {
+            //throw Exception{"Needle must not contain whitespace or separator characters", ErrorCodes::BAD_ARGUMENTS};
+        }
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    ALWAYS_INLINE bool compare(const CharT* haystack, const CharT* haystack_end,
+                               const CharT* pos) const {
+        // use searcher only if pos is in the beginning of token and pos + searcher.needle_size is end of token.
+        if (isToken(haystack, haystack_end, pos))
+            return searcher.compare(haystack, haystack_end, pos);
+
+        return false;
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const CharT* const haystack_end) const {
+        // use searcher.search(), then verify that returned value is a token
+        // if it is not, skip it and re-run
+
+        const auto* pos = haystack;
+        while (pos < haystack_end) {
+            pos = searcher.search(pos, haystack_end);
+            if (pos == haystack_end || isToken(haystack, haystack_end, pos)) return pos;
+
+            // assuming that heendle does not contain any token separators.
+            pos += needle_size;
+        }
+        return haystack_end;
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const size_t haystack_size) const {
+        return search(haystack, haystack + haystack_size);
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    ALWAYS_INLINE bool isToken(const CharT* haystack, const CharT* const haystack_end,
+                               const CharT* p) const {
+        return (p == haystack || isTokenSeparator(*(p - 1))) &&
+               (p + needle_size >= haystack_end || isTokenSeparator(*(p + needle_size)));
+    }
+
+    ALWAYS_INLINE static bool isTokenSeparator(const uint8_t c) {
+        return !(is_alpha_numeric_ascii(c) || !is_ascii(c));
+    }
+};
+
+using ASCIICaseSensitiveStringSearcher = StringSearcher<true, true>;
+// using ASCIICaseInsensitiveStringSearcher = StringSearcher<false, true>;
+using UTF8CaseSensitiveStringSearcher = StringSearcher<true, false>;
+// using UTF8CaseInsensitiveStringSearcher = StringSearcher<false, false>;
+using ASCIICaseSensitiveTokenSearcher = TokenSearcher<ASCIICaseSensitiveStringSearcher>;
+// using ASCIICaseInsensitiveTokenSearcher = TokenSearcher<ASCIICaseInsensitiveStringSearcher>;
+
+/** Uses functions from libc.
+  * It makes sense to use only with short haystacks when cheap initialization is required.
+  * There is no option for case-insensitive search for UTF-8 strings.
+  * It is required that strings are zero-terminated.
+  */
+
+struct LibCASCIICaseSensitiveStringSearcher : public StringSearcherBase {
+    const char* const needle;
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    LibCASCIICaseSensitiveStringSearcher(const CharT* const needle_, const size_t /* needle_size */)
+            : needle(reinterpret_cast<const char*>(needle_)) {}
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const CharT* const haystack_end) const {
+        const auto* res = strstr(reinterpret_cast<const char*>(haystack),
+                                 reinterpret_cast<const char*>(needle));
+        if (!res) return haystack_end;
+        return reinterpret_cast<const CharT*>(res);
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const size_t haystack_size) const {
+        return search(haystack, haystack + haystack_size);
+    }
+};
+
+struct LibCASCIICaseInsensitiveStringSearcher : public StringSearcherBase {
+    const char* const needle;
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    LibCASCIICaseInsensitiveStringSearcher(const CharT* const needle_,
+                                           const size_t /* needle_size */)
+            : needle(reinterpret_cast<const char*>(needle_)) {}
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const CharT* const haystack_end) const {
+        const auto* res = strcasestr(reinterpret_cast<const char*>(haystack),
+                                     reinterpret_cast<const char*>(needle));
+        if (!res) return haystack_end;
+        return reinterpret_cast<const CharT*>(res);
+    }
+
+    template <typename CharT>
+    // requires (sizeof(CharT) == 1)
+    const CharT* search(const CharT* haystack, const size_t haystack_size) const {
+        return search(haystack, haystack + haystack_size);
+    }
+};
+
+} // namespace doris
diff --git a/be/src/vec/common/volnitsky.h b/be/src/vec/common/volnitsky.h
new file mode 100644
index 0000000000..97883e75df
--- /dev/null
+++ b/be/src/vec/common/volnitsky.h
@@ -0,0 +1,713 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+// This file is copied from
+// https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/Volnitsky.h
+// and modified by Doris
+
+#pragma once
+
+#include <stdint.h>
+#include <string.h>
+
+#include <algorithm>
+#include <limits>
+#include <vector>
+
+#include "vec/common/string_searcher.h"
+#include "vec/common/unaligned.h"
+
+/** Search for a substring in a string by Volnitsky's algorithm
+  * http://volnitsky.com/project/str_search/
+  *
+  * `haystack` and `needle` can contain zero bytes.
+  *
+  * Algorithm:
+  * - if the `needle` is too small or too large, or too small `haystack`, use std::search or memchr;
+  * - when initializing, fill in an open-addressing linear probing hash table of the form
+  *    hash from the bigram of needle -> the position of this bigram in needle + 1.
+  *    (one is added only to distinguish zero offset from an empty cell)
+  * - the keys are not stored in the hash table, only the values are stored;
+  * - bigrams can be inserted several times if they occur in the needle several times;
+  * - when searching, take from haystack bigram, which should correspond to the last bigram of needle (comparing from the end);
+  * - look for it in the hash table, if found - get the offset from the hash table and compare the string bytewise;
+  * - 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;
+  *
+  * MultiVolnitsky - search for multiple substrings in a string:
+  * - 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
+  */
+
+namespace doris {
+using UInt8 = uint8_t;
+using UInt16 = uint16_t;
+using UInt64 = uint64_t;
+
+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 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).
+
+/** Fits into the L2 cache (of common Intel CPUs).
+     * This number is extremely good for compilers as it is numeric_limits<Uint16>::max() and there are optimizations with movzwl and other instructions with 2 bytes
+     */
+static constexpr size_t hash_size = 64 * 1024;
+
+/// min haystack size to use main algorithm instead of fallback
+static constexpr size_t min_haystack_size_for_algorithm = 20000;
+
+static inline bool isFallbackNeedle(const size_t needle_size, size_t haystack_size_hint = 0) {
+    return needle_size < 2 * sizeof(Ngram) || needle_size >= std::numeric_limits<Offset>::max() ||
+           (haystack_size_hint && haystack_size_hint < min_haystack_size_for_algorithm);
+}
+
+static inline Ngram toNGram(const UInt8* const pos) {
+    return unaligned_load<Ngram>(pos);
+}
+
+template <typename Callback>
+static inline bool putNGramASCIICaseInsensitive(const UInt8* pos, int offset,
+                                                Callback&& putNGramBase) {
+    struct Chars {
+        UInt8 c0;
+        UInt8 c1;
+    };
+
+    union {
+        Ngram n;
+        Chars chars;
+    };
+
+    n = toNGram(pos);
+
+    const auto c0_al = isAlphaASCII(chars.c0);
+    const auto c1_al = isAlphaASCII(chars.c1);
+
+    if (c0_al && c1_al) {
+        /// 4 combinations: AB, aB, Ab, ab
+        putNGramBase(n, offset);
+        chars.c0 = alternateCaseIfAlphaASCII(chars.c0);
+        putNGramBase(n, offset);
+        chars.c1 = alternateCaseIfAlphaASCII(chars.c1);
+        putNGramBase(n, offset);
+        chars.c0 = alternateCaseIfAlphaASCII(chars.c0);
+        putNGramBase(n, offset);
+    } else if (c0_al) {
+        /// 2 combinations: A1, a1
+        putNGramBase(n, offset);
+        chars.c0 = alternateCaseIfAlphaASCII(chars.c0);
+        putNGramBase(n, offset);
+    } else if (c1_al) {
+        /// 2 combinations: 0B, 0b
+        putNGramBase(n, offset);
+        chars.c1 = alternateCaseIfAlphaASCII(chars.c1);
+        putNGramBase(n, offset);
+    } else
+        /// 1 combination: 01
+        putNGramBase(n, offset);
+
+    return true;
+}
+
+// comment out since it's not used in doris and UTF8 dependency is not easy to meet
+// template <typename Callback>
+// static inline bool putNGramUTF8CaseInsensitive(
+//     const UInt8 * pos, int offset, const UInt8 * begin, size_t size, Callback && putNGramBase)
+// {
+//     const UInt8 * end = begin + size;
+
+//     struct Chars
+//     {
+//         UInt8 c0;
+//         UInt8 c1;
+//     };
+
+//     union
+//     {
+//         VolnitskyTraits::Ngram n;
+//         Chars chars;
+//     };
+
+//     n = toNGram(pos);
+
+//     if (isascii(chars.c0) && isascii(chars.c1))
+//     {
+//         return putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
+//     }
+//     else
+//     {
+//         /** n-gram (in the case of n = 2)
+//             *  can be entirely located within one code point,
+//             *  or intersect with two code points.
+//             *
+//             * In the first case, you need to consider up to two alternatives - this code point in upper and lower case,
+//             *  and in the second case - up to four alternatives - fragments of two code points in all combinations of cases.
+//             *
+//             * It does not take into account the dependence of the case-transformation from the locale (for example - Turkish `Ii`)
+//             *  as well as composition / decomposition and other features.
+//             *
+//             * It also does not work if characters with lower and upper cases are represented by different number of bytes or code points.
+//             */
+
+//         using Seq = UInt8[6];
+
+//         if (UTF8::isContinuationOctet(chars.c1))
+//         {
+//             /// ngram is inside a sequence
+//             const auto * seq_pos = pos;
+//             UTF8::syncBackward(seq_pos, begin);
+
+//             auto u32 = UTF8::convertUTF8ToCodePoint(seq_pos, end - seq_pos);
+//             /// Invalid UTF-8
+//             if (!u32)
+//             {
+//                 putNGramBase(n, offset);
+//             }
+//             else
+//             {
+//                 int l_u32 = Poco::Unicode::toLower(*u32);
+//                 int u_u32 = Poco::Unicode::toUpper(*u32);
+
+//                 /// symbol is case-independent
+//                 if (l_u32 == u_u32)
+//                 {
+//                     putNGramBase(n, offset);
+//                 }
+//                 else
+//                 {
+//                     /// where is the given ngram in respect to the start of UTF-8 sequence?
+//                     size_t seq_ngram_offset = pos - seq_pos;
+
+//                     Seq seq_l;
+//                     size_t length_l = UTF8::convertCodePointToUTF8(l_u32, seq_l, sizeof(seq_l));
+
+//                     Seq seq_r;
+//                     size_t length_r = UTF8::convertCodePointToUTF8(u_u32, seq_r, sizeof(seq_r));
+
+//                     if (length_l != length_r)
+//                         return false;
+
+//                     assert(length_l >= 2 && length_r >= 2);
+
+//                     chars.c0 = seq_l[seq_ngram_offset];
+//                     chars.c1 = seq_l[seq_ngram_offset + 1];
+//                     putNGramBase(n, offset);
+
+//                     chars.c0 = seq_r[seq_ngram_offset]; //-V519
+//                     chars.c1 = seq_r[seq_ngram_offset + 1]; //-V519
+//                     putNGramBase(n, offset);
+
+//                 }
+//             }
+//         }
+//         else
+//         {
+//             /// ngram is on the boundary of two sequences
+//             /// first sequence may start before u_pos if it is not ASCII
+//             const auto * first_seq_pos = pos;
+//             UTF8::syncBackward(first_seq_pos, begin);
+//             /// where is the given ngram in respect to the start of first UTF-8 sequence?
+//             size_t seq_ngram_offset = pos - first_seq_pos;
+
+//             auto first_u32 = UTF8::convertUTF8ToCodePoint(first_seq_pos, end - first_seq_pos);
+//             int first_l_u32 = 0;
+//             int first_u_u32 = 0;
+
+//             if (first_u32)
+//             {
+//                 first_l_u32 = Poco::Unicode::toLower(*first_u32);
+//                 first_u_u32 = Poco::Unicode::toUpper(*first_u32);
+//             }
+
+//             /// second sequence always start immediately after u_pos
+//             const auto * second_seq_pos = pos + 1;
+
+//             auto second_u32 = UTF8::convertUTF8ToCodePoint(second_seq_pos, end - second_seq_pos);
+//             int second_l_u32 = 0;
+//             int second_u_u32 = 0;
+
+//             if (second_u32)
+//             {
+//                 second_l_u32 = Poco::Unicode::toLower(*second_u32);
+//                 second_u_u32 = Poco::Unicode::toUpper(*second_u32);
+//             }
+
+//             /// both symbols are case-independent
+//             if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
+//             {
+//                 putNGramBase(n, offset);
+//             }
+//             else if (first_l_u32 == first_u_u32)
+//             {
+//                 /// first symbol is case-independent
+//                 Seq seq_l;
+//                 size_t size_l = UTF8::convertCodePointToUTF8(second_l_u32, seq_l, sizeof(seq_l));
+
+//                 Seq seq_u;
+//                 size_t size_u = UTF8::convertCodePointToUTF8(second_u_u32, seq_u, sizeof(seq_u));
+
+//                 if (size_l != size_u)
+//                     return false;
+
+//                 assert(size_l >= 1 && size_u >= 1);
+//                 chars.c1 = seq_l[0];
+//                 putNGramBase(n, offset);
+
+//                 /// put ngram from uppercase, if it is different
+//                 if (chars.c1 != seq_u[0])
+//                 {
+//                     chars.c1 = seq_u[0];
+//                     putNGramBase(n, offset);
+//                 }
+//             }
+//             else if (second_l_u32 == second_u_u32)
+//             {
+//                 /// second symbol is case-independent
+
+//                 Seq seq_l;
+//                 size_t size_l = UTF8::convertCodePointToUTF8(first_l_u32, seq_l, sizeof(seq_l));
+//                 Seq seq_u;
+//                 size_t size_u = UTF8::convertCodePointToUTF8(first_u_u32, seq_u, sizeof(seq_u));
+
+//                 if (size_l != size_u)
+//                     return false;
+
+//                 assert(size_l > seq_ngram_offset && size_u > seq_ngram_offset);
+
+//                 chars.c0 = seq_l[seq_ngram_offset];
+//                 putNGramBase(n, offset);
+
+//                 /// put ngram for uppercase, if it is different
+//                 if (chars.c0 != seq_u[seq_ngram_offset])
+//                 {
+//                     chars.c0 = seq_u[seq_ngram_offset];
+//                     putNGramBase(n, offset);
+//                 }
+//             }
+//             else
+//             {
+//                 Seq first_l_seq;
+//                 Seq first_u_seq;
+//                 Seq second_l_seq;
+//                 Seq second_u_seq;
+
+//                 size_t size_first_l = UTF8::convertCodePointToUTF8(first_l_u32, first_l_seq, sizeof(first_l_seq));
+//                 size_t size_first_u = UTF8::convertCodePointToUTF8(first_u_u32, first_u_seq, sizeof(first_u_seq));
+//                 size_t size_second_l = UTF8::convertCodePointToUTF8(second_l_u32, second_l_seq, sizeof(second_l_seq));
+//                 size_t size_second_u = UTF8::convertCodePointToUTF8(second_u_u32, second_u_seq, sizeof(second_u_seq));
+//                 if (size_first_l != size_first_u || size_second_l != size_second_u)
+//                     return false;
+
+//                 assert(size_first_l > seq_ngram_offset);
+//                 assert(size_first_u > seq_ngram_offset);
+//                 assert(size_second_l > 0);
+//                 assert(size_second_u > 0);
+
+//                 auto c0l = first_l_seq[seq_ngram_offset];
+//                 auto c0u = first_u_seq[seq_ngram_offset];
+//                 auto c1l = second_l_seq[0];
+//                 auto c1u = second_u_seq[0];
+
+//                 /// ngram for ll
+//                 chars.c0 = c0l;
+//                 chars.c1 = c1l;
+//                 putNGramBase(n, offset);
+
+//                 if (c0l != c0u)
+//                 {
+//                     /// ngram for Ul
+//                     chars.c0 = c0u;
+//                     chars.c1 = c1l; //-V1048
+//                     putNGramBase(n, offset);
+//                 }
+
+//                 if (c1l != c1u)
+//                 {
+//                     /// ngram for lU
+//                     chars.c0 = c0l;
+//                     chars.c1 = c1u;
+//                     putNGramBase(n, offset);
+//                 }
+
+//                 if (c0l != c0u && c1l != c1u)
+//                 {
+//                     /// ngram for UU
+//                     chars.c0 = c0u;
+//                     chars.c1 = c1u;
+//                     putNGramBase(n, offset);
+//                 }
+//             }
+//         }
+//     }
+//     return true;
+// }
+
+template <bool CaseSensitive, bool ASCII, typename Callback>
+static inline bool putNGram(const UInt8* pos, int offset, [[maybe_unused]] const UInt8* begin,
+                            size_t size, Callback&& putNGramBase) {
+    if constexpr (CaseSensitive) {
+        putNGramBase(toNGram(pos), offset);
+        return true;
+    } else if constexpr (ASCII) {
+        return putNGramASCIICaseInsensitive(pos, offset, std::forward<Callback>(putNGramBase));
+    } else {
+        // return putNGramUTF8CaseInsensitive(pos, offset, begin, size, std::forward<Callback>(putNGramBase));
+        return false;
+    }
+}
+} // 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* needle;
+    size_t needle_size;
+    const UInt8* needle_end = needle + needle_size;
+    /// For how long we move, if the n-gram from haystack is not found in the hash table.
+    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 */
+    std::unique_ptr<VolnitskyTraits::Offset[]> hash; /// Hash table.
+
+    bool fallback; /// Do we need to use the fallback algorithm.
+
+    FallbackSearcher fallback_searcher;
+
+public:
+    using Searcher = FallbackSearcher;
+
+    /** 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 || fallback_searcher.force_fallback) return;
+
+        hash = std::unique_ptr<VolnitskyTraits::Offset[]>(
+                new VolnitskyTraits::Offset[VolnitskyTraits::hash_size] {});
+
+        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
+        /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
+        for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0;
+             --i) {
+            bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, i + 1, needle,
+                                                                      needle_size, callback);
+
+            /** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases
+              * are represented by different number of bytes or code points.
+              * So, use fallback if error occurred.
+              */
+            if (!ok) {
+                fallback_searcher.force_fallback = true;
+                hash = nullptr;
+                return;
+            }
+        }
+    }
+
+    /// 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 || fallback_searcher.force_fallback)
+            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(haystack, haystack_end, 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 {
+private:
+    /// needles and their offsets
+    const std::vector<StringRef>& needles;
+
+    /// fallback searchers
+    std::vector<size_t> fallback_needles;
+    std::vector<FallbackSearcher> fallback_searchers;
+
+    /// because std::pair<> is not POD
+    struct OffsetId {
+        VolnitskyTraits::Id id;
+        VolnitskyTraits::Offset off;
+    };
+
+    std::unique_ptr<OffsetId[]> hash;
+
+    /// step for each bunch of strings
+    size_t step;
+
+    /// last index of offsets that was not processed
+    size_t last;
+
+    /// limit for adding to hashtable. In worst case with case insentive search, the table will be filled at most as half
+    static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8;
+
+public:
+    explicit MultiVolnitskyBase(const std::vector<StringRef>& needles_)
+            : needles {needles_}, step {0}, last {0} {
+        fallback_searchers.reserve(needles.size());
+        hash = std::unique_ptr<OffsetId[]>(
+                new OffsetId[VolnitskyTraits::
+                                     hash_size]); /// No zero initialization, it will be done later.
+    }
+
+    /**
+     * 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 (hasMoreToSearch())
+     * {
+     *     search inside the haystack with the known needles
+     * }
+     */
+    bool hasMoreToSearch() {
+        if (last == needles.size()) return false;
+
+        memset(hash.get(), 0, VolnitskyTraits::hash_size * sizeof(OffsetId));
+        fallback_needles.clear();
+        step = std::numeric_limits<size_t>::max();
+
+        size_t buf = 0;
+        size_t size = needles.size();
+
+        for (; last < size; ++last) {
+            const char* cur_needle_data = needles[last].data;
+            const size_t cur_needle_size = needles[last].size;
+
+            /// save the indices of fallback searchers
+            if (VolnitskyTraits::isFallbackNeedle(cur_needle_size)) {
+                fallback_needles.push_back(last);
+            } else {
+                /// put all bigrams
+                auto callback = [this](const VolnitskyTraits::Ngram ngram, const int offset) {
+                    return this->putNGramBase(ngram, offset, this->last);
+                };
+
+                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) break;
+
+                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) {
+                    VolnitskyTraits::putNGram<CaseSensitive, ASCII>(
+                            reinterpret_cast<const UInt8*>(cur_needle_data) + i, i + 1,
+                            reinterpret_cast<const UInt8*>(cur_needle_data), cur_needle_size,
+                            callback);
+                }
+            }
+            fallback_searchers.emplace_back(cur_needle_data, cur_needle_size);
+        }
+        return true;
+    }
+
+    inline bool searchOne(const UInt8* haystack, const UInt8* haystack_end) const {
+        const size_t fallback_size = fallback_needles.size();
+        for (size_t i = 0; i < fallback_size; ++i)
+            if (fallback_searchers[fallback_needles[i]].search(haystack, haystack_end) !=
+                haystack_end)
+                return true;
+
+        /// check if we have one non empty volnitsky searcher
+        if (step != std::numeric_limits<size_t>::max()) {
+            const auto* pos = haystack + step - sizeof(VolnitskyTraits::Ngram);
+            for (; pos <= haystack_end - sizeof(VolnitskyTraits::Ngram); pos += step) {
+                for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size;
+                     hash[cell_num].off; cell_num = (cell_num + 1) % VolnitskyTraits::hash_size) {
+                    if (pos >= haystack + hash[cell_num].off - 1) {
+                        const auto res = pos - (hash[cell_num].off - 1);
+                        const size_t ind = hash[cell_num].id;
+                        if (res + needles[ind].size <= haystack_end &&
+                            fallback_searchers[ind].compare(haystack, haystack_end, res))
+                            return true;
+                    }
+                }
+            }
+        }
+        return false;
+    }
+
+    inline size_t searchOneFirstIndex(const UInt8* haystack, const UInt8* haystack_end) const {
+        const size_t fallback_size = fallback_needles.size();
+
+        size_t answer = std::numeric_limits<size_t>::max();
+
+        for (size_t i = 0; i < fallback_size; ++i)
+            if (fallback_searchers[fallback_needles[i]].search(haystack, haystack_end) !=
+                haystack_end)
+                answer = std::min(answer, fallback_needles[i]);
+
+        /// check if we have one non empty volnitsky searcher
+        if (step != std::numeric_limits<size_t>::max()) {
+            const auto* pos = haystack + step - sizeof(VolnitskyTraits::Ngram);
+            for (; pos <= haystack_end - sizeof(VolnitskyTraits::Ngram); pos += step) {
+                for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size;
+                     hash[cell_num].off; cell_num = (cell_num + 1) % VolnitskyTraits::hash_size) {
+                    if (pos >= haystack + hash[cell_num].off - 1) {
+                        const auto res = pos - (hash[cell_num].off - 1);
+                        const size_t ind = hash[cell_num].id;
+                        if (res + needles[ind].size <= haystack_end &&
+                            fallback_searchers[ind].compare(haystack, haystack_end, res))
+                            answer = std::min(answer, ind);
+                    }
+                }
+            }
+        }
+
+        /*
+        * if nothing was found, answer + 1 will be equal to zero and we can
+        * assign it into the result because we need to return the position starting with one
+        */
+        return answer + 1;
+    }
+
+    template <typename CountCharsCallback>
+    inline UInt64 searchOneFirstPosition(const UInt8* haystack, const UInt8* haystack_end,
+                                         const CountCharsCallback& count_chars) const {
+        const size_t fallback_size = fallback_needles.size();
+
+        UInt64 answer = std::numeric_limits<UInt64>::max();
+
+        for (size_t i = 0; i < fallback_size; ++i)
+            if (auto pos = fallback_searchers[fallback_needles[i]].search(haystack, haystack_end);
+                pos != haystack_end)
+                answer = std::min<UInt64>(answer, pos - haystack);
+
+        /// check if we have one non empty volnitsky searcher
+        if (step != std::numeric_limits<size_t>::max()) {
+            const auto* pos = haystack + step - sizeof(VolnitskyTraits::Ngram);
+            for (; pos <= haystack_end - sizeof(VolnitskyTraits::Ngram); pos += step) {
+                for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size;
+                     hash[cell_num].off; cell_num = (cell_num + 1) % VolnitskyTraits::hash_size) {
+                    if (pos >= haystack + hash[cell_num].off - 1) {
+                        const auto res = pos - (hash[cell_num].off - 1);
+                        const size_t ind = hash[cell_num].id;
+                        if (res + needles[ind].size <= haystack_end &&
+                            fallback_searchers[ind].compare(haystack, haystack_end, res))
+                            answer = std::min<UInt64>(answer, res - haystack);
+                    }
+                }
+            }
+        }
+        if (answer == std::numeric_limits<UInt64>::max()) return 0;
+        return count_chars(haystack, haystack + answer);
+    }
+
+    template <typename CountCharsCallback, typename AnsType>
+    inline void searchOneAll(const UInt8* haystack, const UInt8* haystack_end, AnsType* answer,
+                             const CountCharsCallback& count_chars) const {
+        const size_t fallback_size = fallback_needles.size();
+        for (size_t i = 0; i < fallback_size; ++i) {
+            const UInt8* ptr =
+                    fallback_searchers[fallback_needles[i]].search(haystack, haystack_end);
+            if (ptr != haystack_end) answer[fallback_needles[i]] = count_chars(haystack, ptr);
+        }
+
+        /// check if we have one non empty volnitsky searcher
+        if (step != std::numeric_limits<size_t>::max()) {
+            const auto* pos = haystack + step - sizeof(VolnitskyTraits::Ngram);
+            for (; pos <= haystack_end - sizeof(VolnitskyTraits::Ngram); pos += step) {
+                for (size_t cell_num = VolnitskyTraits::toNGram(pos) % VolnitskyTraits::hash_size;
+                     hash[cell_num].off; cell_num = (cell_num + 1) % VolnitskyTraits::hash_size) {
+                    if (pos >= haystack + hash[cell_num].off - 1) {
+                        const auto* res = pos - (hash[cell_num].off - 1);
+                        const size_t ind = hash[cell_num].id;
+                        if (answer[ind] == 0 && res + needles[ind].size <= haystack_end &&
+                            fallback_searchers[ind].compare(haystack, haystack_end, res))
+                            answer[ind] = count_chars(haystack, res);
+                    }
+                }
+            }
+        }
+    }
+
+    void putNGramBase(const VolnitskyTraits::Ngram ngram, const int offset, const size_t num) {
+        size_t cell_num = ngram % VolnitskyTraits::hash_size;
+
+        while (hash[cell_num].off) cell_num = (cell_num + 1) % VolnitskyTraits::hash_size;
+
+        hash[cell_num] = {static_cast<VolnitskyTraits::Id>(num),
+                          static_cast<VolnitskyTraits::Offset>(offset)};
+    }
+};
+
+using Volnitsky = VolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
+using VolnitskyUTF8 =
+        VolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>; /// exactly same as Volnitsky
+// using VolnitskyCaseInsensitive = VolnitskyBase<false, true, ASCIICaseInsensitiveStringSearcher>; /// ignores non-ASCII bytes
+// using VolnitskyCaseInsensitiveUTF8 = VolnitskyBase<false, false, UTF8CaseInsensitiveStringSearcher>;
+
+using VolnitskyCaseSensitiveToken = VolnitskyBase<true, true, ASCIICaseSensitiveTokenSearcher>;
+// using VolnitskyCaseInsensitiveToken = VolnitskyBase<false, true, ASCIICaseInsensitiveTokenSearcher>;
+
+using MultiVolnitsky = MultiVolnitskyBase<true, true, ASCIICaseSensitiveStringSearcher>;
+using MultiVolnitskyUTF8 = MultiVolnitskyBase<true, false, ASCIICaseSensitiveStringSearcher>;
+// using MultiVolnitskyCaseInsensitive = MultiVolnitskyBase<false, true, ASCIICaseInsensitiveStringSearcher>;
+// using MultiVolnitskyCaseInsensitiveUTF8 = MultiVolnitskyBase<false, false, UTF8CaseInsensitiveStringSearcher>;
+
+} // namespace doris
diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp
index e11e27c5e7..1c61454b71 100644
--- a/be/src/vec/functions/like.cpp
+++ b/be/src/vec/functions/like.cpp
@@ -101,7 +101,8 @@ Status FunctionLikeBase::execute_impl(FunctionContext* context, Block& block,
     // result column
     auto res = ColumnUInt8::create();
     ColumnUInt8::Container& vec_res = res->get_data();
-    vec_res.resize(values->size());
+    // set default value to 0, and match functions only need to set 1/true
+    vec_res.resize_fill(values->size());
 
     auto* state = reinterpret_cast<LikeState*>(
             context->get_function_state(FunctionContext::THREAD_LOCAL));
@@ -129,6 +130,42 @@ Status FunctionLikeBase::vector_vector(const ColumnString::Chars& values,
                                        const ColumnString::Offsets& pattern_offsets,
                                        ColumnUInt8::Container& result, const LikeFn& function,
                                        LikeSearchState* search_state) {
+    // for constant_substring_fn, use long run length search for performance
+    if (constant_substring_fn ==
+        *(function.target<doris::Status (*)(LikeSearchState * state, const StringValue&,
+                                            const StringValue&, unsigned char*)>())) {
+        // treat continous multi string data as a long string data
+        const UInt8* begin = values.data();
+        const UInt8* end = begin + values.size();
+        const UInt8* pos = begin;
+
+        /// Current index in the array of strings.
+        size_t i = 0;
+        size_t needle_size = search_state->substring_pattern.get_pattern_length();
+
+        /// We will search for the next occurrence in all strings at once.
+        while (pos < end) {
+            // search return matched substring start offset
+            pos = (UInt8*)search_state->substring_pattern.search((char*)pos, end - pos);
+            if (pos >= end) break;
+
+            /// Determine which index it refers to.
+            /// begin + value_offsets[i] is the start offset of string at i+1
+            while (begin + value_offsets[i] <= pos) ++i;
+
+            /// We check that the entry does not pass through the boundaries of strings.
+            if (pos + needle_size < begin + value_offsets[i]) {
+                result[i] = 1;
+            }
+
+            // move to next string offset
+            pos = begin + value_offsets[i];
+            ++i;
+        }
+
+        return Status::OK();
+    }
+
     const auto size = value_offsets.size();
 
     for (int i = 0; i < size; ++i) {
diff --git a/be/src/vec/functions/like.h b/be/src/vec/functions/like.h
index e66692a0fa..e19b92d0c2 100644
--- a/be/src/vec/functions/like.h
+++ b/be/src/vec/functions/like.h
@@ -61,7 +61,7 @@ struct LikeSearchState {
     void set_search_string(const std::string& search_string_arg) {
         search_string = search_string_arg;
         search_string_sv = StringValue(search_string);
-        substring_pattern = StringSearch(&search_string_sv);
+        substring_pattern.set_pattern(&search_string_sv);
     }
 };
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org