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

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


The following commit(s) were added to refs/heads/master by this push:
     new e1090d6a63 [Fix](column predicate) seperate CHAR primitive type for 
column predicate (#23581)
e1090d6a63 is described below

commit e1090d6a631b29f94cda54de2f7f4d830360f926
Author: airborne12 <[email protected]>
AuthorDate: Fri Sep 1 09:41:53 2023 +0800

    [Fix](column predicate) seperate CHAR primitive type for column predicate 
(#23581)
---
 be/src/exprs/bloom_filter_func.h                   |   1 +
 be/src/exprs/create_predicate_function.h           |  20 +++
 be/src/olap/like_column_predicate.cpp              |  57 +++++---
 be/src/olap/like_column_predicate.h                |   1 +
 be/src/olap/reader.cpp                             |  11 +-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   3 +-
 be/src/olap/schema.cpp                             |   7 +
 be/src/runtime/primitive_type.h                    |   6 +-
 be/src/vec/columns/column_dictionary.h             |  11 +-
 be/src/vec/columns/predicate_column.h              |  15 +-
 be/src/vec/functions/like.cpp                      | 158 +--------------------
 be/src/vec/functions/like.h                        |  53 -------
 be/src/vec/olap/block_reader.cpp                   |  11 --
 be/src/vec/olap/block_reader.h                     |   2 -
 14 files changed, 98 insertions(+), 258 deletions(-)

diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h
index 1c77ce8e90..0a86a35990 100644
--- a/be/src/exprs/bloom_filter_func.h
+++ b/be/src/exprs/bloom_filter_func.h
@@ -408,6 +408,7 @@ struct FixedStringFindOp : public StringFindOp {
         const auto* value = reinterpret_cast<const StringRef*>(input_data);
         int64_t size = value->size;
         const char* data = value->data;
+        // CHAR type may pad the tail with \0, need to trim
         while (size > 0 && data[size - 1] == '\0') {
             size--;
         }
diff --git a/be/src/exprs/create_predicate_function.h 
b/be/src/exprs/create_predicate_function.h
index aefe8e3f03..7d89141c44 100644
--- a/be/src/exprs/create_predicate_function.h
+++ b/be/src/exprs/create_predicate_function.h
@@ -19,10 +19,12 @@
 
 #include "exprs/hybrid_set.h"
 #include "exprs/minmax_predicate.h"
+#include "function_filter.h"
 #include "olap/bitmap_filter_predicate.h"
 #include "olap/bloom_filter_predicate.h"
 #include "olap/column_predicate.h"
 #include "olap/in_list_predicate.h"
+#include "olap/like_column_predicate.h"
 #include "runtime/define_primitive_type.h"
 
 namespace doris {
@@ -256,6 +258,24 @@ ColumnPredicate* create_olap_column_predicate(uint32_t 
column_id,
                                                                 
column->length());
 }
 
+template <PrimitiveType PT>
+ColumnPredicate* create_olap_column_predicate(uint32_t column_id,
+                                              const 
std::shared_ptr<FunctionFilter>& filter, int,
+                                              const TabletColumn* column = 
nullptr) {
+    // currently only support like predicate
+    if constexpr (PT == TYPE_CHAR || PT == TYPE_VARCHAR || PT == TYPE_STRING) {
+        if constexpr (PT == TYPE_CHAR) {
+            return new LikeColumnPredicate<TYPE_CHAR>(filter->_opposite, 
column_id, filter->_fn_ctx,
+                                                      filter->_string_param);
+        } else {
+            return new LikeColumnPredicate<TYPE_STRING>(filter->_opposite, 
column_id,
+                                                        filter->_fn_ctx, 
filter->_string_param);
+        }
+    } else {
+        return nullptr;
+    }
+}
+
 template <typename T>
 ColumnPredicate* create_column_predicate(uint32_t column_id, const 
std::shared_ptr<T>& filter,
                                          FieldType type, int be_exec_version,
diff --git a/be/src/olap/like_column_predicate.cpp 
b/be/src/olap/like_column_predicate.cpp
index ef1374d445..1d20104ee5 100644
--- a/be/src/olap/like_column_predicate.cpp
+++ b/be/src/olap/like_column_predicate.cpp
@@ -22,29 +22,37 @@
 #include "vec/columns/columns_number.h"
 #include "vec/columns/predicate_column.h"
 #include "vec/common/string_ref.h"
+#include "vec/functions/like.h"
 
 namespace doris {
 
-LikeColumnPredicate::LikeColumnPredicate(bool opposite, uint32_t column_id,
-                                         doris::FunctionContext* fn_ctx, 
doris::StringRef val)
+template <PrimitiveType T>
+LikeColumnPredicate<T>::LikeColumnPredicate(bool opposite, uint32_t column_id,
+                                            doris::FunctionContext* fn_ctx, 
doris::StringRef val)
         : ColumnPredicate(column_id, opposite), pattern(val) {
+    static_assert(T == TYPE_VARCHAR || T == TYPE_CHAR || T == TYPE_STRING,
+                  "LikeColumnPredicate only supports the following types: 
TYPE_VARCHAR, TYPE_CHAR, "
+                  "TYPE_STRING");
     _state = reinterpret_cast<StateType*>(
             fn_ctx->get_function_state(doris::FunctionContext::THREAD_LOCAL));
     _state->search_state.clone(_like_state);
 }
 
-void LikeColumnPredicate::evaluate_vec(const vectorized::IColumn& column, 
uint16_t size,
-                                       bool* flags) const {
+template <PrimitiveType T>
+void LikeColumnPredicate<T>::evaluate_vec(const vectorized::IColumn& column, 
uint16_t size,
+                                          bool* flags) const {
     _evaluate_vec<false>(column, size, flags);
 }
 
-void LikeColumnPredicate::evaluate_and_vec(const vectorized::IColumn& column, 
uint16_t size,
-                                           bool* flags) const {
+template <PrimitiveType T>
+void LikeColumnPredicate<T>::evaluate_and_vec(const vectorized::IColumn& 
column, uint16_t size,
+                                              bool* flags) const {
     _evaluate_vec<true>(column, size, flags);
 }
 
-uint16_t LikeColumnPredicate::evaluate(const vectorized::IColumn& column, 
uint16_t* sel,
-                                       uint16_t size) const {
+template <PrimitiveType T>
+uint16_t LikeColumnPredicate<T>::evaluate(const vectorized::IColumn& column, 
uint16_t* sel,
+                                          uint16_t size) const {
     uint16_t new_size = 0;
     if (column.is_nullable()) {
         auto* nullable_col = 
vectorized::check_and_get_column<vectorized::ColumnNullable>(column);
@@ -83,18 +91,18 @@ uint16_t LikeColumnPredicate::evaluate(const 
vectorized::IColumn& column, uint16
                 }
             }
         } else {
-            auto* str_col =
-                    
vectorized::check_and_get_column<vectorized::PredicateColumnType<TYPE_STRING>>(
-                            nested_col);
+            auto* str_col = 
vectorized::check_and_get_column<vectorized::PredicateColumnType<T>>(
+                    nested_col);
             if (!nullable_col->has_null()) {
                 vectorized::ColumnUInt8::Container res(size, 0);
-                (_state->predicate_like_function)(
-                        
const_cast<vectorized::LikeSearchState*>(&_like_state), *str_col, pattern,
-                        res, sel, size);
                 for (uint16_t i = 0; i != size; i++) {
                     uint16_t idx = sel[i];
                     sel[new_size] = idx;
-                    new_size += _opposite ^ res[i];
+                    unsigned char flag = 0;
+                    (_state->scalar_function)(
+                            
const_cast<vectorized::LikeSearchState*>(&_like_state),
+                            str_col->get_data_at(idx), pattern, &flag);
+                    new_size += _opposite ^ flag;
                 }
             } else {
                 for (uint16_t i = 0; i != size; i++) {
@@ -105,7 +113,7 @@ uint16_t LikeColumnPredicate::evaluate(const 
vectorized::IColumn& column, uint16
                         continue;
                     }
 
-                    StringRef cell_value = str_col->get_data()[idx];
+                    StringRef cell_value = str_col->get_data_at(idx);
                     unsigned char flag = 0;
                     (_state->scalar_function)(
                             
const_cast<vectorized::LikeSearchState*>(&_like_state),
@@ -130,21 +138,24 @@ uint16_t LikeColumnPredicate::evaluate(const 
vectorized::IColumn& column, uint16
                 new_size += _opposite ^ flag;
             }
         } else {
-            auto* str_col =
-                    
vectorized::check_and_get_column<vectorized::PredicateColumnType<TYPE_STRING>>(
-                            column);
+            const vectorized::PredicateColumnType<T>* str_col =
+                    
vectorized::check_and_get_column<vectorized::PredicateColumnType<T>>(column);
+
             vectorized::ColumnUInt8::Container res(size, 0);
-            (_state->predicate_like_function)(
-                    const_cast<vectorized::LikeSearchState*>(&_like_state), 
*str_col, pattern, res,
-                    sel, size);
             for (uint16_t i = 0; i != size; i++) {
                 uint16_t idx = sel[i];
                 sel[new_size] = idx;
-                new_size += _opposite ^ res[i];
+                unsigned char flag = 0;
+                
(_state->scalar_function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
+                                          str_col->get_data_at(idx), pattern, 
&flag);
+                new_size += _opposite ^ flag;
             }
         }
     }
     return new_size;
 }
 
+template class LikeColumnPredicate<TYPE_CHAR>;
+template class LikeColumnPredicate<TYPE_STRING>;
+
 } //namespace doris
diff --git a/be/src/olap/like_column_predicate.h 
b/be/src/olap/like_column_predicate.h
index f97ff46453..dca6cf218a 100644
--- a/be/src/olap/like_column_predicate.h
+++ b/be/src/olap/like_column_predicate.h
@@ -47,6 +47,7 @@ namespace segment_v2 {
 class BitmapIndexIterator;
 } // namespace segment_v2
 
+template <PrimitiveType T>
 class LikeColumnPredicate : public ColumnPredicate {
 public:
     LikeColumnPredicate(bool opposite, uint32_t column_id, 
doris::FunctionContext* fn_ctx,
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index 040f98e7e2..17ead3f155 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -501,7 +501,8 @@ Status TabletReader::_init_conditions_param(const 
ReaderParams& read_params) {
 
     // Function filter push down to storage engine
     auto is_like_predicate = [](ColumnPredicate* _pred) {
-        if (dynamic_cast<LikeColumnPredicate*>(_pred)) {
+        if (dynamic_cast<LikeColumnPredicate<TYPE_CHAR>*>(_pred) != nullptr ||
+            dynamic_cast<LikeColumnPredicate<TYPE_STRING>*>(_pred) != nullptr) 
{
             return true;
         }
 
@@ -594,10 +595,10 @@ ColumnPredicate* TabletReader::_parse_to_predicate(const 
FunctionFilter& functio
     if (index < 0) {
         return nullptr;
     }
-
-    // currently only support like predicate
-    return new LikeColumnPredicate(function_filter._opposite, index, 
function_filter._fn_ctx,
-                                   function_filter._string_param);
+    const TabletColumn& column = _tablet_schema->column(index);
+    return create_column_predicate(index, 
std::make_shared<FunctionFilter>(function_filter),
+                                   column.type(), 
_reader_context.runtime_state->be_exec_version(),
+                                   &column);
 }
 
 Status TabletReader::_init_delete_condition(const ReaderParams& read_params) {
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp 
b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 86d5d1ad85..d5e0321455 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -682,7 +682,8 @@ bool 
SegmentIterator::_check_apply_by_inverted_index(ColumnPredicate* pred, bool
     }
 
     // Function filter no apply inverted index
-    if (dynamic_cast<LikeColumnPredicate*>(pred)) {
+    if (dynamic_cast<LikeColumnPredicate<TYPE_CHAR>*>(pred) != nullptr ||
+        dynamic_cast<LikeColumnPredicate<TYPE_STRING>*>(pred) != nullptr) {
         return false;
     }
 
diff --git a/be/src/olap/schema.cpp b/be/src/olap/schema.cpp
index 95f6a47a70..d465db4f1d 100644
--- a/be/src/olap/schema.cpp
+++ b/be/src/olap/schema.cpp
@@ -171,6 +171,13 @@ vectorized::IColumn::MutablePtr 
Schema::get_predicate_column_ptr(const Field& fi
         ptr = doris::vectorized::PredicateColumnType<TYPE_DATETIME>::create();
         break;
     case FieldType::OLAP_FIELD_TYPE_CHAR:
+        if (config::enable_low_cardinality_optimize && reader_type == 
ReaderType::READER_QUERY) {
+            ptr = 
doris::vectorized::ColumnDictionary<doris::vectorized::Int32>::create(
+                    field.type());
+        } else {
+            ptr = doris::vectorized::PredicateColumnType<TYPE_CHAR>::create();
+        }
+        break;
     case FieldType::OLAP_FIELD_TYPE_VARCHAR:
     case FieldType::OLAP_FIELD_TYPE_STRING:
         if (config::enable_low_cardinality_optimize && reader_type == 
ReaderType::READER_QUERY) {
diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h
index 5a4c05ef40..07ef91f11c 100644
--- a/be/src/runtime/primitive_type.h
+++ b/be/src/runtime/primitive_type.h
@@ -91,6 +91,10 @@ constexpr bool is_string_type(PrimitiveType type) {
     return type == TYPE_CHAR || type == TYPE_VARCHAR || type == TYPE_STRING;
 }
 
+constexpr bool is_variant_string_type(PrimitiveType type) {
+    return type == TYPE_VARCHAR || type == TYPE_STRING;
+}
+
 constexpr bool is_float_or_double(PrimitiveType type) {
     return type == TYPE_FLOAT || type == TYPE_DOUBLE;
 }
@@ -110,7 +114,7 @@ TTypeDesc gen_type_desc(const TPrimitiveType::type val);
 TTypeDesc gen_type_desc(const TPrimitiveType::type val, const std::string& 
name);
 
 template <PrimitiveType type>
-constexpr PrimitiveType PredicateEvaluateType = is_string_type(type) ? 
TYPE_STRING : type;
+constexpr PrimitiveType PredicateEvaluateType = is_variant_string_type(type) ? 
TYPE_STRING : type;
 
 template <PrimitiveType type>
 struct PrimitiveTypeTraits;
diff --git a/be/src/vec/columns/column_dictionary.h 
b/be/src/vec/columns/column_dictionary.h
index e89bc64868..b6db5af7e9 100644
--- a/be/src/vec/columns/column_dictionary.h
+++ b/be/src/vec/columns/column_dictionary.h
@@ -318,7 +318,16 @@ public:
         if (is_dict_sorted() && !is_dict_code_converted()) {
             convert_dict_codes_if_necessary();
         }
-        auto res = vectorized::PredicateColumnType<TYPE_STRING>::create();
+        // if type is OLAP_FIELD_TYPE_CHAR, we need to construct TYPE_CHAR 
PredicateColumnType,
+        // because the string length will different from varchar and string 
which needed to be processed after.
+        auto create_column = [this]() -> MutableColumnPtr {
+            if (_type == FieldType::OLAP_FIELD_TYPE_CHAR) {
+                return vectorized::PredicateColumnType<TYPE_CHAR>::create();
+            }
+            return vectorized::PredicateColumnType<TYPE_STRING>::create();
+        };
+
+        auto res = create_column();
         res->reserve(_codes.capacity());
         for (size_t i = 0; i < _codes.size(); ++i) {
             auto& code = reinterpret_cast<T&>(_codes[i]);
diff --git a/be/src/vec/columns/predicate_column.h 
b/be/src/vec/columns/predicate_column.h
index fd7548c9f5..21e503817c 100644
--- a/be/src/vec/columns/predicate_column.h
+++ b/be/src/vec/columns/predicate_column.h
@@ -43,12 +43,11 @@ class PredicateColumnType final : public COWHelper<IColumn, 
PredicateColumnType<
 private:
     PredicateColumnType() {}
     PredicateColumnType(const size_t n) : data(n) {}
+    PredicateColumnType(const PredicateColumnType& src) : 
data(src.data.begin(), src.data.end()) {}
     friend class COWHelper<IColumn, PredicateColumnType<Type>>;
     using T = typename PredicatePrimitiveTypeTraits<Type>::PredicateFieldType;
     using ColumnType = typename PrimitiveTypeTraits<Type>::ColumnType;
 
-    PredicateColumnType(const PredicateColumnType& src) : 
data(src.data.begin(), src.data.end()) {}
-
     uint64_t get_date_at(uint16_t idx) {
         const T val = data[idx];
         const char* val_ptr = reinterpret_cast<const char*>(&val);
@@ -154,8 +153,16 @@ public:
 
     size_t size() const override { return data.size(); }
 
-    [[noreturn]] StringRef get_data_at(size_t n) const override {
-        LOG(FATAL) << "get_data_at not supported in PredicateColumnType";
+    StringRef get_data_at(size_t n) const override {
+        if constexpr (std::is_same_v<T, StringRef>) {
+            auto res = reinterpret_cast<const StringRef&>(data[n]);
+            if constexpr (Type == TYPE_CHAR) {
+                res.size = strnlen(res.data, res.size);
+            }
+            return res;
+        } else {
+            LOG(FATAL) << "should not call get_data_at in predicate column 
except for string type";
+        }
     }
 
     void insert_from(const IColumn& src, size_t n) override {
diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp
index 1a83dc8a87..8851b777fe 100644
--- a/be/src/vec/functions/like.cpp
+++ b/be/src/vec/functions/like.cpp
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "vec/functions/like.h"
+#include "like.h"
 
 #include <fmt/format.h>
 #include <hs/hs_compile.h>
@@ -148,68 +148,6 @@ Status 
FunctionLikeBase::constant_substring_fn(LikeSearchState* state, const Col
     return Status::OK();
 }
 
-Status FunctionLikeBase::constant_allpass_fn_predicate(LikeSearchState* state,
-                                                       const 
PredicateColumnType<TYPE_STRING>& val,
-                                                       const StringRef& 
pattern,
-                                                       ColumnUInt8::Container& 
result,
-                                                       const uint16_t* sel, 
size_t sz) {
-    for (size_t i = 0; i < sz; i++) {
-        result[i] = 1;
-    }
-    return Status::OK();
-}
-
-Status FunctionLikeBase::constant_starts_with_fn_predicate(
-        LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringRef& pattern, ColumnUInt8::Container& result, const 
uint16_t* sel, size_t sz) {
-    auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
-    for (size_t i = 0; i < sz; i++) {
-        result[i] = (data_ptr[sel[i]].size >= state->search_string_sv.size) &&
-                    (state->search_string_sv ==
-                     data_ptr[sel[i]].substring(0, 
state->search_string_sv.size));
-    }
-    return Status::OK();
-}
-
-Status FunctionLikeBase::constant_ends_with_fn_predicate(
-        LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringRef& pattern, ColumnUInt8::Container& result, const 
uint16_t* sel, size_t sz) {
-    auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
-    for (size_t i = 0; i < sz; i++) {
-        result[i] =
-                (data_ptr[sel[i]].size >= state->search_string_sv.size) &&
-                (state->search_string_sv ==
-                 data_ptr[sel[i]].substring(data_ptr[sel[i]].size - 
state->search_string_sv.size,
-                                            state->search_string_sv.size));
-    }
-    return Status::OK();
-}
-
-Status FunctionLikeBase::constant_equals_fn_predicate(LikeSearchState* state,
-                                                      const 
PredicateColumnType<TYPE_STRING>& val,
-                                                      const StringRef& pattern,
-                                                      ColumnUInt8::Container& 
result,
-                                                      const uint16_t* sel, 
size_t sz) {
-    auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
-    for (size_t i = 0; i < sz; i++) {
-        result[i] = (data_ptr[sel[i]] == state->search_string_sv);
-    }
-    return Status::OK();
-}
-
-Status FunctionLikeBase::constant_substring_fn_predicate(
-        LikeSearchState* state, const PredicateColumnType<TYPE_STRING>& val,
-        const StringRef& pattern, ColumnUInt8::Container& result, const 
uint16_t* sel, size_t sz) {
-    auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
-    for (size_t i = 0; i < sz; i++) {
-        if (state->search_string_sv.size == 0) {
-            result[i] = true;
-        }
-        result[i] = state->substring_pattern.search(data_ptr[sel[i]]) != -1;
-    }
-    return Status::OK();
-}
-
 Status FunctionLikeBase::constant_allpass_fn_scalar(LikeSearchState* state, 
const StringRef& val,
                                                     const StringRef& pattern,
                                                     unsigned char* result) {
@@ -351,75 +289,6 @@ Status FunctionLikeBase::regexp_fn(LikeSearchState* state, 
const ColumnString& v
     return Status::OK();
 }
 
-Status FunctionLikeBase::constant_regex_fn_predicate(LikeSearchState* state,
-                                                     const 
PredicateColumnType<TYPE_STRING>& val,
-                                                     const StringRef& pattern,
-                                                     ColumnUInt8::Container& 
result,
-                                                     const uint16_t* sel, 
size_t sz) {
-    auto data_ptr = reinterpret_cast<const StringRef*>(val.get_data().data());
-
-    if (state->hs_database) { // use hyperscan
-        for (size_t i = 0; i < sz; i++) {
-            auto ret = hs_scan(state->hs_database.get(), data_ptr[sel[i]].data,
-                               data_ptr[sel[i]].size, 0, 
state->hs_scratch.get(),
-                               
doris::vectorized::LikeSearchState::hs_match_handler,
-                               (void*)(result.data() + i));
-            if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
-                return Status::RuntimeError(fmt::format("hyperscan error: {}", 
ret));
-            }
-        }
-    } else { // fallback to re2
-        for (size_t i = 0; i < sz; i++) {
-            *(result.data() + i) = RE2::PartialMatch(
-                    re2::StringPiece(data_ptr[sel[i]].data, 
data_ptr[sel[i]].size),
-                    *state->regex.get());
-        }
-    }
-
-    return Status::OK();
-}
-
-Status FunctionLikeBase::regexp_fn_predicate(LikeSearchState* state,
-                                             const 
PredicateColumnType<TYPE_STRING>& val,
-                                             const StringRef& pattern,
-                                             ColumnUInt8::Container& result, 
const uint16_t* sel,
-                                             size_t sz) {
-    std::string re_pattern(pattern.data, pattern.size);
-
-    hs_database_t* database = nullptr;
-    hs_scratch_t* scratch = nullptr;
-    if (hs_prepare(nullptr, re_pattern.c_str(), &database, &scratch).ok()) { 
// use hyperscan
-        auto data_ptr = reinterpret_cast<const 
StringRef*>(val.get_data().data());
-        for (size_t i = 0; i < sz; i++) {
-            auto ret = hs_scan(database, data_ptr[sel[i]].data, 
data_ptr[sel[i]].size, 0, scratch,
-                               
doris::vectorized::LikeSearchState::hs_match_handler,
-                               (void*)(result.data() + i));
-            if (ret != HS_SUCCESS && ret != HS_SCAN_TERMINATED) {
-                return Status::RuntimeError(fmt::format("hyperscan error: {}", 
ret));
-            }
-        }
-
-        hs_free_scratch(scratch);
-        hs_free_database(database);
-    } else { // fallback to re2
-        RE2::Options opts;
-        opts.set_never_nl(false);
-        opts.set_dot_nl(true);
-        re2::RE2 re(re_pattern, opts);
-        if (re.ok()) {
-            auto data_ptr = reinterpret_cast<const 
StringRef*>(val.get_data().data());
-            for (size_t i = 0; i < sz; i++) {
-                *(result.data() + i) = RE2::PartialMatch(
-                        re2::StringPiece(data_ptr[sel[i]].data, 
data_ptr[sel[i]].size), re);
-            }
-        } else {
-            return Status::RuntimeError("Invalid pattern: {}", 
pattern.debug_string());
-        }
-    }
-
-    return Status::OK();
-}
-
 // hyperscan compile expression to database and allocate scratch space
 Status FunctionLikeBase::hs_prepare(FunctionContext* context, const char* 
expression,
                                     hs_database_t** database, hs_scratch_t** 
scratch) {
@@ -559,17 +428,6 @@ Status FunctionLike::like_fn(LikeSearchState* state, const 
ColumnString& val,
     return regexp_fn(state, val, {re_pattern.c_str(), re_pattern.size()}, 
result);
 }
 
-Status FunctionLike::like_fn_predicate(LikeSearchState* state,
-                                       const PredicateColumnType<TYPE_STRING>& 
val,
-                                       const StringRef& pattern, 
ColumnUInt8::Container& result,
-                                       uint16_t* sel, size_t sz) {
-    std::string re_pattern;
-    convert_like_pattern(state, std::string(pattern.data, pattern.size), 
&re_pattern);
-
-    return regexp_fn_predicate(state, val, {re_pattern.c_str(), 
re_pattern.size()}, result, sel,
-                               sz);
-}
-
 Status FunctionLike::like_fn_scalar(LikeSearchState* state, const StringRef& 
val,
                                     const StringRef& pattern, unsigned char* 
result) {
     std::string re_pattern;
@@ -680,7 +538,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
     std::shared_ptr<LikeState> state = std::make_shared<LikeState>();
     context->set_function_state(scope, state);
     state->function = like_fn;
-    state->predicate_like_function = like_fn_predicate;
     state->scalar_function = like_fn_scalar;
     if (context->is_col_constant(1)) {
         const auto pattern_col = context->get_constant_col(1)->column_ptr;
@@ -693,7 +550,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
         if (!pattern_str.empty() && RE2::FullMatch(pattern_str, 
LIKE_ALLPASS_RE)) {
             state->search_state.set_search_string("");
             state->function = constant_allpass_fn;
-            state->predicate_like_function = constant_allpass_fn_predicate;
             state->scalar_function = constant_allpass_fn_scalar;
         } else if (pattern_str.empty() ||
                    RE2::FullMatch(pattern_str, LIKE_EQUALS_RE, 
&search_string)) {
@@ -709,7 +565,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
             }
             state->search_state.set_search_string(search_string);
             state->function = constant_equals_fn;
-            state->predicate_like_function = constant_equals_fn_predicate;
             state->scalar_function = constant_equals_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, LIKE_STARTS_WITH_RE, 
&search_string)) {
             if (VLOG_DEBUG_IS_ON) {
@@ -724,7 +579,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
             }
             state->search_state.set_search_string(search_string);
             state->function = constant_starts_with_fn;
-            state->predicate_like_function = constant_starts_with_fn_predicate;
             state->scalar_function = constant_starts_with_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, LIKE_ENDS_WITH_RE, 
&search_string)) {
             if (VLOG_DEBUG_IS_ON) {
@@ -739,7 +593,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
             }
             state->search_state.set_search_string(search_string);
             state->function = constant_ends_with_fn;
-            state->predicate_like_function = constant_ends_with_fn_predicate;
             state->scalar_function = constant_ends_with_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, LIKE_SUBSTRING_RE, 
&search_string)) {
             if (VLOG_DEBUG_IS_ON) {
@@ -754,7 +607,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
             }
             state->search_state.set_search_string(search_string);
             state->function = constant_substring_fn;
-            state->predicate_like_function = constant_substring_fn_predicate;
             state->scalar_function = constant_substring_fn_scalar;
         } else {
             std::string re_pattern;
@@ -787,7 +639,6 @@ Status FunctionLike::open(FunctionContext* context, 
FunctionContext::FunctionSta
             }
 
             state->function = constant_regex_fn;
-            state->predicate_like_function = constant_regex_fn_predicate;
             state->scalar_function = constant_regex_fn_scalar;
         }
     }
@@ -801,7 +652,6 @@ Status FunctionRegexp::open(FunctionContext* context, 
FunctionContext::FunctionS
     std::shared_ptr<LikeState> state = std::make_shared<LikeState>();
     context->set_function_state(scope, state);
     state->function = regexp_fn;
-    state->predicate_like_function = regexp_fn_predicate;
     state->scalar_function = regexp_fn_scalar;
     if (context->is_col_constant(1)) {
         const auto pattern_col = context->get_constant_col(1)->column_ptr;
@@ -812,27 +662,22 @@ Status FunctionRegexp::open(FunctionContext* context, 
FunctionContext::FunctionS
         if (RE2::FullMatch(pattern_str, ALLPASS_RE)) {
             state->search_state.set_search_string("");
             state->function = constant_allpass_fn;
-            state->predicate_like_function = constant_allpass_fn_predicate;
             state->scalar_function = constant_allpass_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, EQUALS_RE, &search_string)) {
             state->search_state.set_search_string(search_string);
             state->function = constant_equals_fn;
-            state->predicate_like_function = constant_equals_fn_predicate;
             state->scalar_function = constant_equals_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, STARTS_WITH_RE, 
&search_string)) {
             state->search_state.set_search_string(search_string);
             state->function = constant_starts_with_fn;
-            state->predicate_like_function = constant_starts_with_fn_predicate;
             state->scalar_function = constant_starts_with_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, ENDS_WITH_RE, &search_string)) {
             state->search_state.set_search_string(search_string);
             state->function = constant_ends_with_fn;
-            state->predicate_like_function = constant_ends_with_fn_predicate;
             state->scalar_function = constant_ends_with_fn_scalar;
         } else if (RE2::FullMatch(pattern_str, SUBSTRING_RE, &search_string)) {
             state->search_state.set_search_string(search_string);
             state->function = constant_substring_fn;
-            state->predicate_like_function = constant_substring_fn_predicate;
             state->scalar_function = constant_substring_fn_scalar;
         } else {
             hs_database_t* database = nullptr;
@@ -855,7 +700,6 @@ Status FunctionRegexp::open(FunctionContext* context, 
FunctionContext::FunctionS
                 }
             }
             state->function = constant_regex_fn;
-            state->predicate_like_function = constant_regex_fn_predicate;
             state->scalar_function = constant_regex_fn_scalar;
         }
     }
diff --git a/be/src/vec/functions/like.h b/be/src/vec/functions/like.h
index 07309be5b7..3726518a9b 100644
--- a/be/src/vec/functions/like.h
+++ b/be/src/vec/functions/like.h
@@ -115,18 +115,12 @@ struct LikeSearchState {
 using LikeFn = std::function<doris::Status(LikeSearchState*, const 
ColumnString&, const StringRef&,
                                            ColumnUInt8::Container&)>;
 
-using LikePredicateFn = std::function<doris::Status(
-        LikeSearchState*, const PredicateColumnType<TYPE_STRING>&, const 
StringRef&,
-        ColumnUInt8::Container&, uint16_t* sel, size_t sz)>;
-
 using ScalarLikeFn = std::function<doris::Status(LikeSearchState*, const 
StringRef&,
                                                  const StringRef&, unsigned 
char*)>;
 
 struct LikeState {
     LikeSearchState search_state;
     LikeFn function;
-    // Two functions below are used only for predicate.
-    LikePredicateFn predicate_like_function;
     ScalarLikeFn scalar_function;
 };
 
@@ -173,48 +167,6 @@ protected:
     static Status regexp_fn(LikeSearchState* state, const ColumnString& val,
                             const StringRef& pattern, ColumnUInt8::Container& 
result);
 
-    // These functions below are used only for predicate.
-    static Status constant_regex_fn_predicate(LikeSearchState* state,
-                                              const 
PredicateColumnType<TYPE_STRING>& val,
-                                              const StringRef& pattern,
-                                              ColumnUInt8::Container& result, 
const uint16_t* sel,
-                                              size_t sz);
-
-    static Status regexp_fn_predicate(LikeSearchState* state,
-                                      const PredicateColumnType<TYPE_STRING>& 
val,
-                                      const StringRef& pattern, 
ColumnUInt8::Container& result,
-                                      const uint16_t* sel, size_t sz);
-
-    static Status constant_allpass_fn_predicate(LikeSearchState* state,
-                                                const 
PredicateColumnType<TYPE_STRING>& val,
-                                                const StringRef& pattern,
-                                                ColumnUInt8::Container& 
result, const uint16_t* sel,
-                                                size_t sz);
-
-    static Status constant_starts_with_fn_predicate(LikeSearchState* state,
-                                                    const 
PredicateColumnType<TYPE_STRING>& val,
-                                                    const StringRef& pattern,
-                                                    ColumnUInt8::Container& 
result,
-                                                    const uint16_t* sel, 
size_t sz);
-
-    static Status constant_ends_with_fn_predicate(LikeSearchState* state,
-                                                  const 
PredicateColumnType<TYPE_STRING>& val,
-                                                  const StringRef& pattern,
-                                                  ColumnUInt8::Container& 
result,
-                                                  const uint16_t* sel, size_t 
sz);
-
-    static Status constant_equals_fn_predicate(LikeSearchState* state,
-                                               const 
PredicateColumnType<TYPE_STRING>& val,
-                                               const StringRef& pattern,
-                                               ColumnUInt8::Container& result, 
const uint16_t* sel,
-                                               size_t sz);
-
-    static Status constant_substring_fn_predicate(LikeSearchState* state,
-                                                  const 
PredicateColumnType<TYPE_STRING>& val,
-                                                  const StringRef& pattern,
-                                                  ColumnUInt8::Container& 
result,
-                                                  const uint16_t* sel, size_t 
sz);
-
     static Status constant_allpass_fn_scalar(LikeSearchState* state, const 
StringRef& val,
                                              const StringRef& pattern, 
unsigned char* result);
 
@@ -257,11 +209,6 @@ private:
     static Status like_fn(LikeSearchState* state, const ColumnString& val, 
const StringRef& pattern,
                           ColumnUInt8::Container& result);
 
-    static Status like_fn_predicate(LikeSearchState* state,
-                                    const PredicateColumnType<TYPE_STRING>& 
val,
-                                    const StringRef& pattern, 
ColumnUInt8::Container& result,
-                                    uint16_t* sel, size_t sz);
-
     static Status like_fn_scalar(LikeSearchState* state, const StringRef& val,
                                  const StringRef& pattern, unsigned char* 
result);
 
diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp
index 80fffd4401..2342c9faf9 100644
--- a/be/src/vec/olap/block_reader.cpp
+++ b/be/src/vec/olap/block_reader.cpp
@@ -503,15 +503,4 @@ bool BlockReader::_get_next_row_same() {
     }
 }
 
-ColumnPredicate* BlockReader::_parse_to_predicate(const FunctionFilter& 
function_filter) {
-    int32_t index = _tablet_schema->field_index(function_filter._col_name);
-    if (index < 0) {
-        return nullptr;
-    }
-
-    // currently only support like predicate
-    return new LikeColumnPredicate(function_filter._opposite, index, 
function_filter._fn_ctx,
-                                   function_filter._string_param);
-}
-
 } // namespace doris::vectorized
diff --git a/be/src/vec/olap/block_reader.h b/be/src/vec/olap/block_reader.h
index 0fe188419e..8184409945 100644
--- a/be/src/vec/olap/block_reader.h
+++ b/be/src/vec/olap/block_reader.h
@@ -57,8 +57,6 @@ public:
         return _vcollect_iter.update_profile(profile);
     }
 
-    ColumnPredicate* _parse_to_predicate(const FunctionFilter& 
function_filter) override;
-
 private:
     // Directly read row from rowset and pass to upper caller. No need to do 
aggregation.
     // This is usually used for DUPLICATE KEY tables


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to