You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2023/01/13 13:25:44 UTC

[GitHub] [doris] github-actions[bot] commented on a diff in pull request #15916: [Refactor]Merge string_value into string_ref

github-actions[bot] commented on code in PR #15916:
URL: https://github.com/apache/doris/pull/15916#discussion_r1069450660


##########
be/src/runtime/string_search.hpp:
##########
@@ -57,7 +57,7 @@ class StringSearch {
     }
 
     int search(const StringRef& str) const {

Review Comment:
   warning: unknown type name 'StringRef' [clang-diagnostic-error]
   ```cpp
       int search(const StringRef& str) const {
                        ^
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (!compareSSE2(p1 + 16, p2 + 16)) { return false;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (u1 != u2) { return u1 - u2;
   }
   ```
   



##########
be/src/olap/in_list_predicate.h:
##########
@@ -37,49 +37,47 @@
 #include "vec/columns/column_dictionary.h"
 #include "vec/core/types.h"
 
-namespace std {
 // for string value
 template <>
-struct hash<doris::StringValue> {
+struct std::hash<doris::StringValue> {
     uint64_t operator()(const doris::StringValue& rhs) const { return hash_value(rhs); }

Review Comment:
   warning: no matching function for call to 'hash_value' [clang-diagnostic-error]
   ```cpp
       uint64_t operator()(const doris::StringValue& rhs) const { return hash_value(rhs); }
                                                                         ^
   ```
   **be/src/util/uid_util.h:141:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::TUniqueId' for 1st argument
   ```cpp
   std::size_t hash_value(const doris::TUniqueId& id);
               ^
   ```
   **be/src/runtime/decimalv2_value.h:327:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::DecimalV2Value' for 1st argument
   ```cpp
   std::size_t hash_value(DecimalV2Value const& value);
               ^
   ```
   **be/src/runtime/datetime_value.h:717:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::DateTimeValue' for 1st argument
   ```cpp
   std::size_t hash_value(DateTimeValue const& value);
               ^
   ```
   **be/src/runtime/jsonb_value.h:112:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::JsonBinaryValue' for 1st argument
   ```cpp
   inline std::size_t hash_value(const JsonBinaryValue& v) {
                      ^
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+        return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+    }
+    bool end_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data + size - search_string.size, search_string.data,
+                                search_string.size);
+#else
+        return 0 ==
+               memcmp(data + size - search_string.size, search_string.data, search_string.size);
+#endif
+    }
+
+    // Byte-by-byte comparison. Returns:
+    // this < other: -1
+    // this == other: 0
+    // this > other: 1
+    int compare(const StringRef& other) const {
+        int l = std::min(size, other.size);
+
+        if (l == 0) {
+            if (size == other.size) {
+                return 0;
+            } else if (size == 0) {
+                return -1;
+            } else {
+                DCHECK_EQ(other.size, 0);
+                return 1;
+            }
+        }
+
+        return string_compare(this->data, this->size, other.data, other.size, l);
+    }
+
+    void replace(char* ptr, int len) {
+        this->data = ptr;
+        this->size = len;
+    }
+
+    // Find the first position char of appear, return -1 if not found
+    size_t find_first_of(char c) const;
+
+    // ==
+    bool eq(const StringRef& other) const {
+        if (this->size != other.size) {
+            return false;
+        }
+#if defined(__SSE2__)
+        return memequalSSE2Wide(this->data, other.data, this->size);
+#endif
+        return string_compare(this->data, this->size, other.data, other.size, this->size) == 0;
+    }
+    // TODO: no need to implement?
+    std::size_t operator-(const StringValue& v1, const StringValue& v2);
+
+    bool operator==(const StringRef& other) const { return eq(other); }
+    // !=
+    bool ne(const StringRef& other) const { return !eq(other); }
+    // <=
+    bool le(const StringRef& other) const { return compare(other) <= 0; }
+    // >=
+    bool ge(const StringRef& other) const { return compare(other) >= 0; }
+    // <
+    bool lt(const StringRef& other) const { return compare(other) < 0; }
+    // >
+    bool gt(const StringRef& other) const { return compare(other) > 0; }
+
+    bool operator!=(const StringRef& other) const { return ne(other); }
+
+    bool operator<=(const StringRef& other) const { return le(other); }
+
+    bool operator>=(const StringRef& other) const { return ge(other); }
+
+    bool operator<(const StringRef& other) const { return lt(other); }
+
+    bool operator>(const StringRef& other) const { return gt(other); }
+
+    struct Comparator {
+        bool operator()(const StringRef& a, const StringRef& b) const { return a.compare(b) < 0; }
+    };
+
+    struct HashOfStringValue {
+        size_t operator()(const StringRef& v) const { return HashUtil::hash(v.data, v.size, 0); }
+    };
+}; // class StringRef
+
+// This function must be called 'hash_value' to be picked up by boost.
+inline std::size_t hash_value(const StringRef& v) {
+    return HashUtil::hash(v.data, v.size, 0);
+}
+
+using StringRefs = std::vector<StringRef>;
+
+inline bool operator!=(const StringRef& lhs, const StringRef& rhs) {
+    return !(lhs == rhs);
+}
+
+inline bool operator<(const StringRef& lhs, const StringRef& rhs) {
+    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
+    return cmp < 0 || (cmp == 0 && lhs.size < rhs.size);
+}
+
+inline bool operator>(const StringRef& lhs, const StringRef& rhs) {
+    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
+    return cmp > 0 || (cmp == 0 && lhs.size > rhs.size);
+}
+
+// namespace for hash functions
+namespace {
+
+/** Hash functions.
+  * You can use either CityHash64,
+  *  or a function based on the crc32 statement,
+  *  which is obviously less qualitative, but on real data sets,
+  *  when used in a hash table, works much faster.
+  * For more information, see hash_map_string_3.cpp
+  */
+
+struct StringRefHash64 {
+    size_t operator()(StringRef x) const { return util_hash::CityHash64(x.data, x.size); }
+};
+
+#if defined(__SSE4_2__) || defined(__aarch64__)
+
+/// Parts are taken from CityHash.
+
+inline doris::vectorized::UInt64 hash_len16(doris::vectorized::UInt64 u,
+                                            doris::vectorized::UInt64 v) {
+    return Hash128to64(uint128(u, v));
+}
+
+inline doris::vectorized::UInt64 shift_mix(doris::vectorized::UInt64 val) {
+    return val ^ (val >> 47);
+}
+
+inline doris::vectorized::UInt64 rotate_by_at_least1(doris::vectorized::UInt64 val, int shift) {
+    return (val >> shift) | (val << (64 - shift));
+}
+
+inline size_t hash_less_than8(const char* data, size_t size) {
+    static constexpr doris::vectorized::UInt64 k2 = 0x9ae16a3b2f90404fULL;
+    static constexpr doris::vectorized::UInt64 k3 = 0xc949d7c7509e6557ULL;
+
+    if (size >= 4) {
+        doris::vectorized::UInt64 a = unaligned_load<uint32_t>(data);
+        return hash_len16(size + (a << 3), unaligned_load<uint32_t>(data + size - 4));
+    }
+
+    if (size > 0) {
+        uint8_t a = data[0];
+        uint8_t b = data[size >> 1];
+        uint8_t c = data[size - 1];
+        uint32_t y = static_cast<uint32_t>(a) + (static_cast<uint32_t>(b) << 8);
+        uint32_t z = size + (static_cast<uint32_t>(c) << 2);
+        return shift_mix(y * k2 ^ z * k3) * k2;
+    }
+
+    return k2;
+}
+
+inline size_t hash_less_than16(const char* data, size_t size) {
+    if (size > 8) {
+        doris::vectorized::UInt64 a = unaligned_load<doris::vectorized::UInt64>(data);
+        doris::vectorized::UInt64 b = unaligned_load<doris::vectorized::UInt64>(data + size - 8);
+        return hash_len16(a, rotate_by_at_least1(b + size, size)) ^ b;
+    }
+
+    return hash_less_than8(data, size);
+}
+
+struct CRC32Hash {
+    size_t operator()(StringRef x) const {
+        const char* pos = x.data;
+        size_t size = x.size;
+
+        if (size == 0) return 0;
+
+        if (size < 8) {
+            return hash_less_than8(x.data, x.size);
+        }
+
+        const char* end = pos + size;
+        size_t res = -1ULL;
+
+        do {
+            doris::vectorized::UInt64 word = unaligned_load<doris::vectorized::UInt64>(pos);
+            res = _mm_crc32_u64(res, word);
+
+            pos += 8;
+        } while (pos + 8 < end);
+
+        doris::vectorized::UInt64 word = unaligned_load<doris::vectorized::UInt64>(
+                end - 8); /// I'm not sure if this is normal.
+        res = _mm_crc32_u64(res, word);
+
+        return res;
+    }
+};
+
+struct StringRefHash : CRC32Hash {};
+
+#else
+
+struct CRC32Hash {
+    size_t operator()(StringRef /* x */) const {
+        throw std::logic_error {"Not implemented CRC32Hash without SSE"};
+    }
+};
+
+struct StringRefHash : StringRefHash64 {};
+
+#endif
+} // namespace
+
+template <>
+struct std::hash<StringRef> : public StringRefHash {};

Review Comment:
   warning: class template specialization of 'hash' not in a namespace enclosing 'std' [clang-diagnostic-error]
   ```cpp
   struct std::hash<StringRef> : public StringRefHash {};
               ^
   ```
   **/usr/include/c++/11/string_view:686:** explicitly specialized declaration is here
   ```cpp
       struct hash;
              ^
   ```
   



##########
be/src/runtime/string_value.cpp:
##########
@@ -26,26 +26,6 @@
 
 namespace doris {
 
-std::string StringValue::debug_string() const {
-    return std::string(ptr, len);
-}
-
-std::string StringValue::to_string() const {
-    return std::string(ptr, len);
-}
-
-std::ostream& operator<<(std::ostream& os, const StringValue& string_value) {
-    return os << string_value.debug_string();
-}
-
-std::size_t operator-(const StringValue& v1, const StringValue& v2) {
-    return 0;
-}
-
-constexpr char StringValue::MIN_CHAR = 0x00;
-
-constexpr char StringValue::MAX_CHAR = 0xff;
-
 StringValue StringValue::min_string_val() {

Review Comment:
   warning: out-of-line definition of 'min_string_val' does not match any declaration in 'doris::StringValue' [clang-diagnostic-error]
   ```cpp
   StringValue StringValue::min_string_val() {
                            ^
   ```
   



##########
be/src/runtime/string_value.cpp:
##########
@@ -26,26 +26,6 @@
 
 namespace doris {
 
-std::string StringValue::debug_string() const {
-    return std::string(ptr, len);
-}
-
-std::string StringValue::to_string() const {
-    return std::string(ptr, len);
-}
-
-std::ostream& operator<<(std::ostream& os, const StringValue& string_value) {
-    return os << string_value.debug_string();
-}
-
-std::size_t operator-(const StringValue& v1, const StringValue& v2) {
-    return 0;
-}
-
-constexpr char StringValue::MIN_CHAR = 0x00;
-
-constexpr char StringValue::MAX_CHAR = 0xff;
-
 StringValue StringValue::min_string_val() {
     return StringValue((char*)(&StringValue::MIN_CHAR), 0);

Review Comment:
   warning: no matching constructor for initialization of 'doris::StringValue' [clang-diagnostic-error]
   ```cpp
       return StringValue((char*)(&StringValue::MIN_CHAR), 0);
              ^
   ```
   **be/src/runtime/string_value.h:25:** candidate constructor (the implicit copy constructor) not viable: requires 1 argument, but 2 were provided
   ```cpp
   struct StringValue {
          ^
   ```
   **be/src/runtime/string_value.h:25:** candidate constructor (the implicit move constructor) not viable: requires 1 argument, but 2 were provided
   ```cpp
   struct StringValue {
          ^
   ```
   **be/src/runtime/string_value.h:25:** candidate constructor (the implicit default constructor) not viable: requires 0 arguments, but 2 were provided
   ```cpp
   struct StringValue {
          ^
   ```
   



##########
be/src/vec/common/string_tmp.cpp:
##########
@@ -0,0 +1,21 @@
+#include "string_tmp.h"
+
+namespace doris {
+
+inline StringValue StringValue::trim() const {

Review Comment:
   warning: incomplete type 'doris::StringValue' named in nested name specifier [clang-diagnostic-error]
   ```cpp
   inline StringValue StringValue::trim() const {
                      ^
   ```
   **be/src/udf/udf.h:34:** forward declaration of 'doris::StringValue'
   ```cpp
   struct StringValue;
          ^
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (!compareSSE2(p1 + 32, p2 + 32)) { return false;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+        return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+    }
+    bool end_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (search_string.size == 0) { return true;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           } else {
               return false;
   }
   ```
   



##########
be/src/runtime/string_value.h:
##########
@@ -78,134 +28,9 @@ struct StringValue {
     const static char MAX_CHAR;
 
     static const int MAX_LENGTH = (1 << 30);
-    // TODO: change ptr to an offset relative to a contiguous memory block,
-    // so that we can send row batches between nodes without having to swizzle
-    // pointers
-    // NOTE: This struct should keep the same memory layout with Slice, otherwise
-    // it will lead to BE crash.
-    // TODO(zc): we should unify this struct with Slice some day.
-    char* ptr;
-    size_t len;
-
-    StringValue(char* ptr, int len) : ptr(ptr), len(len) {}
-    StringValue(const char* ptr, int len) : ptr(const_cast<char*>(ptr)), len(len) {}
-    StringValue() : ptr(nullptr), len(0) {}
-    StringValue(const StringRef& str) : ptr(const_cast<char*>(str.data)), len(str.size) {}
-
-    /// Construct a StringValue from 's'.  's' must be valid for as long as
-    /// this object is valid.
-    explicit StringValue(const std::string& s) : ptr(const_cast<char*>(s.c_str())), len(s.size()) {
-        DCHECK_LE(len, MAX_LENGTH);
-    }
-
-    void replace(char* ptr, int len) {
-        this->ptr = ptr;
-        this->len = len;
-    }
-
-    // Byte-by-byte comparison. Returns:
-    // this < other: -1
-    // this == other: 0
-    // this > other: 1
-    int compare(const StringValue& other) const {
-        int l = std::min(len, other.len);
-
-        if (l == 0) {
-            if (len == other.len) {
-                return 0;
-            } else if (len == 0) {
-                return -1;
-            } else {
-                DCHECK_EQ(other.len, 0);
-                return 1;
-            }
-        }
-
-        return string_compare(this->ptr, this->len, other.ptr, other.len, l);
-    }
-
-    // ==
-    bool eq(const StringValue& other) const {
-        if (this->len != other.len) {
-            return false;
-        }
-
-#if defined(__SSE2__)
-        return memequalSSE2Wide(this->ptr, other.ptr, this->len);
-#endif
-
-        return string_compare(this->ptr, this->len, other.ptr, other.len, this->len) == 0;
-    }
-
-    bool operator==(const StringValue& other) const { return eq(other); }
-    // !=
-    bool ne(const StringValue& other) const { return !eq(other); }
-    // <=
-    bool le(const StringValue& other) const { return compare(other) <= 0; }
-    // >=
-    bool ge(const StringValue& other) const { return compare(other) >= 0; }
-    // <
-    bool lt(const StringValue& other) const { return compare(other) < 0; }
-    // >
-    bool gt(const StringValue& other) const { return compare(other) > 0; }
-
-    bool operator!=(const StringValue& other) const { return ne(other); }
-
-    bool operator<=(const StringValue& other) const { return le(other); }
 
-    bool operator>=(const StringValue& other) const { return ge(other); }
-
-    bool operator<(const StringValue& other) const { return lt(other); }
-
-    bool operator>(const StringValue& other) const { return gt(other); }
-
-    std::string debug_string() const;
-
-    std::string to_string() const;
-
-    // Returns the substring starting at start_pos until the end of string.
-    StringValue substring(int start_pos) const;
-
-    // Returns the substring starting at start_pos with given length.
-    // If new_len < 0 then the substring from start_pos to end of string is returned.
-    StringValue substring(int start_pos, int new_len) const;
-
-    // Trims leading and trailing spaces.
-    StringValue trim() const;
-
-    // Find the first position char of appear, return -1 if not found
-    int64_t find_first_of(char c) const;
-
-    void to_string_val(doris_udf::StringVal* sv) const {
-        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(ptr), len);
-    }
-
-    static StringValue from_string_val(const doris_udf::StringVal& sv) {
-        return StringValue(reinterpret_cast<char*>(sv.ptr), sv.len);
-    }
-
-    static StringValue min_string_val();
-
-    static StringValue max_string_val();
-
-    struct Comparator {
-        bool operator()(const StringValue& a, const StringValue& b) const {
-            return a.compare(b) < 0;
-        }
-    };
-
-    struct HashOfStringValue {
-        size_t operator()(const StringValue& v) const { return HashUtil::hash(v.ptr, v.len, 0); }
-    };
+    const char* ptr;
+    size_t len;

Review Comment:
   warning: unknown type name 'size_t' [clang-diagnostic-error]
   ```cpp
       size_t len;
       ^
   ```
   



##########
be/src/olap/in_list_predicate.h:
##########
@@ -37,49 +37,47 @@
 #include "vec/columns/column_dictionary.h"
 #include "vec/core/types.h"
 
-namespace std {
 // for string value
 template <>
-struct hash<doris::StringValue> {
+struct std::hash<doris::StringValue> {
     uint64_t operator()(const doris::StringValue& rhs) const { return hash_value(rhs); }
 };
 
 template <>
-struct equal_to<doris::StringValue> {
+struct std::equal_to<doris::StringValue> {
     bool operator()(const doris::StringValue& lhs, const doris::StringValue& rhs) const {
         return lhs == rhs;

Review Comment:
   warning: invalid operands to binary expression ('const doris::StringValue' and 'const doris::StringValue') [clang-diagnostic-error]
   ```cpp
           return lhs == rhs;
                      ^
   ```
   **/usr/include/c++/11/system_error:361:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const std::error_code' for 1st argument
   ```cpp
     operator==(const error_code& __lhs, const error_code& __rhs) noexcept
     ^
   ```
   **/usr/include/c++/11/system_error:367:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const std::error_code' for 1st argument
   ```cpp
     operator==(const error_code& __lhs, const error_condition& __rhs) noexcept
     ^
   ```
   **/usr/include/c++/11/system_error:375:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const std::error_condition' for 1st argument
   ```cpp
     operator==(const error_condition& __lhs,
     ^
   ```
   **/usr/include/c++/11/system_error:407:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const std::error_condition' for 1st argument
   ```cpp
     operator==(const error_condition& __lhs, const error_code& __rhs) noexcept
     ^
   ```
   **be/src/util/slice.h:203:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::Slice' for 1st argument
   ```cpp
   inline bool operator==(const Slice& x, const Slice& y) {
               ^
   ```
   **/usr/include/c++/11/bits/std_thread.h:280:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'thread::id' for 1st argument
   ```cpp
     operator==(thread::id __x, thread::id __y) noexcept
     ^
   ```
   **/usr/include/c++/11/variant:1255:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'std::monostate' for 1st argument
   ```cpp
     constexpr bool operator==(monostate, monostate) noexcept { return true; }
                    ^
   ```
   **be/src/olap/tablet_schema.h:118:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::TabletColumn' for 1st argument
   ```cpp
   bool operator==(const TabletColumn& a, const TabletColumn& b);
        ^
   ```
   **be/src/olap/tablet_schema.h:266:** candidate function not viable: no known conversion from 'const doris::StringValue' to 'const doris::TabletSchema' for 1st argument
   ```cpp
   bool operator==(const TabletSchema& a, const TabletSchema& b);
        ^
   ```
   **/usr/include/c++/11/bits/stl_pair.h:465:** candidate template ignored: could not match 'const pair<_T1, _T2>' against 'const doris::StringValue'
   ```cpp
       operator==(const pair<_T1, _T2>& __x, const pair<_T1, _T2>& __y)
       ^
   ```
   **/usr/include/c++/11/bits/stl_iterator.h:418:** candidate template ignored: could not match 'const reverse_iterator<_Iterator>' against 'const doris::StringValue'
   ```cpp
       operator==(const reverse_iterator<_Iterator>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_iterator.h:457:** candidate template ignored: could not match 'const reverse_iterator<_IteratorL>' against 'const doris::StringValue'
   ```cpp
       operator==(const reverse_iterator<_IteratorL>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_iterator.h:1532:** candidate template ignored: could not match 'const move_iterator<_IteratorL>' against 'const doris::StringValue'
   ```cpp
       operator==(const move_iterator<_IteratorL>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_iterator.h:1600:** candidate template ignored: could not match 'const move_iterator<_Iterator>' against 'const doris::StringValue'
   ```cpp
       operator==(const move_iterator<_Iterator>& __x,
       ^
   ```
   **/usr/include/c++/11/array:275:** candidate template ignored: could not match 'const array<_Tp, _Nm>' against 'const doris::StringValue'
   ```cpp
       operator==(const array<_Tp, _Nm>& __one, const array<_Tp, _Nm>& __two)
       ^
   ```
   **/usr/include/c++/11/tuple:1527:** candidate template ignored: could not match 'const tuple<_TElements...>' against 'const doris::StringValue'
   ```cpp
       operator==(const tuple<_TElements...>& __t,
       ^
   ```
   **/usr/include/c++/11/bits/std_function.h:717:** candidate template ignored: could not match 'const function<_Res (_Args...)>' against 'const doris::StringValue'
   ```cpp
       operator==(const function<_Res(_Args...)>& __f, nullptr_t) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/std_function.h:724:** candidate template ignored: could not match 'const function<_Res (_Args...)>' against 'const doris::StringValue'
   ```cpp
       operator==(nullptr_t, const function<_Res(_Args...)>& __f) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/allocator.h:217:** candidate template ignored: could not match 'const allocator<_T1>' against 'const doris::StringValue'
   ```cpp
       operator==(const allocator<_T1>&, const allocator<_T2>&)
       ^
   ```
   **/usr/include/c++/11/bits/unordered_map.h:2133:** candidate template ignored: could not match 'const unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const unordered_map<_Key, _Tp, _Hash, _Pred, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/unordered_map.h:2147:** candidate template ignored: could not match 'const unordered_multimap<_Key, _Tp, _Hash, _Pred, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const unordered_multimap<_Key, _Tp, _Hash, _Pred, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_vector.h:1891:** candidate template ignored: could not match 'const vector<_Tp, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const vector<_Tp, _Alloc>& __x, const vector<_Tp, _Alloc>& __y)
       ^
   ```
   **/usr/include/c++/11/bits/postypes.h:221:** candidate template ignored: could not match 'const fpos<_StateT>' against 'const doris::StringValue'
   ```cpp
       operator==(const fpos<_StateT>& __lhs, const fpos<_StateT>& __rhs)
       ^
   ```
   **/usr/include/c++/11/string_view:534:** candidate template ignored: could not match 'basic_string_view<_CharT, _Traits>' against 'doris::StringValue'
   ```cpp
       operator==(basic_string_view<_CharT, _Traits> __x,
       ^
   ```
   **/usr/include/c++/11/string_view:540:** candidate template ignored: could not match 'basic_string_view<_CharT, _Traits>' against 'doris::StringValue'
   ```cpp
       operator==(basic_string_view<_CharT, _Traits> __x,
       ^
   ```
   **/usr/include/c++/11/string_view:563:** candidate template ignored: could not match 'basic_string_view<_CharT, _Traits>' against 'doris::StringValue'
   ```cpp
       operator==(__type_identity_t<basic_string_view<_CharT, _Traits>> __x,
       ^
   ```
   **/usr/include/c++/11/bits/basic_string.h:6225:** candidate template ignored: could not match 'const basic_string<_CharT, _Traits, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const basic_string<_CharT, _Traits, _Alloc>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/basic_string.h:6233:** candidate template ignored: could not match 'const basic_string<_CharT>' against 'const doris::StringValue'
   ```cpp
       operator==(const basic_string<_CharT>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/basic_string.h:6247:** candidate template ignored: could not match 'const basic_string<_CharT, _Traits, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const basic_string<_CharT, _Traits, _Alloc>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/basic_string.h:6288:** candidate template ignored: could not match 'const _CharT *' against 'doris::StringValue'
   ```cpp
       operator==(const _CharT* __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/streambuf_iterator.h:225:** candidate template ignored: could not match 'const istreambuf_iterator<_CharT, _Traits>' against 'const doris::StringValue'
   ```cpp
       operator==(const istreambuf_iterator<_CharT, _Traits>& __a,
       ^
   ```
   **/usr/include/c++/11/bits/unique_ptr.h:752:** candidate template ignored: could not match 'const unique_ptr<_Tp, _Dp>' against 'const doris::StringValue'
   ```cpp
       operator==(const unique_ptr<_Tp, _Dp>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/unique_ptr.h:759:** candidate template ignored: could not match 'const unique_ptr<_Tp, _Dp>' against 'const doris::StringValue'
   ```cpp
       operator==(const unique_ptr<_Tp, _Dp>& __x, nullptr_t) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/unique_ptr.h:766:** candidate template ignored: could not match 'const unique_ptr<_Tp, _Dp>' against 'const doris::StringValue'
   ```cpp
       operator==(nullptr_t, const unique_ptr<_Tp, _Dp>& __x) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr_base.h:1409:** candidate template ignored: could not match 'const __shared_ptr<_Tp1, _Lp>' against 'const doris::StringValue'
   ```cpp
       operator==(const __shared_ptr<_Tp1, _Lp>& __a,
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr_base.h:1415:** candidate template ignored: could not match 'const __shared_ptr<_Tp, _Lp>' against 'const doris::StringValue'
   ```cpp
       operator==(const __shared_ptr<_Tp, _Lp>& __a, nullptr_t) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr_base.h:1435:** candidate template ignored: could not match 'const __shared_ptr<_Tp, _Lp>' against 'const doris::StringValue'
   ```cpp
       operator==(nullptr_t, const __shared_ptr<_Tp, _Lp>& __a) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr.h:436:** candidate template ignored: could not match 'const shared_ptr<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const shared_ptr<_Tp>& __a, const shared_ptr<_Up>& __b) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr.h:442:** candidate template ignored: could not match 'const shared_ptr<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const shared_ptr<_Tp>& __a, nullptr_t) noexcept
       ^
   ```
   **/usr/include/c++/11/bits/shared_ptr.h:463:** candidate template ignored: could not match 'const shared_ptr<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(nullptr_t, const shared_ptr<_Tp>& __a) noexcept
       ^
   ```
   **/usr/include/c++/11/optional:1044:** candidate template ignored: could not match 'const optional<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const optional<_Tp>& __lhs, const optional<_Up>& __rhs)
       ^
   ```
   **/usr/include/c++/11/optional:1104:** candidate template ignored: could not match 'const optional<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const optional<_Tp>& __lhs, nullopt_t) noexcept
       ^
   ```
   **/usr/include/c++/11/optional:1115:** candidate template ignored: could not match 'const optional<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(nullopt_t, const optional<_Tp>& __rhs) noexcept
       ^
   ```
   **/usr/include/c++/11/optional:1172:** candidate template ignored: could not match 'const optional<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const optional<_Tp>& __lhs, const _Up& __rhs)
       ^
   ```
   **/usr/include/c++/11/optional:1178:** candidate template ignored: could not match 'const optional<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const _Up& __lhs, const optional<_Tp>& __rhs)
       ^
   ```
   **/usr/include/c++/11/bits/stl_list.h:1994:** candidate template ignored: could not match 'const list<_Tp, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const list<_Tp, _Alloc>& __x, const list<_Tp, _Alloc>& __y)
       ^
   ```
   **/usr/include/c++/11/bits/stl_set.h:984:** candidate template ignored: could not match 'const set<_Key, _Compare, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const set<_Key, _Compare, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_multiset.h:970:** candidate template ignored: could not match 'const multiset<_Key, _Compare, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const multiset<_Key, _Compare, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_map.h:1462:** candidate template ignored: could not match 'const map<_Key, _Tp, _Compare, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const map<_Key, _Tp, _Compare, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_multimap.h:1127:** candidate template ignored: could not match 'const multimap<_Key, _Tp, _Compare, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const multimap<_Key, _Tp, _Compare, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/unordered_set.h:1803:** candidate template ignored: could not match 'const unordered_set<_Value, _Hash, _Pred, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const unordered_set<_Value, _Hash, _Pred, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/unordered_set.h:1817:** candidate template ignored: could not match 'const unordered_multiset<_Value, _Hash, _Pred, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const unordered_multiset<_Value, _Hash, _Pred, _Alloc>& __x,
       ^
   ```
   **/usr/include/c++/11/bits/stl_deque.h:2245:** candidate template ignored: could not match 'const deque<_Tp, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const deque<_Tp, _Alloc>& __x, const deque<_Tp, _Alloc>& __y)
       ^
   ```
   **/usr/include/c++/11/variant:1248:** candidate template ignored: could not match 'const variant<_Types...>' against 'const doris::StringValue'
   ```cpp
     _VARIANT_RELATION_FUNCTION_TEMPLATE(==, equal)
     ^
   ```
   **/usr/include/c++/11/variant:1223:** expanded from macro '_VARIANT_RELATION_FUNCTION_TEMPLATE'
   ```cpp
       constexpr bool operator __OP(const variant<_Types...>& __lhs, \
                      ^
   ```
   **/usr/include/c++/11/complex:463:** candidate template ignored: could not match 'const complex<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const complex<_Tp>& __x, const complex<_Tp>& __y)
       ^
   ```
   **/usr/include/c++/11/complex:468:** candidate template ignored: could not match 'const complex<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const complex<_Tp>& __x, const _Tp& __y)
       ^
   ```
   **/usr/include/c++/11/complex:474:** candidate template ignored: could not match 'const complex<_Tp>' against 'const doris::StringValue'
   ```cpp
       operator==(const _Tp& __x, const complex<_Tp>& __y)
       ^
   ```
   **/usr/include/c++/11/bits/stl_stack.h:318:** candidate template ignored: could not match 'const stack<_Tp, _Seq>' against 'const doris::StringValue'
   ```cpp
       operator==(const stack<_Tp, _Seq>& __x, const stack<_Tp, _Seq>& __y)
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1036:** candidate template ignored: could not match 'const sub_match<_BiIter>' against 'const doris::StringValue'
   ```cpp
       operator==(const sub_match<_BiIter>& __lhs, const sub_match<_BiIter>& __rhs)
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1132:** candidate template ignored: could not match 'const __sub_match_string<_Bi_iter, _Ch_traits, _Ch_alloc>' (aka 'const basic_string<typename iterator_traits<type-parameter-0-0>::value_type, type-parameter-0-1, type-parameter-0-2>') against 'const doris::StringValue'
   ```cpp
       operator==(const __sub_match_string<_Bi_iter, _Ch_traits, _Ch_alloc>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1207:** candidate template ignored: could not match 'const sub_match<_Bi_iter>' against 'const doris::StringValue'
   ```cpp
       operator==(const sub_match<_Bi_iter>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1299:** candidate template ignored: could not match 'const sub_match<_Bi_iter>' against 'const doris::StringValue'
   ```cpp
       operator==(typename iterator_traits<_Bi_iter>::value_type const* __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1374:** candidate template ignored: could not match 'const sub_match<_Bi_iter>' against 'const doris::StringValue'
   ```cpp
       operator==(const sub_match<_Bi_iter>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1467:** candidate template ignored: could not match 'const sub_match<_Bi_iter>' against 'const doris::StringValue'
   ```cpp
       operator==(typename iterator_traits<_Bi_iter>::value_type const& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:1546:** candidate template ignored: could not match 'const sub_match<_Bi_iter>' against 'const doris::StringValue'
   ```cpp
       operator==(const sub_match<_Bi_iter>& __lhs,
       ^
   ```
   **/usr/include/c++/11/bits/regex.h:2102:** candidate template ignored: could not match 'const match_results<_Bi_iter, _Alloc>' against 'const doris::StringValue'
   ```cpp
       operator==(const match_results<_Bi_iter, _Alloc>& __m1,
       ^
   ```
   **/usr/include/c++/11/bits/stl_queue.h:343:** candidate template ignored: could not match 'const queue<_Tp, _Seq>' against 'const doris::StringValue'
   ```cpp
       operator==(const queue<_Tp, _Seq>& __x, const queue<_Tp, _Seq>& __y)
       ^
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+        return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+    }
+    bool end_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data + size - search_string.size, search_string.data,
+                                search_string.size);
+#else
+        return 0 ==
+               memcmp(data + size - search_string.size, search_string.data, search_string.size);
+#endif
+    }
+
+    // Byte-by-byte comparison. Returns:
+    // this < other: -1
+    // this == other: 0
+    // this > other: 1
+    int compare(const StringRef& other) const {
+        int l = std::min(size, other.size);
+
+        if (l == 0) {
+            if (size == other.size) {
+                return 0;
+            } else if (size == 0) {
+                return -1;
+            } else {
+                DCHECK_EQ(other.size, 0);
+                return 1;
+            }
+        }
+
+        return string_compare(this->data, this->size, other.data, other.size, l);
+    }
+
+    void replace(char* ptr, int len) {

Review Comment:
   warning: pointer parameter 'ptr' can be pointer to const [readability-non-const-parameter]
   
   ```suggestion
       void replace(const char* ptr, int len) {
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (search_string.size == 0) { return true;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (!compareSSE2(p1, p2)) { return false;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+        return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+    }
+    bool end_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data + size - search_string.size, search_string.data,
+                                search_string.size);
+#else
+        return 0 ==
+               memcmp(data + size - search_string.size, search_string.data, search_string.size);
+#endif
+    }
+
+    // Byte-by-byte comparison. Returns:
+    // this < other: -1
+    // this == other: 0
+    // this > other: 1
+    int compare(const StringRef& other) const {
+        int l = std::min(size, other.size);
+
+        if (l == 0) {
+            if (size == other.size) {
+                return 0;
+            } else if (size == 0) {
+                return -1;
+            } else {
+                DCHECK_EQ(other.size, 0);
+                return 1;
+            }
+        }
+
+        return string_compare(this->data, this->size, other.data, other.size, l);
+    }
+
+    void replace(char* ptr, int len) {
+        this->data = ptr;
+        this->size = len;
+    }
+
+    // Find the first position char of appear, return -1 if not found
+    size_t find_first_of(char c) const;
+
+    // ==
+    bool eq(const StringRef& other) const {
+        if (this->size != other.size) {
+            return false;
+        }
+#if defined(__SSE2__)
+        return memequalSSE2Wide(this->data, other.data, this->size);
+#endif
+        return string_compare(this->data, this->size, other.data, other.size, this->size) == 0;
+    }
+    // TODO: no need to implement?
+    std::size_t operator-(const StringValue& v1, const StringValue& v2);

Review Comment:
   warning: overloaded 'operator-' must be a unary or binary operator (has 3 parameters) [clang-diagnostic-error]
   ```cpp
       std::size_t operator-(const StringValue& v1, const StringValue& v2);
                   ^
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (u1 == '\0') { return n1 - n2;
   }
   ```
   



##########
be/src/vec/common/string_tmp.h:
##########
@@ -0,0 +1,475 @@
+// 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/base/base/StringRef.h
+// and modified by Doris
+
+#pragma once
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <string_view>
+#include <vector>
+
+#include "gutil/hash/city.h"
+#include "gutil/hash/hash128to64.h"
+#include "udf/udf.h"
+#include "util/cpu_info.h"
+#include "util/hash_util.hpp"
+#include "util/slice.h"
+#include "vec/common/string_ref.h"
+#include "vec/common/unaligned.h"
+#include "vec/core/types.h"
+
+#if defined(__SSE2__)
+#include <emmintrin.h>
+#endif
+
+#if defined(__SSE4_2__)
+#include <nmmintrin.h>
+#include <smmintrin.h>
+
+#include "util/sse_util.hpp"
+#endif
+
+#if defined(__aarch64__)
+#include <sse2neon.h>
+#endif
+
+namespace doris {
+
+/// unnamed namespace packaging simd-style *equality* compare functions.
+namespace {
+
+#if defined(__SSE2__) || defined(__aarch64__)
+
+/** Compare strings for equality.
+  * The approach is controversial and does not win in all cases.
+  * For more information, see hash_map_string_2.cpp
+  */
+
+inline bool compareSSE2(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                            _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))));
+}
+
+inline bool compareSSE2x4(const char* p1, const char* p2) {
+    return 0xFFFF ==
+           _mm_movemask_epi8(_mm_and_si128(
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i*>(p1)),
+                                          _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2))),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 1),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 1))),
+                   _mm_and_si128(
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 2),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 2)),
+                           _mm_cmpeq_epi8(
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p1) + 3),
+                                   _mm_loadu_si128(reinterpret_cast<const __m128i*>(p2) + 3)))));
+}
+
+inline bool memequalSSE2Wide(const char* p1, const char* p2, size_t size) {
+    /** The order of branches and the trick with overlapping comparisons
+      * are the same as in memcpy implementation.
+      * See the comments in
+      * https://github.com/ClickHouse/ClickHouse/blob/master/base/glibc-compatibility/memcpy/memcpy.h
+      */
+
+    if (size <= 16) {
+        if (size >= 8) {
+            /// Chunks of [8,16] bytes.
+            return unaligned_load<uint64_t>(p1) == unaligned_load<uint64_t>(p2) &&
+                   unaligned_load<uint64_t>(p1 + size - 8) ==
+                           unaligned_load<uint64_t>(p2 + size - 8);
+        } else if (size >= 4) {
+            /// Chunks of [4,7] bytes.
+            return unaligned_load<uint32_t>(p1) == unaligned_load<uint32_t>(p2) &&
+                   unaligned_load<uint32_t>(p1 + size - 4) ==
+                           unaligned_load<uint32_t>(p2 + size - 4);
+        } else if (size >= 2) {
+            /// Chunks of [2,3] bytes.
+            return unaligned_load<uint16_t>(p1) == unaligned_load<uint16_t>(p2) &&
+                   unaligned_load<uint16_t>(p1 + size - 2) ==
+                           unaligned_load<uint16_t>(p2 + size - 2);
+        } else if (size >= 1) {
+            /// A single byte.
+            return *p1 == *p2;
+        }
+        return true;
+    }
+
+    while (size >= 64) {
+        if (compareSSE2x4(p1, p2)) {
+            p1 += 64;
+            p2 += 64;
+            size -= 64;
+        } else
+            return false;
+    }
+
+    switch (size / 16) {
+    case 3:
+        if (!compareSSE2(p1 + 32, p2 + 32)) return false;
+        [[fallthrough]];
+    case 2:
+        if (!compareSSE2(p1 + 16, p2 + 16)) return false;
+        [[fallthrough]];
+    case 1:
+        if (!compareSSE2(p1, p2)) return false;
+    }
+
+    return compareSSE2(p1 + size - 16, p2 + size - 16);
+}
+
+#endif
+
+// Compare two strings using sse4.2 intrinsics if they are available. This code assumes
+// that the trivial cases are already handled (i.e. one string is empty).
+// Returns:
+//   < 0 if s1 < s2
+//   0 if s1 == s2
+//   > 0 if s1 > s2
+// The SSE code path is just under 2x faster than the non-sse code path.
+//   - s1/n1: ptr/len for the first string
+//   - s2/n2: ptr/len for the second string
+//   - len: min(n1, n2) - this can be more cheaply passed in by the caller
+inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2, int64_t len) {
+    DCHECK_EQ(len, std::min(n1, n2));
+#ifdef __SSE4_2__
+    while (len >= sse_util::CHARS_PER_128_BIT_REGISTER) {
+        __m128i xmm0 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s1));
+        __m128i xmm1 = _mm_loadu_si128(reinterpret_cast<const __m128i*>(s2));
+        int chars_match = _mm_cmpestri(xmm0, sse_util::CHARS_PER_128_BIT_REGISTER, xmm1,
+                                       sse_util::CHARS_PER_128_BIT_REGISTER, sse_util::STRCMP_MODE);
+        if (chars_match != sse_util::CHARS_PER_128_BIT_REGISTER) {
+            return (unsigned char)s1[chars_match] - (unsigned char)s2[chars_match];
+        }
+        len -= sse_util::CHARS_PER_128_BIT_REGISTER;
+        s1 += sse_util::CHARS_PER_128_BIT_REGISTER;
+        s2 += sse_util::CHARS_PER_128_BIT_REGISTER;
+    }
+#endif
+    unsigned char u1, u2;
+    while (len-- > 0) {
+        u1 = (unsigned char)*s1++;
+        u2 = (unsigned char)*s2++;
+        if (u1 != u2) return u1 - u2;
+        if (u1 == '\0') return n1 - n2;
+    }
+
+    return n1 - n2;
+}
+
+} // unnamed namespace
+
+/// The thing to avoid creating strings to find substrings in the hash table.
+/// User should make sure data source is const
+struct StringRef {
+    const char* data = nullptr;
+    size_t size = 0;
+
+    StringRef(const char* data_, size_t size_) : data(data_), size(size_) {}
+    StringRef(const unsigned char* data_, size_t size_)
+            : StringRef(reinterpret_cast<const char*>(data_), size_) {}
+
+    StringRef(const std::string& s) : data(s.data()), size(s.size()) {}
+    StringRef() = default;
+
+    StringRef(const StringVal& src) : StringRef(src.ptr, src.len) {}
+    // TODO: useful?
+    static StringRef from_string_val(const StringVal& sv) { return StringRef(sv); }
+
+    static constexpr char MIN_CHAR = 0x00;
+    static constexpr char MAX_CHAR = 0xFF;
+    // TODO: useful?
+    static StringRef min_string_val();
+    static StringRef max_string_val();
+
+    std::string to_string() const { return std::string(data, size); }
+    std::string debug_string() const { return to_string(); }
+    std::string_view to_string_view() const { return std::string_view(data, size); }
+    Slice to_slice() const { return doris::Slice(data, size); }
+
+    // this is just for show, e.g. print data to error log, to avoid print large string.
+    std::string to_prefix(size_t length) const { return std::string(data, std::min(length, size)); }
+
+    explicit operator std::string() const { return to_string(); }
+
+    StringRef substring(int start_pos, int new_len) const {
+        return StringRef(data + start_pos, (new_len < 0) ? (size - start_pos) : new_len);
+    }
+
+    StringRef substring(int start_pos) const { return substring(start_pos, size - start_pos); }
+
+    // Trims leading and trailing spaces.
+    StringRef trim() const;
+
+    // TODO: this function is dangerous!
+    StringVal to_string_val() {
+        return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+    // and rename to `set_string_val` ?
+    void to_string_val(doris_udf::StringVal* sv) const {
+        *sv = doris_udf::StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(data)), size);
+    }
+
+    bool start_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data, search_string.data, search_string.size);
+#else
+        return 0 == memcmp(data, search_string.data, search_string.size);
+#endif
+    }
+    bool end_with(const StringRef& search_string) const {
+        DCHECK(size >= search_string.size);
+        if (search_string.size == 0) return true;
+
+#if defined(__SSE2__) || defined(__aarch64__)
+        return memequalSSE2Wide(data + size - search_string.size, search_string.data,
+                                search_string.size);
+#else
+        return 0 ==
+               memcmp(data + size - search_string.size, search_string.data, search_string.size);
+#endif
+    }
+
+    // Byte-by-byte comparison. Returns:
+    // this < other: -1
+    // this == other: 0
+    // this > other: 1
+    int compare(const StringRef& other) const {
+        int l = std::min(size, other.size);
+
+        if (l == 0) {
+            if (size == other.size) {
+                return 0;
+            } else if (size == 0) {
+                return -1;
+            } else {
+                DCHECK_EQ(other.size, 0);
+                return 1;
+            }
+        }
+
+        return string_compare(this->data, this->size, other.data, other.size, l);
+    }
+
+    void replace(char* ptr, int len) {
+        this->data = ptr;
+        this->size = len;
+    }
+
+    // Find the first position char of appear, return -1 if not found
+    size_t find_first_of(char c) const;
+
+    // ==
+    bool eq(const StringRef& other) const {
+        if (this->size != other.size) {
+            return false;
+        }
+#if defined(__SSE2__)
+        return memequalSSE2Wide(this->data, other.data, this->size);
+#endif
+        return string_compare(this->data, this->size, other.data, other.size, this->size) == 0;
+    }
+    // TODO: no need to implement?
+    std::size_t operator-(const StringValue& v1, const StringValue& v2);
+
+    bool operator==(const StringRef& other) const { return eq(other); }
+    // !=
+    bool ne(const StringRef& other) const { return !eq(other); }
+    // <=
+    bool le(const StringRef& other) const { return compare(other) <= 0; }
+    // >=
+    bool ge(const StringRef& other) const { return compare(other) >= 0; }
+    // <
+    bool lt(const StringRef& other) const { return compare(other) < 0; }
+    // >
+    bool gt(const StringRef& other) const { return compare(other) > 0; }
+
+    bool operator!=(const StringRef& other) const { return ne(other); }
+
+    bool operator<=(const StringRef& other) const { return le(other); }
+
+    bool operator>=(const StringRef& other) const { return ge(other); }
+
+    bool operator<(const StringRef& other) const { return lt(other); }
+
+    bool operator>(const StringRef& other) const { return gt(other); }
+
+    struct Comparator {
+        bool operator()(const StringRef& a, const StringRef& b) const { return a.compare(b) < 0; }
+    };
+
+    struct HashOfStringValue {
+        size_t operator()(const StringRef& v) const { return HashUtil::hash(v.data, v.size, 0); }
+    };
+}; // class StringRef
+
+// This function must be called 'hash_value' to be picked up by boost.
+inline std::size_t hash_value(const StringRef& v) {
+    return HashUtil::hash(v.data, v.size, 0);
+}
+
+using StringRefs = std::vector<StringRef>;
+
+inline bool operator!=(const StringRef& lhs, const StringRef& rhs) {
+    return !(lhs == rhs);
+}
+
+inline bool operator<(const StringRef& lhs, const StringRef& rhs) {
+    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
+    return cmp < 0 || (cmp == 0 && lhs.size < rhs.size);
+}
+
+inline bool operator>(const StringRef& lhs, const StringRef& rhs) {
+    int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
+    return cmp > 0 || (cmp == 0 && lhs.size > rhs.size);
+}
+
+// namespace for hash functions
+namespace {
+
+/** Hash functions.
+  * You can use either CityHash64,
+  *  or a function based on the crc32 statement,
+  *  which is obviously less qualitative, but on real data sets,
+  *  when used in a hash table, works much faster.
+  * For more information, see hash_map_string_3.cpp
+  */
+
+struct StringRefHash64 {
+    size_t operator()(StringRef x) const { return util_hash::CityHash64(x.data, x.size); }
+};
+
+#if defined(__SSE4_2__) || defined(__aarch64__)
+
+/// Parts are taken from CityHash.
+
+inline doris::vectorized::UInt64 hash_len16(doris::vectorized::UInt64 u,
+                                            doris::vectorized::UInt64 v) {
+    return Hash128to64(uint128(u, v));
+}
+
+inline doris::vectorized::UInt64 shift_mix(doris::vectorized::UInt64 val) {
+    return val ^ (val >> 47);
+}
+
+inline doris::vectorized::UInt64 rotate_by_at_least1(doris::vectorized::UInt64 val, int shift) {
+    return (val >> shift) | (val << (64 - shift));
+}
+
+inline size_t hash_less_than8(const char* data, size_t size) {
+    static constexpr doris::vectorized::UInt64 k2 = 0x9ae16a3b2f90404fULL;
+    static constexpr doris::vectorized::UInt64 k3 = 0xc949d7c7509e6557ULL;
+
+    if (size >= 4) {
+        doris::vectorized::UInt64 a = unaligned_load<uint32_t>(data);
+        return hash_len16(size + (a << 3), unaligned_load<uint32_t>(data + size - 4));
+    }
+
+    if (size > 0) {
+        uint8_t a = data[0];
+        uint8_t b = data[size >> 1];
+        uint8_t c = data[size - 1];
+        uint32_t y = static_cast<uint32_t>(a) + (static_cast<uint32_t>(b) << 8);
+        uint32_t z = size + (static_cast<uint32_t>(c) << 2);
+        return shift_mix(y * k2 ^ z * k3) * k2;
+    }
+
+    return k2;
+}
+
+inline size_t hash_less_than16(const char* data, size_t size) {
+    if (size > 8) {
+        doris::vectorized::UInt64 a = unaligned_load<doris::vectorized::UInt64>(data);
+        doris::vectorized::UInt64 b = unaligned_load<doris::vectorized::UInt64>(data + size - 8);
+        return hash_len16(a, rotate_by_at_least1(b + size, size)) ^ b;
+    }
+
+    return hash_less_than8(data, size);
+}
+
+struct CRC32Hash {
+    size_t operator()(StringRef x) const {
+        const char* pos = x.data;
+        size_t size = x.size;
+
+        if (size == 0) return 0;

Review Comment:
   warning: statement should be inside braces [readability-braces-around-statements]
   
   ```suggestion
           if (size == 0) { return 0;
   }
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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