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

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


The following commit(s) were added to refs/heads/branch-4.0 by this push:
     new 99dc7079857 [refactor](index scan) Rename inverted index status to 
index status (… (#58374)
99dc7079857 is described below

commit 99dc7079857b912d11f999631b250bc8e9f11c86
Author: zhiqiang <[email protected]>
AuthorDate: Wed Nov 26 15:29:12 2025 +0800

    [refactor](index scan) Rename inverted index status to index status (… 
(#58374)
    
    …#57911)
    
    After https://github.com/apache/doris/pull/57243, previous codes used
    for inverted index scan can be rename to a more common name.
    
    ### What problem does this PR solve?
    
    Issue Number: close #xxx
    
    Related PR: #xxx
    
    Problem Summary:
    
    ### Release note
    
    None
    
    ### Check List (For Author)
    
    - Test <!-- At least one of them must be included. -->
        - [ ] Regression test
        - [ ] Unit Test
        - [ ] Manual test (add detailed scripts or steps below)
        - [ ] No need to test or manual test. Explain why:
    - [ ] This is a refactor/code format and no logic has been changed.
            - [ ] Previous test can cover this change.
            - [ ] No code files have been changed.
            - [ ] Other reason <!-- Add your reason?  -->
    
    - Behavior changed:
        - [ ] No.
        - [ ] Yes. <!-- Explain the behavior change -->
    
    - Does this need documentation?
        - [ ] No.
    - [ ] Yes. <!-- Add document PR link here. eg:
    https://github.com/apache/doris-website/pull/1214 -->
    
    ### Check List (For Reviewer who merge this PR)
    
    - [ ] Confirm the release note
    - [ ] Confirm test cases
    - [ ] Confirm document
    - [ ] Add branch pick label <!-- Add branch pick label that this PR
    should merge into -->
---
 be/src/olap/rowset/segment_v2/segment_iterator.cpp | 35 ++++++-------
 be/src/olap/rowset/segment_v2/segment_iterator.h   | 14 +++--
 be/src/util/threadpool.cpp                         |  2 +
 be/src/vec/exprs/vcompound_pred.h                  | 20 +++----
 be/src/vec/exprs/vexpr.cpp                         | 25 ++++-----
 be/src/vec/exprs/vexpr_context.cpp                 |  5 +-
 be/src/vec/exprs/vexpr_context.h                   | 61 ++++++++++------------
 be/src/vec/exprs/vsearch.cpp                       | 10 ++--
 .../exprs/vexpr_evalute_inverted_index_test.cpp    | 11 ++--
 be/test/vec/exprs/vsearch_expr_test.cpp            | 26 ++++-----
 10 files changed, 101 insertions(+), 108 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp 
b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 31bde616a36..9b2eed85348 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -384,7 +384,7 @@ void SegmentIterator::_initialize_predicate_results() {
     // Initialize from _col_predicates
     for (auto* pred : _col_predicates) {
         int cid = pred->column_id();
-        _column_predicate_inverted_index_status[cid][pred] = false;
+        _column_predicate_index_exec_status[cid][pred] = false;
     }
 
     _calculate_expr_in_remaining_conjunct_root();
@@ -620,9 +620,8 @@ Status 
SegmentIterator::_get_row_ranges_by_column_conditions() {
             for (auto it = _common_expr_ctxs_push_down.begin();
                  it != _common_expr_ctxs_push_down.end();) {
                 if ((*it)->all_expr_inverted_index_evaluated()) {
-                    const auto* result =
-                            
(*it)->get_inverted_index_context()->get_inverted_index_result_for_expr(
-                                    (*it)->root().get());
+                    const auto* result = 
(*it)->get_index_context()->get_index_result_for_expr(
+                            (*it)->root().get());
                     if (result != nullptr) {
                         _row_bitmap &= *result->get_data_bitmap();
                         auto root = (*it)->root();
@@ -1166,7 +1165,7 @@ Status 
SegmentIterator::_apply_inverted_index_on_column_predicate(
             return Status::OK();
         }
         if (!pred->is_runtime_filter()) {
-            _column_predicate_inverted_index_status[pred->column_id()][pred] = 
true;
+            _column_predicate_index_exec_status[pred->column_id()][pred] = 
true;
         }
     }
     return Status::OK();
@@ -1271,8 +1270,8 @@ bool 
SegmentIterator::_check_all_conditions_passed_inverted_index_for_column(Col
     if (_opts.runtime_state && 
!_opts.runtime_state->query_options().enable_common_expr_pushdown) {
         return false;
     }
-    auto pred_it = _column_predicate_inverted_index_status.find(cid);
-    if (pred_it != _column_predicate_inverted_index_status.end()) {
+    auto pred_it = _column_predicate_index_exec_status.find(cid);
+    if (pred_it != _column_predicate_index_exec_status.end()) {
         const auto& pred_map = pred_it->second;
         bool pred_passed = std::all_of(pred_map.begin(), pred_map.end(),
                                        [](const auto& pred_entry) { return 
pred_entry.second; });
@@ -1283,8 +1282,8 @@ bool 
SegmentIterator::_check_all_conditions_passed_inverted_index_for_column(Col
         }
     }
 
-    auto expr_it = _common_expr_inverted_index_status.find(cid);
-    if (expr_it != _common_expr_inverted_index_status.end()) {
+    auto expr_it = _common_expr_index_exec_status.find(cid);
+    if (expr_it != _common_expr_index_exec_status.end()) {
         const auto& expr_map = expr_it->second;
         return std::all_of(expr_map.begin(), expr_map.end(),
                            [](const auto& expr_entry) { return 
expr_entry.second; });
@@ -2716,7 +2715,7 @@ void 
SegmentIterator::_output_index_result_column_for_expr(uint16_t* sel_rowid_i
     }
     for (auto& expr_ctx : _common_expr_ctxs_push_down) {
         for (auto& inverted_index_result_bitmap_for_expr :
-             
expr_ctx->get_inverted_index_context()->get_inverted_index_result_bitmap()) {
+             expr_ctx->get_index_context()->get_index_result_bitmap()) {
             const auto* expr = inverted_index_result_bitmap_for_expr.first;
             const auto& result_bitmap = 
inverted_index_result_bitmap_for_expr.second;
             const auto& index_result_bitmap = result_bitmap.get_data_bitmap();
@@ -2754,11 +2753,11 @@ void 
SegmentIterator::_output_index_result_column_for_expr(uint16_t* sel_rowid_i
             DCHECK(block->rows() == vec_match_pred.size());
 
             if (null_map_column) {
-                
expr_ctx->get_inverted_index_context()->set_inverted_index_result_column_for_expr(
+                
expr_ctx->get_index_context()->set_index_result_column_for_expr(
                         expr, 
vectorized::ColumnNullable::create(std::move(index_result_column),
                                                                  
std::move(null_map_column)));
             } else {
-                
expr_ctx->get_inverted_index_context()->set_inverted_index_result_column_for_expr(
+                
expr_ctx->get_index_context()->set_index_result_column_for_expr(
                         expr, std::move(index_result_column));
             }
         }
@@ -2813,14 +2812,14 @@ Status 
SegmentIterator::current_block_row_locations(std::vector<RowLocation>* bl
 }
 
 Status SegmentIterator::_construct_compound_expr_context() {
-    auto inverted_index_context = 
std::make_shared<vectorized::InvertedIndexContext>(
+    auto inverted_index_context = 
std::make_shared<vectorized::IndexExecContext>(
             _schema->column_ids(), _index_iterators, _storage_name_and_type,
-            _common_expr_inverted_index_status, _score_runtime);
+            _common_expr_index_exec_status, _score_runtime);
     for (const auto& expr_ctx : _opts.common_expr_ctxs_push_down) {
         vectorized::VExprContextSPtr context;
         // _ann_range_search_runtime will do deep copy.
         RETURN_IF_ERROR(expr_ctx->clone(_opts.runtime_state, context));
-        context->set_inverted_index_context(inverted_index_context);
+        context->set_index_context(inverted_index_context);
         _common_expr_ctxs_push_down.emplace_back(context);
     }
     return Status::OK();
@@ -2862,7 +2861,7 @@ void 
SegmentIterator::_calculate_expr_in_remaining_conjunct_root() {
                                 if (vir_child->is_slot_ref()) {
                                     auto* inner_slot_ref =
                                             
assert_cast<vectorized::VSlotRef*>(vir_child.get());
-                                    
_common_expr_inverted_index_status[_schema->column_id(
+                                    
_common_expr_index_exec_status[_schema->column_id(
                                             
inner_slot_ref->column_id())][expr.get()] = false;
                                     
_common_expr_to_slotref_map[root_expr_ctx.get()]
                                                                
[inner_slot_ref->column_id()] =
@@ -2878,8 +2877,8 @@ void 
SegmentIterator::_calculate_expr_in_remaining_conjunct_root() {
                 }
                 if (child->is_slot_ref()) {
                     auto* column_slot_ref = 
assert_cast<vectorized::VSlotRef*>(child.get());
-                    _common_expr_inverted_index_status[_schema->column_id(
-                            column_slot_ref->column_id())][expr.get()] = false;
+                    
_common_expr_index_exec_status[_schema->column_id(column_slot_ref->column_id())]
+                                                  [expr.get()] = false;
                     
_common_expr_to_slotref_map[root_expr_ctx.get()][column_slot_ref->column_id()] =
                             expr.get();
                 }
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h 
b/be/src/olap/rowset/segment_v2/segment_iterator.h
index b6d49c053a1..7f765a5f4b4 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.h
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.h
@@ -485,14 +485,21 @@ private:
     * a boolean value to indicate whether the column has been read by the 
index.
     */
     std::unordered_map<ColumnId, std::unordered_map<ColumnPredicate*, bool>>
-            _column_predicate_inverted_index_status;
+            _column_predicate_index_exec_status;
 
     /*
     * column and common expr on it.
     * a boolean value to indicate whether the column has been read by the 
index.
     */
     std::unordered_map<ColumnId, std::unordered_map<const vectorized::VExpr*, 
bool>>
-            _common_expr_inverted_index_status;
+            _common_expr_index_exec_status;
+
+    /*
+    * common expr context to slotref map
+    * slot ref map is used to get slot ref expr by using column id.
+    */
+    std::unordered_map<vectorized::VExprContext*, std::unordered_map<ColumnId, 
vectorized::VExpr*>>
+            _common_expr_to_slotref_map;
 
     vectorized::ScoreRuntimeSPtr _score_runtime;
 
@@ -506,9 +513,6 @@ private:
 
     // key is column uid, value is the sparse column cache
     std::unordered_map<int32_t, PathToSparseColumnCacheUPtr> 
_variant_sparse_column_cache;
-
-    std::unordered_map<vectorized::VExprContext*, std::unordered_map<ColumnId, 
vectorized::VExpr*>>
-            _common_expr_to_slotref_map;
 };
 
 } // namespace segment_v2
diff --git a/be/src/util/threadpool.cpp b/be/src/util/threadpool.cpp
index cd0224c2a07..9b7ff2dfe3f 100644
--- a/be/src/util/threadpool.cpp
+++ b/be/src/util/threadpool.cpp
@@ -269,6 +269,7 @@ ThreadPool::~ThreadPool() {
     CHECK_EQ(1, _tokens.size()) << absl::Substitute(
             "Threadpool $0 destroyed with $1 allocated tokens", _name, 
_tokens.size());
     shutdown();
+    VLOG_DEBUG << fmt::format("Thread pool {} destroyed", _name);
 }
 
 Status ThreadPool::try_create_thread(int thread_num, 
std::lock_guard<std::mutex>&) {
@@ -332,6 +333,7 @@ Status ThreadPool::init() {
 }
 
 void ThreadPool::shutdown() {
+    VLOG_DEBUG << fmt::format("Shutting down thread pool {}", _name);
     // Why access to doris_metrics is safe here?
     // Since DorisMetrics is a singleton, it will be destroyed only after 
doris_main is exited.
     // The shutdown/destroy of ThreadPool is guaranteed to take place before 
doris_main exits by
diff --git a/be/src/vec/exprs/vcompound_pred.h 
b/be/src/vec/exprs/vcompound_pred.h
index 126198f486c..604a23a2680 100644
--- a/be/src/vec/exprs/vcompound_pred.h
+++ b/be/src/vec/exprs/vcompound_pred.h
@@ -74,10 +74,10 @@ public:
                     all_pass = false;
                     continue;
                 }
-                auto inverted_index_context = 
context->get_inverted_index_context();
-                if 
(inverted_index_context->has_inverted_index_result_for_expr(child.get())) {
+                auto inverted_index_context = context->get_index_context();
+                if 
(inverted_index_context->has_index_result_for_expr(child.get())) {
                     const auto* index_result =
-                            
inverted_index_context->get_inverted_index_result_for_expr(child.get());
+                            
inverted_index_context->get_index_result_for_expr(child.get());
                     if (res.is_empty()) {
                         res = *index_result;
                     } else {
@@ -103,11 +103,9 @@ public:
                     all_pass = false;
                     continue;
                 }
-                if 
(context->get_inverted_index_context()->has_inverted_index_result_for_expr(
-                            child.get())) {
+                if 
(context->get_index_context()->has_index_result_for_expr(child.get())) {
                     const auto* index_result =
-                            context->get_inverted_index_context()
-                                    
->get_inverted_index_result_for_expr(child.get());
+                            
context->get_index_context()->get_index_result_for_expr(child.get());
                     if (res.is_empty()) {
                         res = *index_result;
                     } else {
@@ -132,11 +130,9 @@ public:
                 return st;
             }
 
-            if 
(context->get_inverted_index_context()->has_inverted_index_result_for_expr(
-                        child.get())) {
+            if 
(context->get_index_context()->has_index_result_for_expr(child.get())) {
                 const auto* index_result =
-                        
context->get_inverted_index_context()->get_inverted_index_result_for_expr(
-                                child.get());
+                        
context->get_index_context()->get_index_result_for_expr(child.get());
                 roaring::Roaring full_result;
                 full_result.addRange(0, segment_num_rows);
                 res = index_result->op_not(&full_result);
@@ -151,7 +147,7 @@ public:
         }
 
         if (all_pass && !res.is_empty()) {
-            
context->get_inverted_index_context()->set_inverted_index_result_for_expr(this, 
res);
+            context->get_index_context()->set_index_result_for_expr(this, res);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index f4076ad01a1..30ba764b1af 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -823,7 +823,7 @@ Status VExpr::_evaluate_inverted_index(VExprContext* 
context, const FunctionBase
     column_ids.reserve(estimated_size);
     children_exprs.reserve(estimated_size);
 
-    auto index_context = context->get_inverted_index_context();
+    auto index_context = context->get_index_context();
 
     // if child is cast expr, we need to ensure target data type is the same 
with storage data type.
     // or they are all string type
@@ -836,8 +836,8 @@ Status VExpr::_evaluate_inverted_index(VExprContext* 
context, const FunctionBase
                 auto* column_slot_ref = 
assert_cast<VSlotRef*>(cast_expr->get_child(0).get());
                 auto column_id = column_slot_ref->column_id();
                 const auto* storage_name_type =
-                        context->get_inverted_index_context()
-                                
->get_storage_name_and_type_by_column_id(column_id);
+                        
context->get_index_context()->get_storage_name_and_type_by_column_id(
+                                column_id);
                 auto storage_type = remove_nullable(storage_name_type->second);
                 auto target_type = 
remove_nullable(cast_expr->get_target_type());
                 auto origin_primitive_type = 
storage_type->get_primitive_type();
@@ -879,16 +879,14 @@ Status VExpr::_evaluate_inverted_index(VExprContext* 
context, const FunctionBase
         if (child->is_slot_ref()) {
             auto* column_slot_ref = assert_cast<VSlotRef*>(child.get());
             auto column_id = column_slot_ref->column_id();
-            auto* iter =
-                    
context->get_inverted_index_context()->get_inverted_index_iterator_by_column_id(
-                            column_id);
+            auto* iter = 
context->get_index_context()->get_inverted_index_iterator_by_column_id(
+                    column_id);
             //column does not have inverted index
             if (iter == nullptr) {
                 continue;
             }
             const auto* storage_name_type =
-                    
context->get_inverted_index_context()->get_storage_name_and_type_by_column_id(
-                            column_id);
+                    
context->get_index_context()->get_storage_name_and_type_by_column_id(column_id);
             if (storage_name_type == nullptr) {
                 auto err_msg = fmt::format(
                         "storage_name_type cannot be found for column {} while 
in {} "
@@ -924,9 +922,9 @@ Status VExpr::_evaluate_inverted_index(VExprContext* 
context, const FunctionBase
         return res;
     }
     if (!result_bitmap.is_empty()) {
-        index_context->set_inverted_index_result_for_expr(this, result_bitmap);
+        index_context->set_index_result_for_expr(this, result_bitmap);
         for (int column_id : column_ids) {
-            index_context->set_true_for_inverted_index_status(this, column_id);
+            index_context->set_true_for_index_status(this, column_id);
         }
     }
     return Status::OK();
@@ -951,11 +949,10 @@ size_t VExpr::estimate_memory(const size_t rows) {
 }
 
 bool VExpr::fast_execute(VExprContext* context, ColumnPtr& result_column) 
const {
-    if (context->get_inverted_index_context() &&
-        
context->get_inverted_index_context()->get_inverted_index_result_column().contains(this))
 {
+    if (context->get_index_context() &&
+        
context->get_index_context()->get_index_result_column().contains(this)) {
         // prepare a column to save result
-        result_column =
-                
context->get_inverted_index_context()->get_inverted_index_result_column()[this];
+        result_column = 
context->get_index_context()->get_index_result_column()[this];
         if (_data_type->is_nullable()) {
             result_column = make_nullable(result_column);
         }
diff --git a/be/src/vec/exprs/vexpr_context.cpp 
b/be/src/vec/exprs/vexpr_context.cpp
index 67514ea364c..38b83a1467f 100644
--- a/be/src/vec/exprs/vexpr_context.cpp
+++ b/be/src/vec/exprs/vexpr_context.cpp
@@ -156,7 +156,7 @@ Status VExprContext::evaluate_inverted_index(uint32_t 
segment_num_rows) {
 }
 
 bool VExprContext::all_expr_inverted_index_evaluated() {
-    return 
_inverted_index_context->has_inverted_index_result_for_expr(_root.get());
+    return _index_context->has_index_result_for_expr(_root.get());
 }
 
 Status VExprContext::filter_block(VExprContext* vexpr_ctx, Block* block, 
size_t column_to_keep) {
@@ -503,8 +503,7 @@ Status VExprContext::evaluate_ann_range_search(
         return Status::OK();
     }
     const VExpr* slot_ref_expr_addr = slot_ref_map.find(cid)->second;
-    
_inverted_index_context->set_true_for_inverted_index_status(slot_ref_expr_addr,
-                                                                
idx_to_cid[cid]);
+    _index_context->set_true_for_index_status(slot_ref_expr_addr, 
idx_to_cid[cid]);
 
     VLOG_DEBUG << fmt::format(
             "Evaluate ann range search for expr {}, src_col_idx {}, cid {}, 
row_bitmap "
diff --git a/be/src/vec/exprs/vexpr_context.h b/be/src/vec/exprs/vexpr_context.h
index f02296dc268..74e37ad57ec 100644
--- a/be/src/vec/exprs/vexpr_context.h
+++ b/be/src/vec/exprs/vexpr_context.h
@@ -52,19 +52,19 @@ namespace doris::vectorized {
 class ScoreRuntime;
 using ScoreRuntimeSPtr = std::shared_ptr<ScoreRuntime>;
 
-class InvertedIndexContext {
+class IndexExecContext {
 public:
-    InvertedIndexContext(
+    IndexExecContext(
             const std::vector<ColumnId>& col_ids,
             const std::vector<std::unique_ptr<segment_v2::IndexIterator>>& 
index_iterators,
             const std::vector<vectorized::IndexFieldNameAndTypePair>& 
storage_name_and_type_vec,
             std::unordered_map<ColumnId, std::unordered_map<const 
vectorized::VExpr*, bool>>&
-                    common_expr_inverted_index_status,
+                    common_expr_index_status,
             ScoreRuntimeSPtr score_runtime)
             : _col_ids(col_ids),
               _index_iterators(index_iterators),
               _storage_name_and_type(storage_name_and_type_vec),
-              _expr_inverted_index_status(common_expr_inverted_index_status),
+              _expr_index_status(common_expr_index_status),
               _score_runtime(std::move(score_runtime)) {}
 
     segment_v2::IndexIterator* get_inverted_index_iterator_by_column_id(int 
column_index) const {
@@ -93,46 +93,45 @@ public:
         return &_storage_name_and_type[column_id];
     }
 
-    bool has_inverted_index_result_for_expr(const vectorized::VExpr* expr) 
const {
-        return _inverted_index_result_bitmap.contains(expr);
+    bool has_index_result_for_expr(const vectorized::VExpr* expr) const {
+        return _index_result_bitmap.contains(expr);
     }
 
-    void set_inverted_index_result_for_expr(const vectorized::VExpr* expr,
-                                            
segment_v2::InvertedIndexResultBitmap bitmap) {
-        _inverted_index_result_bitmap[expr] = std::move(bitmap);
+    void set_index_result_for_expr(const vectorized::VExpr* expr,
+                                   segment_v2::InvertedIndexResultBitmap 
bitmap) {
+        _index_result_bitmap[expr] = std::move(bitmap);
     }
 
     std::unordered_map<const vectorized::VExpr*, 
segment_v2::InvertedIndexResultBitmap>&
-    get_inverted_index_result_bitmap() {
-        return _inverted_index_result_bitmap;
+    get_index_result_bitmap() {
+        return _index_result_bitmap;
     }
 
-    std::unordered_map<const vectorized::VExpr*, ColumnPtr>& 
get_inverted_index_result_column() {
-        return _inverted_index_result_column;
+    std::unordered_map<const vectorized::VExpr*, ColumnPtr>& 
get_index_result_column() {
+        return _index_result_column;
     }
 
-    const segment_v2::InvertedIndexResultBitmap* 
get_inverted_index_result_for_expr(
+    const segment_v2::InvertedIndexResultBitmap* get_index_result_for_expr(
             const vectorized::VExpr* expr) {
-        auto iter = _inverted_index_result_bitmap.find(expr);
-        if (iter == _inverted_index_result_bitmap.end()) {
+        auto iter = _index_result_bitmap.find(expr);
+        if (iter == _index_result_bitmap.end()) {
             return nullptr;
         }
         return &iter->second;
     }
 
-    void set_inverted_index_result_column_for_expr(const vectorized::VExpr* 
expr,
-                                                   ColumnPtr column) {
-        _inverted_index_result_column[expr] = std::move(column);
+    void set_index_result_column_for_expr(const vectorized::VExpr* expr, 
ColumnPtr column) {
+        _index_result_column[expr] = std::move(column);
     }
 
-    void set_true_for_inverted_index_status(const vectorized::VExpr* expr, int 
column_index) {
+    void set_true_for_index_status(const vectorized::VExpr* expr, int 
column_index) {
         if (column_index < 0 || column_index >= _col_ids.size()) {
             return;
         }
         const auto& column_id = _col_ids[column_index];
-        if (_expr_inverted_index_status.contains(column_id)) {
-            if (_expr_inverted_index_status[column_id].contains(expr)) {
-                _expr_inverted_index_status[column_id][expr] = true;
+        if (_expr_index_status.contains(column_id)) {
+            if (_expr_index_status[column_id].contains(expr)) {
+                _expr_index_status[column_id][expr] = true;
             }
         }
     }
@@ -151,14 +150,14 @@ private:
 
     // A map of expressions to their corresponding inverted index result 
bitmaps.
     std::unordered_map<const vectorized::VExpr*, 
segment_v2::InvertedIndexResultBitmap>
-            _inverted_index_result_bitmap;
+            _index_result_bitmap;
 
     // A map of expressions to their corresponding result columns.
-    std::unordered_map<const vectorized::VExpr*, ColumnPtr> 
_inverted_index_result_column;
+    std::unordered_map<const vectorized::VExpr*, ColumnPtr> 
_index_result_column;
 
     // A reference to a map of common expressions to their inverted index 
evaluation status.
     std::unordered_map<ColumnId, std::unordered_map<const vectorized::VExpr*, 
bool>>&
-            _expr_inverted_index_status;
+            _expr_index_status;
 
     ScoreRuntimeSPtr _score_runtime;
 };
@@ -178,13 +177,11 @@ public:
 
     VExprSPtr root() { return _root; }
     void set_root(const VExprSPtr& expr) { _root = expr; }
-    void set_inverted_index_context(std::shared_ptr<InvertedIndexContext> 
inverted_index_context) {
-        _inverted_index_context = std::move(inverted_index_context);
+    void set_index_context(std::shared_ptr<IndexExecContext> index_context) {
+        _index_context = std::move(index_context);
     }
 
-    std::shared_ptr<InvertedIndexContext> get_inverted_index_context() const {
-        return _inverted_index_context;
-    }
+    std::shared_ptr<IndexExecContext> get_index_context() const { return 
_index_context; }
 
     /// Creates a FunctionContext, and returns the index that's passed to 
fn_context() to
     /// retrieve the created context. Exprs that need a FunctionContext should 
call this in
@@ -340,7 +337,7 @@ private:
     // Force to materialize even if the slot need_materialize is false, we 
just ignore need_materialize flag
     bool _force_materialize_slot = false;
 
-    std::shared_ptr<InvertedIndexContext> _inverted_index_context;
+    std::shared_ptr<IndexExecContext> _index_context;
     size_t _memory_usage = 0;
 
     segment_v2::AnnRangeSearchRuntime _ann_range_search_runtime;
diff --git a/be/src/vec/exprs/vsearch.cpp b/be/src/vec/exprs/vsearch.cpp
index a0ea7a13306..0bba97626f6 100644
--- a/be/src/vec/exprs/vsearch.cpp
+++ b/be/src/vec/exprs/vsearch.cpp
@@ -46,7 +46,7 @@ Status collect_search_inputs(const VSearchExpr& expr, 
VExprContext* context,
                              SearchInputBundle* bundle) {
     DCHECK(bundle != nullptr);
 
-    auto index_context = context->get_inverted_index_context();
+    auto index_context = context->get_index_context();
     if (index_context == nullptr) {
         LOG(WARNING) << "collect_search_inputs: No inverted index context 
available";
         return Status::InternalError("No inverted index context available");
@@ -141,7 +141,7 @@ Status VSearchExpr::evaluate_inverted_index(VExprContext* 
context, uint32_t segm
         return Status::InvalidArgument("search DSL is empty");
     }
 
-    auto index_context = context->get_inverted_index_context();
+    auto index_context = context->get_index_context();
     if (!index_context) {
         LOG(WARNING) << "VSearchExpr: No inverted index context available";
         return Status::OK();
@@ -157,7 +157,7 @@ Status VSearchExpr::evaluate_inverted_index(VExprContext* 
context, uint32_t segm
                      << _original_dsl;
         auto empty_bitmap = 
InvertedIndexResultBitmap(std::make_shared<roaring::Roaring>(),
                                                       
std::make_shared<roaring::Roaring>());
-        index_context->set_inverted_index_result_for_expr(this, 
std::move(empty_bitmap));
+        index_context->set_index_result_for_expr(this, 
std::move(empty_bitmap));
         return Status::OK();
     }
 
@@ -171,9 +171,9 @@ Status VSearchExpr::evaluate_inverted_index(VExprContext* 
context, uint32_t segm
         return status;
     }
 
-    index_context->set_inverted_index_result_for_expr(this, result_bitmap);
+    index_context->set_index_result_for_expr(this, result_bitmap);
     for (int column_id : bundle.column_ids) {
-        index_context->set_true_for_inverted_index_status(this, column_id);
+        index_context->set_true_for_index_status(this, column_id);
     }
 
     return Status::OK();
diff --git a/be/test/vec/exprs/vexpr_evalute_inverted_index_test.cpp 
b/be/test/vec/exprs/vexpr_evalute_inverted_index_test.cpp
index 45db081e49b..0fd4c73cdb1 100644
--- a/be/test/vec/exprs/vexpr_evalute_inverted_index_test.cpp
+++ b/be/test/vec/exprs/vexpr_evalute_inverted_index_test.cpp
@@ -158,18 +158,17 @@ TEST(TExprInvertedIndexTest, 
test_expr_evaluate_inverted_index) {
 
     std::unordered_map<doris::ColumnId, std::unordered_map<const 
doris::vectorized::VExpr*, bool>>
             common_expr_inverted_index_status;
-    auto inverted_index_context = 
std::make_shared<doris::vectorized::InvertedIndexContext>(
+    auto inverted_index_context = 
std::make_shared<doris::vectorized::IndexExecContext>(
             std::vector<doris::ColumnId>(),
             std::vector<std::unique_ptr<doris::segment_v2::IndexIterator>>(),
             std::vector<doris::vectorized::IndexFieldNameAndTypePair>(),
             common_expr_inverted_index_status, nullptr);
-    expr_ctx.set_inverted_index_context(inverted_index_context);
+    expr_ctx.set_index_context(inverted_index_context);
     doris::RuntimeState state;
     doris::RowDescriptor row_desc;
     EXPECT_TRUE(in_expr->prepare(&state, row_desc, &expr_ctx).ok());
     EXPECT_TRUE(in_expr->evaluate_inverted_index(&expr_ctx, 100).ok());
-    
EXPECT_FALSE(expr_ctx.get_inverted_index_context()->has_inverted_index_result_for_expr(
-            in_expr.get()));
-    
EXPECT_TRUE(expr_ctx.get_inverted_index_context()->_inverted_index_result_bitmap.empty());
-    
EXPECT_TRUE(expr_ctx.get_inverted_index_context()->_expr_inverted_index_status.empty());
+    
EXPECT_FALSE(expr_ctx.get_index_context()->has_index_result_for_expr(in_expr.get()));
+    EXPECT_TRUE(expr_ctx.get_index_context()->_index_result_bitmap.empty());
+    EXPECT_TRUE(expr_ctx.get_index_context()->_expr_index_status.empty());
 }
\ No newline at end of file
diff --git a/be/test/vec/exprs/vsearch_expr_test.cpp 
b/be/test/vec/exprs/vsearch_expr_test.cpp
index 7b33ca121d2..acd3def2f69 100644
--- a/be/test/vec/exprs/vsearch_expr_test.cpp
+++ b/be/test/vec/exprs/vsearch_expr_test.cpp
@@ -92,13 +92,13 @@ VExprSPtr create_slot_ref(int column_id, const std::string& 
column_name) {
     return slot;
 }
 
-std::shared_ptr<InvertedIndexContext> make_inverted_context(
+std::shared_ptr<IndexExecContext> make_inverted_context(
         std::vector<ColumnId>& col_ids,
         std::vector<std::unique_ptr<segment_v2::IndexIterator>>& 
index_iterators,
         std::vector<IndexFieldNameAndTypePair>& storage_types,
         std::unordered_map<ColumnId, std::unordered_map<const VExpr*, bool>>& 
status_map) {
-    return std::make_shared<InvertedIndexContext>(col_ids, index_iterators, 
storage_types,
-                                                  status_map, nullptr);
+    return std::make_shared<IndexExecContext>(col_ids, index_iterators, 
storage_types, status_map,
+                                              nullptr);
 }
 
 } // namespace
@@ -1068,8 +1068,8 @@ TEST_F(VSearchExprTest, 
TestCollectSearchInputsWithUnsupportedChildType) {
     VExprContext context(dummy_expr);
 
     // This should trigger the collect_search_inputs function, but since we 
don't have
-    // a real InvertedIndexContext, it will return early with Status::OK
-    // If we had a real InvertedIndexContext, it would reach the unsupported 
child type error
+    // a real IndexExecContext, it will return early with Status::OK
+    // If we had a real IndexExecContext, it would reach the unsupported child 
type error
     auto status = vsearch_expr->evaluate_inverted_index(&context, 100);
     EXPECT_TRUE(status.ok()); // Early return due to nullptr index_context
 }
@@ -1145,7 +1145,7 @@ TEST_F(VSearchExprTest, 
TestEvaluateInvertedIndexWithEmptyIterators) {
     // This covers lines 138-141 in evaluate_inverted_index
     auto vsearch_expr = VSearchExpr::create_shared(test_node);
 
-    // Create a mock InvertedIndexContext that returns empty iterators
+    // Create a mock IndexExecContext that returns empty iterators
     // For now, we test the early return path when index_context is nullptr
     auto dummy_expr = VSearchExpr::create_shared(test_node);
     VExprContext context(dummy_expr);
@@ -1301,8 +1301,8 @@ TEST_F(VSearchExprTest, 
FastExecuteReturnsPrecomputedColumn) {
 
     auto inverted_ctx = make_inverted_context(col_ids, index_iterators, 
storage_types, status_map);
     MutableColumnPtr result_column = ColumnUInt8::create();
-    inverted_ctx->set_inverted_index_result_column_for_expr(expr.get(), 
std::move(result_column));
-    context->set_inverted_index_context(inverted_ctx);
+    inverted_ctx->set_index_result_column_for_expr(expr.get(), 
std::move(result_column));
+    context->set_index_context(inverted_ctx);
 
     Block block;
     int result_column_id = -1;
@@ -1325,7 +1325,7 @@ TEST_F(VSearchExprTest, 
EvaluateInvertedIndexFailsWithoutStorageType) {
 
     auto inverted_ctx = make_inverted_context(col_ids, index_iterators, 
storage_types, status_map);
     auto context = std::make_shared<VExprContext>(expr);
-    context->set_inverted_index_context(inverted_ctx);
+    context->set_index_context(inverted_ctx);
 
     auto status = expr->evaluate_inverted_index(context.get(), 128);
     EXPECT_FALSE(status.ok());
@@ -1343,7 +1343,7 @@ TEST_F(VSearchExprTest, 
EvaluateInvertedIndexWithUnsupportedChildReturnsError) {
 
     auto inverted_ctx = make_inverted_context(col_ids, index_iterators, 
storage_types, status_map);
     auto context = std::make_shared<VExprContext>(expr);
-    context->set_inverted_index_context(inverted_ctx);
+    context->set_index_context(inverted_ctx);
 
     auto status = expr->evaluate_inverted_index(context.get(), 64);
     EXPECT_FALSE(status.ok());
@@ -1363,7 +1363,7 @@ TEST_F(VSearchExprTest, 
EvaluateInvertedIndexHandlesMissingIterators) {
 
     auto inverted_ctx = make_inverted_context(col_ids, index_iterators, 
storage_types, status_map);
     auto context = std::make_shared<VExprContext>(expr);
-    context->set_inverted_index_context(inverted_ctx);
+    context->set_index_context(inverted_ctx);
 
     auto status = expr->evaluate_inverted_index(context.get(), 32);
     EXPECT_TRUE(status.ok());
@@ -1384,7 +1384,7 @@ TEST_F(VSearchExprTest, 
EvaluateInvertedIndexPropagatesFunctionFailure) {
 
     auto inverted_ctx = make_inverted_context(col_ids, index_iterators, 
storage_types, status_map);
     auto context = std::make_shared<VExprContext>(expr);
-    context->set_inverted_index_context(inverted_ctx);
+    context->set_index_context(inverted_ctx);
 
     auto status = expr->evaluate_inverted_index(context.get(), 256);
     EXPECT_FALSE(status.ok());
@@ -1392,7 +1392,7 @@ TEST_F(VSearchExprTest, 
EvaluateInvertedIndexPropagatesFunctionFailure) {
     EXPECT_FALSE(status_map[0][expr.get()]);
 }
 
-// Note: Full testing with actual InvertedIndexContext and real iterators
+// Note: Full testing with actual IndexExecContext and real iterators
 // would require complex setup and is better suited for integration tests
 // The tests above cover the main execution paths in evaluate_inverted_index
 


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


Reply via email to