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

dataroaring pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit b295cec2fae10f2350b2293ab192c83fdc4bd9d6
Author: zzzxl <33418555+zzzxl1...@users.noreply.github.com>
AuthorDate: Mon Jul 22 17:11:08 2024 +0800

    [feature](inverted index) Add multi_match function (#37722)
    
    ## Proposed changes
    
    1. select * from tbl where multi_match(c1, 'c2, c3, c4',
    'phrase_prefix', 'xxx');
    2. multi_match performs a match_phrase_prefix query on the columns
    specified in the first and second parameters. The value to be searched
    is specified in the fourth parameter.
---
 .../olap/rowset/segment_v2/inverted_index_reader.h |   4 +-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp | 175 ++++++++++++++++---
 be/src/olap/rowset/segment_v2/segment_iterator.h   |  28 +++
 be/src/vec/exprs/vectorized_fn_call.cpp            |  43 ++++-
 be/src/vec/exprs/vectorized_fn_call.h              |   7 +-
 be/src/vec/exprs/vexpr.cpp                         |  37 ++--
 be/src/vec/exprs/vexpr.h                           |  15 +-
 be/src/vec/exprs/vexpr_context.cpp                 |   1 +
 be/src/vec/exprs/vin_predicate.cpp                 |   4 +-
 be/src/vec/exprs/vin_predicate.h                   |   3 +-
 be/src/vec/exprs/vliteral.cpp                      |  21 +++
 be/src/vec/exprs/vliteral.h                        |   2 +
 be/src/vec/exprs/vslot_ref.cpp                     |  17 ++
 be/src/vec/exprs/vslot_ref.h                       |   2 +
 be/src/vec/functions/function.h                    |  36 ++--
 be/src/vec/functions/function_multi_match.cpp      | 187 +++++++++++++++++++++
 be/src/vec/functions/function_multi_match.h        |  70 ++++++++
 be/src/vec/functions/simple_function_factory.h     |   4 +-
 .../doris/catalog/BuiltinScalarFunctions.java      |   4 +-
 .../expressions/functions/scalar/MultiMatch.java   |  73 ++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 +
 gensrc/script/doris_builtins_functions.py          |   5 +
 22 files changed, 671 insertions(+), 72 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h 
b/be/src/olap/rowset/segment_v2/inverted_index_reader.h
index abb600fd2d6..a598ccc9ee7 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h
+++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h
@@ -73,7 +73,6 @@ class InvertedIndexIterator;
 class InvertedIndexQueryCacheHandle;
 class InvertedIndexFileReader;
 struct InvertedIndexQueryInfo;
-
 class InvertedIndexReader : public 
std::enable_shared_from_this<InvertedIndexReader> {
 public:
     explicit InvertedIndexReader(
@@ -153,6 +152,7 @@ protected:
     TabletIndex _index_meta;
     bool _has_null = true;
 };
+using InvertedIndexReaderPtr = std::shared_ptr<InvertedIndexReader>;
 
 class FullTextIndexReader : public InvertedIndexReader {
     ENABLE_FACTORY_CREATOR(FullTextIndexReader);
@@ -379,6 +379,8 @@ public:
     [[nodiscard]] const std::map<string, string>& get_index_properties() const;
     [[nodiscard]] bool has_null() { return _reader->has_null(); };
 
+    const InvertedIndexReaderPtr& reader() { return _reader; }
+
 private:
     OlapReaderStatistics* _stats = nullptr;
     RuntimeState* _runtime_state = nullptr;
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp 
b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 09ad056189a..fdc6e017dce 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -315,6 +315,7 @@ Status SegmentIterator::_init_impl(const 
StorageReadOptions& opts) {
     for (auto& expr : _remaining_conjunct_roots) {
         _calculate_pred_in_remaining_conjunct_root(expr);
     }
+    _calculate_func_in_remaining_conjunct_root();
 
     _column_predicate_info.reset(new ColumnPredicateInfo());
     if (_schema->rowid_col_idx() > 0) {
@@ -560,6 +561,7 @@ Status 
SegmentIterator::_get_row_ranges_by_column_conditions() {
                 }
             }
             _col_preds_except_leafnode_of_andnode.clear();
+            compound_func_exprs.clear();
             // 1. if all conditions in the compound hit the inverted index and 
there are no other expr to handle.
             // 2. then there is no need to generate index_result_column.
             if (_enable_common_expr_pushdown && 
_remaining_conjunct_roots.empty()) {
@@ -814,25 +816,32 @@ Status 
SegmentIterator::_execute_predicates_except_leafnode_of_andnode(
         auto v_literal_expr = 
std::dynamic_pointer_cast<doris::vectorized::VLiteral>(expr);
         _column_predicate_info->query_values.insert(v_literal_expr->value());
     } else if (node_type == TExprNodeType::BINARY_PRED || node_type == 
TExprNodeType::MATCH_PRED ||
-               node_type == TExprNodeType::IN_PRED) {
-        if (node_type == TExprNodeType::MATCH_PRED) {
-            _column_predicate_info->query_op = "match";
-        } else if (node_type == TExprNodeType::IN_PRED) {
-            if (expr->op() == TExprOpcode::type::FILTER_IN) {
-                _column_predicate_info->query_op = "in";
+               node_type == TExprNodeType::IN_PRED || node_type == 
TExprNodeType::FUNCTION_CALL) {
+        std::string result_sign;
+        if (node_type == TExprNodeType::FUNCTION_CALL) {
+            result_sign =
+                    BeConsts::BLOCK_TEMP_COLUMN_PREFIX + 
std::to_string(expr->index_unique_id());
+        } else {
+            if (node_type == TExprNodeType::MATCH_PRED) {
+                _column_predicate_info->query_op = "match";
+            } else if (node_type == TExprNodeType::IN_PRED) {
+                if (expr->op() == TExprOpcode::type::FILTER_IN) {
+                    _column_predicate_info->query_op = "in";
+                } else {
+                    _column_predicate_info->query_op = "not_in";
+                }
             } else {
-                _column_predicate_info->query_op = "not_in";
+                _column_predicate_info->query_op = 
expr->fn().name.function_name;
             }
-        } else {
-            _column_predicate_info->query_op = expr->fn().name.function_name;
+            result_sign = 
_gen_predicate_result_sign(_column_predicate_info.get());
         }
+
         // get child condition result in compound conditions
-        auto pred_result_sign = 
_gen_predicate_result_sign(_column_predicate_info.get());
         _column_predicate_info.reset(new ColumnPredicateInfo());
-        VLOG_DEBUG << "_gen_predicate_result_sign " << pred_result_sign;
-        if (_rowid_result_for_index.count(pred_result_sign) > 0 &&
-            _rowid_result_for_index[pred_result_sign].first) {
-            auto apply_result = 
_rowid_result_for_index[pred_result_sign].second;
+        VLOG_DEBUG << "result_sign " << result_sign;
+        if (_rowid_result_for_index.count(result_sign) > 0 &&
+            _rowid_result_for_index[result_sign].first) {
+            auto apply_result = _rowid_result_for_index[result_sign].second;
             
_pred_except_leafnode_of_andnode_evaluate_result.push_back(apply_result);
         }
     } else if (node_type == TExprNodeType::COMPOUND_PRED) {
@@ -876,7 +885,7 @@ Status SegmentIterator::_execute_compound_fn(const 
std::string& function_name) {
 
 bool SegmentIterator::_can_filter_by_preds_except_leafnode_of_andnode() {
     // no compound predicates push down, so no need to filter
-    if (_col_preds_except_leafnode_of_andnode.size() == 0) {
+    if (_col_preds_except_leafnode_of_andnode.empty() && 
compound_func_exprs.empty()) {
         return false;
     }
     for (auto pred : _col_preds_except_leafnode_of_andnode) {
@@ -890,6 +899,14 @@ bool 
SegmentIterator::_can_filter_by_preds_except_leafnode_of_andnode() {
             return false;
         }
     }
+    for (const auto& func_expr_pair : compound_func_exprs) {
+        const auto& expr = func_expr_pair.first;
+        std::string pred_result_sign =
+                BeConsts::BLOCK_TEMP_COLUMN_PREFIX + 
std::to_string(expr->index_unique_id());
+        if (!_rowid_result_for_index.contains(pred_result_sign)) {
+            return false;
+        }
+    }
     return true;
 }
 
@@ -1006,6 +1023,16 @@ Status 
SegmentIterator::_apply_index_except_leafnode_of_andnode() {
         }
     }
 
+    for (const auto& func_expr_pair : compound_func_exprs) {
+        const auto& expr = func_expr_pair.first;
+        const auto& expr_ctx = func_expr_pair.second;
+        auto result = std::make_shared<roaring::Roaring>();
+        RETURN_IF_ERROR(execute_func_expr(expr, expr_ctx, result));
+        std::string result_sign =
+                BeConsts::BLOCK_TEMP_COLUMN_PREFIX + 
std::to_string(expr->index_unique_id());
+        _rowid_result_for_index.emplace(result_sign, std::make_pair(true, 
std::move(*result)));
+    }
+
     return Status::OK();
 }
 
@@ -1262,18 +1289,6 @@ Status SegmentIterator::_apply_inverted_index() {
     std::vector<ColumnPredicate*> remaining_predicates;
     std::set<const ColumnPredicate*> no_need_to_pass_column_predicate_set;
 
-    // TODO:Comment out this code before introducing range query functionality
-    /*for (const auto& entry : _opts.col_id_to_predicates) {
-        ColumnId column_id = entry.first;
-        auto pred = entry.second;
-        bool continue_apply = true;
-        RETURN_IF_ERROR(_apply_inverted_index_on_block_column_predicate(
-                column_id, pred.get(), no_need_to_pass_column_predicate_set, 
&continue_apply));
-        if (!continue_apply) {
-            break;
-        }
-    }*/
-
     for (auto pred : _col_predicates) {
         if (no_need_to_pass_column_predicate_set.count(pred) > 0) {
             continue;
@@ -1309,6 +1324,23 @@ Status SegmentIterator::_apply_inverted_index() {
         }
     }
 
+    for (const auto& func_expr_pair : no_compound_func_exprs) {
+        const auto& expr = func_expr_pair.first;
+        const auto& expr_ctx = func_expr_pair.second;
+        auto result = std::make_shared<roaring::Roaring>();
+        RETURN_IF_ERROR(execute_func_expr(expr, expr_ctx, result));
+        _row_bitmap &= *result;
+        for (auto it = _remaining_conjunct_roots.begin(); it != 
_remaining_conjunct_roots.end();) {
+            if (*it == expr) {
+                std::erase_if(_common_expr_ctxs_push_down,
+                              [&it](const auto& iter) { return iter->root() == 
*it; });
+                it = _remaining_conjunct_roots.erase(it);
+            } else {
+                ++it;
+            }
+        }
+    }
+
     // add a switch for inverted index filter
     if (_opts.runtime_state &&
         _opts.runtime_state->enable_common_expr_pushdown_for_inverted_index()) 
{
@@ -1443,6 +1475,18 @@ Status SegmentIterator::_init_inverted_index_iterators() 
{
     return Status::OK();
 }
 
+Status SegmentIterator::_init_inverted_index_iterators(ColumnId cid) {
+    if (_inverted_index_iterators[cid] == nullptr) {
+        return _init_single_inverted_index_iterator.call([&] {
+            return _segment->new_inverted_index_iterator(
+                    _opts.tablet_schema->column(cid),
+                    
_segment->_tablet_schema->get_inverted_index(_opts.tablet_schema->column(cid)),
+                    _opts, &_inverted_index_iterators[cid]);
+        });
+    }
+    return Status::OK();
+}
+
 Status SegmentIterator::_lookup_ordinal(const RowCursor& key, bool is_include, 
rowid_t upper_bound,
                                         rowid_t* rowid) {
     if (_segment->_tablet_schema->keys_type() == UNIQUE_KEYS &&
@@ -2875,6 +2919,64 @@ void 
SegmentIterator::_calculate_pred_in_remaining_conjunct_root(
     }
 }
 
+void SegmentIterator::_calculate_func_in_remaining_conjunct_root() {
+    auto hash = [](const vectorized::VExprSPtr& expr) -> std::size_t {
+        return std::hash<std::string>()(expr->expr_name());
+    };
+    auto equal = [](const vectorized::VExprSPtr& lhs, const 
vectorized::VExprSPtr& rhs) -> bool {
+        return lhs->equals(*rhs);
+    };
+
+    uint32_t next_id = 0;
+    std::unordered_map<vectorized::VExprSPtr, uint32_t, decltype(hash), 
decltype(equal)> unique_map(
+            0, hash, equal);
+
+    auto gen_func_unique_id = [&unique_map, &next_id](const 
vectorized::VExprSPtr& expr) {
+        auto it = unique_map.find(expr);
+        if (it != unique_map.end()) {
+            return it->second;
+        } else {
+            unique_map[expr] = ++next_id;
+            return next_id;
+        }
+    };
+
+    for (const auto& root_expr_ctx : _common_expr_ctxs_push_down) {
+        const auto& root_expr = root_expr_ctx->root();
+        if (root_expr == nullptr) {
+            continue;
+        }
+
+        std::stack<std::pair<vectorized::VExprSPtr, bool>> stack;
+        stack.emplace(root_expr, false);
+
+        while (!stack.empty()) {
+            const auto& [expr, has_compound_pred] = stack.top();
+            stack.pop();
+
+            bool current_has_compound_pred =
+                    has_compound_pred || (expr->node_type() == 
TExprNodeType::COMPOUND_PRED);
+
+            if (expr->node_type() == TExprNodeType::FUNCTION_CALL &&
+                expr->can_push_down_to_index()) {
+                expr->set_index_unique_id(gen_func_unique_id(expr));
+                if (current_has_compound_pred) {
+                    compound_func_exprs.emplace_back(expr, root_expr_ctx);
+                } else {
+                    no_compound_func_exprs.emplace_back(expr, root_expr_ctx);
+                }
+            }
+
+            const auto& children = expr->children();
+            for (int32_t i = children.size() - 1; i >= 0; --i) {
+                if (!children[i]->children().empty()) {
+                    stack.emplace(children[i], current_has_compound_pred);
+                }
+            }
+        }
+    }
+}
+
 bool SegmentIterator::_no_need_read_key_data(ColumnId cid, 
vectorized::MutableColumnPtr& column,
                                              size_t nrows_read) {
     if (_opts.runtime_state && 
!_opts.runtime_state->query_options().enable_no_need_read_data_opt) {
@@ -2949,5 +3051,24 @@ bool SegmentIterator::_can_opt_topn_reads() {
     return all_true;
 }
 
+Status SegmentIterator::execute_func_expr(const vectorized::VExprSPtr& expr,
+                                          const vectorized::VExprContextSPtr& 
expr_ctx,
+                                          std::shared_ptr<roaring::Roaring>& 
result) {
+    const auto& expr0 = expr->get_child(0);
+    if (!expr0 || expr0->node_type() != TExprNodeType::SLOT_REF) {
+        return Status::RuntimeError("cannot perform index filtering");
+    }
+
+    FuncExprParams params;
+    auto slot_expr = std::static_pointer_cast<vectorized::VSlotRef>(expr0);
+    params._column_id = _schema->column_id(slot_expr->column_id());
+    params._unique_id = _schema->unique_id(slot_expr->column_id());
+    params._column_name = 
_opts.tablet_schema->column(params._column_id).name();
+    params._segment_iterator = this;
+    params.result = result;
+
+    return expr->eval_inverted_index(expr_ctx.get(), params);
+}
+
 } // namespace segment_v2
 } // namespace doris
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h 
b/be/src/olap/rowset/segment_v2/segment_iterator.h
index c9284e592e4..f163376d95f 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.h
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.h
@@ -107,6 +107,15 @@ struct ColumnPredicateInfo {
     int32_t column_id;
 };
 
+class SegmentIterator;
+struct FuncExprParams {
+    ColumnId _column_id = 0;
+    uint32_t _unique_id = 0;
+    std::string _column_name;
+    SegmentIterator* _segment_iterator = nullptr;
+    std::shared_ptr<roaring::Roaring> result;
+};
+
 class SegmentIterator : public RowwiseIterator {
 public:
     SegmentIterator(std::shared_ptr<Segment> segment, SchemaSPtr schema);
@@ -123,6 +132,8 @@ public:
             std::vector<RowLocation>* block_row_locations) override;
 
     const Schema& schema() const override { return *_schema; }
+    Segment& segment() { return *_segment; }
+    StorageReadOptions& storage_read_options() { return _opts; }
     bool is_lazy_materialization_read() const override { return 
_lazy_materialization_read; }
     uint64_t data_id() const override { return _segment->id(); }
     RowsetId rowset_id() const { return _segment->rowset_id(); }
@@ -142,6 +153,11 @@ public:
         return updated;
     }
 
+    std::vector<std::unique_ptr<InvertedIndexIterator>>& 
inverted_index_iterators() {
+        return _inverted_index_iterators;
+    }
+    [[nodiscard]] Status _init_inverted_index_iterators(ColumnId cid);
+
 private:
     Status _next_batch_internal(vectorized::Block* block);
 
@@ -310,6 +326,7 @@ private:
     bool _check_column_pred_all_push_down(const std::string& column_name, bool 
in_compound = false,
                                           bool is_match = false);
     void _calculate_pred_in_remaining_conjunct_root(const 
vectorized::VExprSPtr& expr);
+    void _calculate_func_in_remaining_conjunct_root();
 
     // todo(wb) remove this method after RowCursor is removed
     void _convert_rowcursor_to_short_key(const RowCursor& key, size_t 
num_keys) {
@@ -393,6 +410,10 @@ private:
     bool _check_all_predicates_passed_inverted_index_for_column(ColumnId cid,
                                                                 bool 
default_return = false);
 
+    Status execute_func_expr(const vectorized::VExprSPtr& expr,
+                             const vectorized::VExprContextSPtr& expr_ctx,
+                             std::shared_ptr<roaring::Roaring>& result);
+
     class BitmapRangeIterator;
     class BackwardBitmapRangeIterator;
 
@@ -459,6 +480,11 @@ private:
     // make a copy of `_opts.column_predicates` in order to make local changes
     std::vector<ColumnPredicate*> _col_predicates;
     std::vector<ColumnPredicate*> _col_preds_except_leafnode_of_andnode;
+
+    using FuncExprPair = std::pair<vectorized::VExprSPtr, 
vectorized::VExprContextSPtr>;
+    std::vector<FuncExprPair> no_compound_func_exprs;
+    std::vector<FuncExprPair> compound_func_exprs;
+
     vectorized::VExprContextSPtrs _common_expr_ctxs_push_down;
     bool _enable_common_expr_pushdown = false;
     std::vector<vectorized::VExprSPtr> _remaining_conjunct_roots;
@@ -505,6 +531,8 @@ private:
 
     std::unordered_map<int, std::unordered_map<std::string, bool>>
             _column_predicate_inverted_index_status;
+
+    DorisCallOnce<Status> _init_single_inverted_index_iterator;
 };
 
 } // namespace segment_v2
diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp 
b/be/src/vec/exprs/vectorized_fn_call.cpp
index d1ba1c2c746..7be01469db7 100644
--- a/be/src/vec/exprs/vectorized_fn_call.cpp
+++ b/be/src/vec/exprs/vectorized_fn_call.cpp
@@ -119,8 +119,7 @@ Status VectorizedFnCall::prepare(RuntimeState* state, const 
RowDescriptor& desc,
     }
     VExpr::register_function_context(state, context);
     _function_name = _fn.name.function_name;
-    _can_fast_execute = _function->can_fast_execute() && _children.size() == 2 
&&
-                        _children[0]->is_slot_ref() && 
_children[1]->is_literal();
+    _can_fast_execute = can_fast_execute();
     _prepare_finished = true;
     return Status::OK();
 }
@@ -247,4 +246,44 @@ std::string VectorizedFnCall::debug_string(const 
std::vector<VectorizedFnCall*>&
     out << "]";
     return out.str();
 }
+
+bool VectorizedFnCall::can_push_down_to_index() const {
+    return _function->can_push_down_to_index();
+}
+
+bool VectorizedFnCall::can_fast_execute() const {
+    auto function_name = _function->get_name();
+    if (function_name == "eq" || function_name == "ne" || function_name == 
"lt" ||
+        function_name == "gt" || function_name == "le" || function_name == 
"ge") {
+        if (_children.size() == 2 && _children[0]->is_slot_ref() && 
_children[1]->is_literal()) {
+            return true;
+        }
+    }
+    return _function->can_push_down_to_index();
+}
+
+Status VectorizedFnCall::eval_inverted_index(VExprContext* context,
+                                             segment_v2::FuncExprParams& 
params) {
+    return 
_function->eval_inverted_index(context->fn_context(_fn_context_index), params);
+}
+
+bool VectorizedFnCall::equals(const VExpr& other) {
+    const auto* other_ptr = dynamic_cast<const VectorizedFnCall*>(&other);
+    if (!other_ptr) {
+        return false;
+    }
+    if (this->_function_name != other_ptr->_function_name) {
+        return false;
+    }
+    if (this->children().size() != other_ptr->children().size()) {
+        return false;
+    }
+    for (size_t i = 0; i < this->children().size(); i++) {
+        if (!this->get_child(i)->equals(*other_ptr->get_child(i))) {
+            return false;
+        }
+    }
+    return true;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vectorized_fn_call.h 
b/be/src/vec/exprs/vectorized_fn_call.h
index 5c74fa81689..0fa41b88522 100644
--- a/be/src/vec/exprs/vectorized_fn_call.h
+++ b/be/src/vec/exprs/vectorized_fn_call.h
@@ -73,9 +73,13 @@ public:
     }
     static std::string debug_string(const std::vector<VectorizedFnCall*>& 
exprs);
 
+    bool can_push_down_to_index() const override;
+    bool can_fast_execute() const override;
+    Status eval_inverted_index(VExprContext* context, 
segment_v2::FuncExprParams& params) override;
+    bool equals(const VExpr& other) override;
+
 protected:
     FunctionBasePtr _function;
-    bool _can_fast_execute = false;
     std::string _expr_name;
     std::string _function_name;
 
@@ -83,4 +87,5 @@ private:
     Status _do_execute(doris::vectorized::VExprContext* context, 
doris::vectorized::Block* block,
                        int* result_column_id, std::vector<size_t>& args);
 };
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 31a8e04cad1..c9aea590b3c 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -637,24 +637,33 @@ bool VExpr::fast_execute(Block& block, const 
ColumnNumbers& arguments, size_t re
 }
 
 std::string VExpr::gen_predicate_result_sign(Block& block, const 
ColumnNumbers& arguments,
-                                             const std::string& function_name) 
{
+                                             const std::string& function_name) 
const {
     std::string pred_result_sign;
-    std::string column_name = block.get_by_position(arguments[0]).name;
-    pred_result_sign +=
-            BeConsts::BLOCK_TEMP_COLUMN_PREFIX + column_name + "_" + 
function_name + "_";
-    if (function_name == "in" || function_name == "not_in") {
-        // Generating 'result_sign' from 'inlist' requires sorting the values.
-        std::set<std::string> values;
-        for (size_t i = 1; i < arguments.size(); i++) {
-            const auto& entry = block.get_by_position(arguments[i]);
-            values.insert(entry.type->to_string(*entry.column, 0));
-        }
-        pred_result_sign += boost::join(values, ",");
+    if (this->node_type() == TExprNodeType::FUNCTION_CALL) {
+        pred_result_sign =
+                BeConsts::BLOCK_TEMP_COLUMN_PREFIX + 
std::to_string(this->index_unique_id());
     } else {
-        const auto& entry = block.get_by_position(arguments[1]);
-        pred_result_sign += entry.type->to_string(*entry.column, 0);
+        std::string column_name = block.get_by_position(arguments[0]).name;
+        pred_result_sign +=
+                BeConsts::BLOCK_TEMP_COLUMN_PREFIX + column_name + "_" + 
function_name + "_";
+        if (function_name == "in" || function_name == "not_in") {
+            // Generating 'result_sign' from 'inlist' requires sorting the 
values.
+            std::set<std::string> values;
+            for (size_t i = 1; i < arguments.size(); i++) {
+                const auto& entry = block.get_by_position(arguments[i]);
+                values.insert(entry.type->to_string(*entry.column, 0));
+            }
+            pred_result_sign += boost::join(values, ",");
+        } else {
+            const auto& entry = block.get_by_position(arguments[1]);
+            pred_result_sign += entry.type->to_string(*entry.column, 0);
+        }
     }
     return pred_result_sign;
 }
 
+bool VExpr::equals(const VExpr& other) {
+    return false;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h
index 2910e3b733c..bfb4f121b8a 100644
--- a/be/src/vec/exprs/vexpr.h
+++ b/be/src/vec/exprs/vexpr.h
@@ -237,7 +237,16 @@ public:
                       size_t input_rows_count, const std::string& 
function_name);
 
     std::string gen_predicate_result_sign(Block& block, const ColumnNumbers& 
arguments,
-                                          const std::string& function_name);
+                                          const std::string& function_name) 
const;
+
+    virtual bool can_push_down_to_index() const { return false; }
+    virtual bool can_fast_execute() const { return false; }
+    virtual Status eval_inverted_index(VExprContext* context, 
segment_v2::FuncExprParams& params) {
+        return Status::NotSupported("Not supported 
execute_with_inverted_index");
+    }
+    virtual bool equals(const VExpr& other);
+    void set_index_unique_id(uint32_t index_unique_id) { _index_unique_id = 
index_unique_id; }
+    uint32_t index_unique_id() const { return _index_unique_id; }
 
 protected:
     /// Simple debug string that provides no expr subclass-specific information
@@ -303,6 +312,10 @@ protected:
     // for concrete classes
     bool _prepare_finished = false;
     bool _open_finished = false;
+
+    // ensuring uniqueness during index traversal
+    uint32_t _index_unique_id = 0;
+    bool _can_fast_execute = false;
 };
 
 } // namespace vectorized
diff --git a/be/src/vec/exprs/vexpr_context.cpp 
b/be/src/vec/exprs/vexpr_context.cpp
index 1cc24c61ea9..60b10617d4d 100644
--- a/be/src/vec/exprs/vexpr_context.cpp
+++ b/be/src/vec/exprs/vexpr_context.cpp
@@ -119,6 +119,7 @@ int VExprContext::register_function_context(RuntimeState* 
state, const TypeDescr
     
_fn_contexts.back()->set_check_overflow_for_decimal(state->check_overflow_for_decimal());
     return _fn_contexts.size() - 1;
 }
+
 Status VExprContext::eval_inverted_index(
         const std::unordered_map<ColumnId, 
std::pair<vectorized::IndexFieldNameAndTypePair,
                                                      
segment_v2::InvertedIndexIterator*>>&
diff --git a/be/src/vec/exprs/vin_predicate.cpp 
b/be/src/vec/exprs/vin_predicate.cpp
index 8eaf42abf23..4affec791a4 100644
--- a/be/src/vec/exprs/vin_predicate.cpp
+++ b/be/src/vec/exprs/vin_predicate.cpp
@@ -78,9 +78,7 @@ Status VInPredicate::prepare(RuntimeState* state, const 
RowDescriptor& desc,
 
     VExpr::register_function_context(state, context);
     _prepare_finished = true;
-
-    _can_fast_execute = _function->can_fast_execute();
-
+    _can_fast_execute = can_fast_execute();
     return Status::OK();
 }
 
diff --git a/be/src/vec/exprs/vin_predicate.h b/be/src/vec/exprs/vin_predicate.h
index 82e99ac447a..5f197f996e9 100644
--- a/be/src/vec/exprs/vin_predicate.h
+++ b/be/src/vec/exprs/vin_predicate.h
@@ -54,6 +54,7 @@ public:
     const FunctionBasePtr function() { return _function; }
 
     bool is_not_in() const { return _is_not_in; };
+    bool can_fast_execute() const override { return true; }
 
 private:
     FunctionBasePtr _function;
@@ -61,7 +62,5 @@ private:
 
     const bool _is_not_in;
     static const constexpr char* function_name = "in";
-
-    bool _can_fast_execute = false;
 };
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/exprs/vliteral.cpp b/be/src/vec/exprs/vliteral.cpp
index 2dd544a0428..d4e8c883106 100644
--- a/be/src/vec/exprs/vliteral.cpp
+++ b/be/src/vec/exprs/vliteral.cpp
@@ -90,4 +90,25 @@ std::string VLiteral::debug_string() const {
     return out.str();
 }
 
+bool VLiteral::equals(const VExpr& other) {
+    const auto* other_ptr = dynamic_cast<const VLiteral*>(&other);
+    if (!other_ptr) {
+        return false;
+    }
+    if (this->_expr_name != other_ptr->_expr_name) {
+        return false;
+    }
+    if (this->_column_ptr->structure_equals(*other_ptr->_column_ptr)) {
+        if (this->_column_ptr->size() != other_ptr->_column_ptr->size()) {
+            return false;
+        }
+        for (size_t i = 0; i < this->_column_ptr->size(); i++) {
+            if (this->_column_ptr->compare_at(i, i, *other_ptr->_column_ptr, 
-1) != 0) {
+                return false;
+            }
+        }
+    }
+    return true;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vliteral.h b/be/src/vec/exprs/vliteral.h
index 30ea090c2ff..3e56757b313 100644
--- a/be/src/vec/exprs/vliteral.h
+++ b/be/src/vec/exprs/vliteral.h
@@ -55,6 +55,8 @@ public:
 
     bool is_literal() const override { return true; }
 
+    bool equals(const VExpr& other) override;
+
 protected:
     ColumnPtr _column_ptr;
     std::string _expr_name;
diff --git a/be/src/vec/exprs/vslot_ref.cpp b/be/src/vec/exprs/vslot_ref.cpp
index de9a10137f2..df761360d08 100644
--- a/be/src/vec/exprs/vslot_ref.cpp
+++ b/be/src/vec/exprs/vslot_ref.cpp
@@ -112,4 +112,21 @@ std::string VSlotRef::debug_string() const {
     out << "SlotRef(slot_id=" << _slot_id << VExpr::debug_string() << ")";
     return out.str();
 }
+
+bool VSlotRef::equals(const VExpr& other) {
+    if (!VExpr::equals(other)) {
+        return false;
+    }
+    const auto* other_ptr = dynamic_cast<const VSlotRef*>(&other);
+    if (!other_ptr) {
+        return false;
+    }
+    if (this->_slot_id != other_ptr->_slot_id || this->_column_id != 
other_ptr->_column_id ||
+        this->_column_name != other_ptr->_column_name ||
+        this->_column_label != other_ptr->_column_label) {
+        return false;
+    }
+    return true;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vslot_ref.h b/be/src/vec/exprs/vslot_ref.h
index f96728eda5c..2bea6ea5c06 100644
--- a/be/src/vec/exprs/vslot_ref.h
+++ b/be/src/vec/exprs/vslot_ref.h
@@ -52,6 +52,8 @@ public:
 
     int slot_id() const { return _slot_id; }
 
+    bool equals(const VExpr& other) override;
+
 private:
     int _slot_id;
     int _column_id;
diff --git a/be/src/vec/functions/function.h b/be/src/vec/functions/function.h
index c97f73a9d69..f4243875d64 100644
--- a/be/src/vec/functions/function.h
+++ b/be/src/vec/functions/function.h
@@ -41,6 +41,10 @@
 #include "vec/data_types/data_type.h"
 #include "vec/data_types/data_type_nullable.h"
 
+namespace doris::segment_v2 {
+struct FuncExprParams;
+} // namespace doris::segment_v2
+
 namespace doris::vectorized {
 
 #define RETURN_REAL_TYPE_FOR_DATEV2_FUNCTION(TYPE)                             
          \
@@ -60,6 +64,7 @@ namespace doris::vectorized {
     }
 
 class Field;
+class VExpr;
 
 // Only use dispose the variadic argument
 template <typename T>
@@ -200,8 +205,6 @@ public:
         return Status::OK();
     }
 
-    virtual bool can_fast_execute() const { return false; }
-
     virtual bool is_use_default_implementation_for_constants() const = 0;
 
     /// The property of monotonicity for a certain range.
@@ -230,6 +233,13 @@ public:
                                get_name());
         return Monotonicity {};
     }
+
+    virtual bool can_push_down_to_index() const { return false; }
+    virtual Status eval_inverted_index(FunctionContext* context,
+                                       segment_v2::FuncExprParams& params) {
+        return Status::NotSupported("eval_inverted_index is not supported in 
function: ",
+                                    get_name());
+    }
 };
 
 using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
@@ -414,15 +424,6 @@ public:
         return Status::OK();
     }
 
-    // here are lots of function not extends eval_inverted_index.
-    Status eval_inverted_index(FunctionContext* context,
-                               const vectorized::IndexFieldNameAndTypePair& 
data_type_with_name,
-                               segment_v2::InvertedIndexIterator* iter, 
uint32_t num_rows,
-                               roaring::Roaring* bitmap) const override {
-        return Status::NotSupported("eval_inverted_index is not supported in 
function: ",
-                                    get_name());
-    }
-
     [[noreturn]] const DataTypes& get_argument_types() const final {
         throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR,
                                "get_argument_types is not implemented for 
IFunction {}",
@@ -525,13 +526,6 @@ public:
         return function->close(context, scope);
     }
 
-    bool can_fast_execute() const override {
-        auto function_name = function->get_name();
-        return function_name == "eq" || function_name == "ne" || function_name 
== "lt" ||
-               function_name == "gt" || function_name == "le" || function_name 
== "ge" ||
-               function_name == "in" || function_name == "not_in";
-    }
-
     Status eval_inverted_index(FunctionContext* context,
                                const vectorized::IndexFieldNameAndTypePair& 
data_type_with_name,
                                segment_v2::InvertedIndexIterator* iter, 
uint32_t num_rows,
@@ -548,6 +542,12 @@ public:
         return function->is_use_default_implementation_for_constants();
     }
 
+    bool can_push_down_to_index() const override { return 
function->can_push_down_to_index(); }
+    Status eval_inverted_index(FunctionContext* context,
+                               segment_v2::FuncExprParams& params) override {
+        return function->eval_inverted_index(context, params);
+    }
+
 private:
     std::shared_ptr<IFunction> function;
     DataTypes arguments;
diff --git a/be/src/vec/functions/function_multi_match.cpp 
b/be/src/vec/functions/function_multi_match.cpp
new file mode 100644
index 00000000000..d4ff5c10dca
--- /dev/null
+++ b/be/src/vec/functions/function_multi_match.cpp
@@ -0,0 +1,187 @@
+// 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.
+
+#include "vec/functions/function_multi_match.h"
+
+#include <gen_cpp/PaloBrokerService_types.h>
+#include <glog/logging.h>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/algorithm/string/classification.hpp>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <roaring/roaring.hh>
+#include <string>
+#include <vector>
+
+#include "io/fs/file_reader.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/segment_v2/inverted_index/query/phrase_prefix_query.h"
+#include "olap/rowset/segment_v2/segment_iterator.h"
+#include "runtime/primitive_type.h"
+#include "vec/columns/column.h"
+#include "vec/data_types/data_type.h"
+#include "vec/exprs/varray_literal.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/exprs/vslot_ref.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+Status FunctionMultiMatch::execute_impl(FunctionContext* /*context*/, Block& 
block,
+                                        const ColumnNumbers& arguments, size_t 
result,
+                                        size_t /*input_rows_count*/) const {
+    return Status::RuntimeError("only inverted index queries are supported");
+}
+
+Status FunctionMultiMatch::open(FunctionContext* context,
+                                FunctionContext::FunctionStateScope scope) {
+    if (scope == FunctionContext::THREAD_LOCAL) {
+        return Status::OK();
+    }
+
+    DCHECK(context->get_num_args() == 4);
+    for (int i = 0; i < context->get_num_args(); ++i) {
+        DCHECK(is_string_type(context->get_arg_type(i)->type));
+    }
+
+    std::shared_ptr<MatchParam> state = std::make_shared<MatchParam>();
+    context->set_function_state(scope, state);
+    for (int i = 0; i < context->get_num_args(); ++i) {
+        const auto& const_column_ptr = context->get_constant_col(i);
+        if (const_column_ptr) {
+            auto const_data = const_column_ptr->column_ptr->get_data_at(0);
+            switch (i) {
+            case 1: {
+                std::string field_names_str = const_data.to_string();
+                field_names_str.erase(
+                        std::remove_if(field_names_str.begin(), 
field_names_str.end(),
+                                       [](unsigned char c) { return 
std::isspace(c); }),
+                        field_names_str.end());
+                std::vector<std::string> field_names;
+                boost::split(field_names, field_names_str, 
boost::algorithm::is_any_of(","));
+                state->fields.insert(field_names.begin(), field_names.end());
+            } break;
+            case 2:
+                state->type = const_data.to_string();
+                break;
+            case 3:
+                state->query = const_data.to_string();
+                break;
+            default:
+                break;
+            }
+        }
+    }
+
+    return Status::OK();
+}
+
+Status FunctionMultiMatch::eval_inverted_index(FunctionContext* context,
+                                               segment_v2::FuncExprParams& 
params) {
+    auto* match_param = reinterpret_cast<MatchParam*>(
+            context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
+    if (match_param == nullptr) {
+        return Status::RuntimeError("function parameter parsing failed");
+    }
+    match_param->fields.insert(params._column_name);
+
+    const auto& segment_iterator = params._segment_iterator;
+    const auto& opts = segment_iterator->storage_read_options();
+    const auto& tablet_schema = opts.tablet_schema;
+
+    std::vector<ColumnId> columns_ids;
+
+    for (const auto& column_name : match_param->fields) {
+        auto cid = tablet_schema->field_index(column_name);
+        if (cid < 0) {
+            return Status::RuntimeError("column name is incorrect");
+        }
+        const auto& column = tablet_schema->column(cid);
+        if (!is_string_type(column.type())) {
+            return Status::RuntimeError("column type is incorrect");
+        }
+        if (!tablet_schema->has_inverted_index(column)) {
+            return Status::RuntimeError("column index is incorrect");
+        }
+        columns_ids.emplace_back(cid);
+    }
+
+    // query type
+    InvertedIndexQueryType query_type;
+    if (match_param->type == "phrase_prefix") {
+        query_type = InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY;
+    } else {
+        return Status::RuntimeError("query type is incorrect");
+    }
+
+    // cache key
+    roaring::Roaring cids_str;
+    cids_str.addMany(columns_ids.size(), columns_ids.data());
+    cids_str.runOptimize();
+    std::string column_name_binary(cids_str.getSizeInBytes(), 0);
+    cids_str.write(column_name_binary.data());
+
+    InvertedIndexQueryCache::CacheKey cache_key;
+    io::Path index_path = segment_iterator->segment().file_reader()->path();
+    cache_key.index_path = index_path.parent_path() / index_path.stem();
+    cache_key.column_name = column_name_binary;
+    cache_key.query_type = query_type;
+    cache_key.value = match_param->query;
+
+    // query cache
+    auto* cache = InvertedIndexQueryCache::instance();
+    InvertedIndexQueryCacheHandle cache_handler;
+    if (cache->lookup(cache_key, &cache_handler)) {
+        params.result = cache_handler.get_bitmap();
+        return Status::OK();
+    }
+
+    // search
+    bool first = true;
+    for (const auto& column_name : match_param->fields) {
+        auto cid = tablet_schema->field_index(column_name);
+
+        auto& index_iterator = 
segment_iterator->inverted_index_iterators()[cid];
+        if (!index_iterator) {
+            
RETURN_IF_ERROR(segment_iterator->_init_inverted_index_iterators(cid));
+        }
+        const auto& index_reader = index_iterator->reader();
+
+        auto result = std::make_shared<roaring::Roaring>();
+        RETURN_IF_ERROR(index_reader->query(opts.stats, opts.runtime_state, 
column_name,
+                                            match_param->query.data(), 
query_type, result));
+        if (first) {
+            (*params.result).swap(*result);
+            first = false;
+        } else {
+            (*params.result) |= (*result);
+        }
+    }
+
+    params.result->runOptimize();
+    cache->insert(cache_key, params.result, &cache_handler);
+
+    return Status::OK();
+}
+
+void register_function_multi_match(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionMultiMatch>();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_multi_match.h 
b/be/src/vec/functions/function_multi_match.h
new file mode 100644
index 00000000000..55e13778ab7
--- /dev/null
+++ b/be/src/vec/functions/function_multi_match.h
@@ -0,0 +1,70 @@
+// 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.
+
+#pragma once
+
+#include <boost/algorithm/string/split.hpp>
+
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_array.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/functions/function.h"
+
+namespace doris::vectorized {
+
+class MatchParam {
+public:
+    std::string query;
+    std::set<std::string> fields;
+    std::string type;
+};
+
+class FunctionMultiMatch : public IFunction {
+public:
+    static constexpr auto name = "multi_match";
+
+    static FunctionPtr create() { return 
std::make_shared<FunctionMultiMatch>(); }
+    using NullMapType = PaddedPODArray<UInt8>;
+
+    String get_name() const override { return name; }
+
+    bool is_variadic() const override { return false; }
+
+    size_t get_number_of_arguments() const override { return 4; }
+
+    bool use_default_implementation_for_nulls() const override { return false; 
}
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeUInt8>();
+    }
+
+    Status open(FunctionContext* context, FunctionContext::FunctionStateScope 
scope) override;
+
+    Status close(FunctionContext* context, FunctionContext::FunctionStateScope 
scope) override {
+        return Status::OK();
+    }
+
+    Status execute_impl(FunctionContext* /*context*/, Block& block, const 
ColumnNumbers& arguments,
+                        size_t result, size_t /*input_rows_count*/) const 
override;
+
+    bool can_push_down_to_index() const override { return true; }
+
+    Status eval_inverted_index(FunctionContext* context,
+                               segment_v2::FuncExprParams& params) override;
+};
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 40a0cd78508..4a6ef814243 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -97,15 +97,14 @@ void 
register_function_multi_string_position(SimpleFunctionFactory& factory);
 void register_function_multi_string_search(SimpleFunctionFactory& factory);
 void register_function_width_bucket(SimpleFunctionFactory& factory);
 void register_function_ignore(SimpleFunctionFactory& factory);
-
 void register_function_encryption(SimpleFunctionFactory& factory);
 void register_function_regexp_extract(SimpleFunctionFactory& factory);
 void register_function_hex_variadic(SimpleFunctionFactory& factory);
 void register_function_match(SimpleFunctionFactory& factory);
 void register_function_tokenize(SimpleFunctionFactory& factory);
-
 void register_function_url(SimpleFunctionFactory& factory);
 void register_function_ip(SimpleFunctionFactory& factory);
+void register_function_multi_match(SimpleFunctionFactory& factory);
 
 class SimpleFunctionFactory {
     using Creator = std::function<FunctionBuilderPtr()>;
@@ -292,6 +291,7 @@ public:
             register_function_tokenize(instance);
             register_function_ignore(instance);
             register_function_variant_element(instance);
+            register_function_multi_match(instance);
         });
         return instance;
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 0f0dc8b5d2a..bd1d10c7ec9 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -306,6 +306,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MonthName;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatch;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatchAny;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiSearchAllPositions;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash332;
@@ -926,7 +927,8 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(YearWeek.class, "yearweek"),
             scalar(YearsAdd.class, "years_add"),
             scalar(YearsDiff.class, "years_diff"),
-            scalar(YearsSub.class, "years_sub"));
+            scalar(YearsSub.class, "years_sub"),
+            scalar(MultiMatch.class, "multi_match"));
 
     public static final BuiltinScalarFunctions INSTANCE = new 
BuiltinScalarFunctions();
 
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MultiMatch.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MultiMatch.java
new file mode 100644
index 00000000000..3df662808ed
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/MultiMatch.java
@@ -0,0 +1,73 @@
+// 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.
+
+package org.apache.doris.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'multi_match'. This class is generated by GenerateFunction.
+ */
+public class MultiMatch extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, 
AlwaysNotNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BooleanType.INSTANCE)
+                    .args(StringType.INSTANCE,
+                            StringType.INSTANCE,
+                            StringType.INSTANCE,
+                            StringType.INSTANCE)
+    );
+
+    /**
+     * constructor with 4 arguments.
+     */
+    public MultiMatch(Expression arg0, Expression arg1, Expression arg2, 
Expression arg3) {
+        super("multi_match", arg0, arg1, arg2, arg3);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public MultiMatch withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 4);
+        return new MultiMatch(children.get(0), children.get(1), 
children.get(2), children.get(3));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitMultiMatch(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 2f5ef3faa85..e81924b7e7c 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -307,6 +307,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MonthName;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.MonthsSub;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatch;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiMatchAny;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MultiSearchAllPositions;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.MurmurHash332;
@@ -2207,4 +2208,8 @@ public interface ScalarFunctionVisitor<R, C> {
     default R visitStructElement(StructElement structElement, C context) {
         return visitScalarFunction(structElement, context);
     }
+
+    default R visitMultiMatch(MultiMatch multiMatch, C context) {
+        return visitScalarFunction(multiMatch, context);
+    }
 }
diff --git a/gensrc/script/doris_builtins_functions.py 
b/gensrc/script/doris_builtins_functions.py
index 00e81277257..15986dbf5af 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -2254,6 +2254,11 @@ visible_functions = {
         [['ignore'], 'BOOLEAN', ['ARRAY_DECIMAL128', '...'], 
'ALWAYS_NOT_NULLABLE'],
         [['ignore'], 'BOOLEAN', ['ARRAY_VARCHAR', '...'], 
'ALWAYS_NOT_NULLABLE'],
         [['ignore'], 'BOOLEAN', ['ARRAY_STRING', '...'], 'ALWAYS_NOT_NULLABLE']
+    ],
+
+    # multi match functions
+    "MultiMatch": [
+        [['multi_match'], 'BOOLEAN', ['STRING', 'STRING', 'STRING', 'STRING'], 
'ALWAYS_NOT_NULLABLE']
     ]
 }
 


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

Reply via email to