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

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

commit 92010b66001dea9a76fc14dda74e12e8696fefa3
Author: zhiqiang <hezhiqi...@selectdb.com>
AuthorDate: Tue May 20 19:40:49 2025 +0800

    fix virtual column expr context (#51085)
    
    failure of tpchq13 is fixed.
---
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |  5 +-
 .../rowset/segment_v2/virtual_column_iterator.cpp  |  9 ---
 be/src/pipeline/exec/olap_scan_operator.cpp        | 14 ++--
 be/src/vec/exec/scan/olap_scanner.cpp              |  1 +
 be/src/vec/exec/scan/scanner.cpp                   |  4 --
 be/src/vec/exprs/vann_topn_predicate.cpp           |  4 +-
 be/src/vec/exprs/vectorized_fn_call.cpp            |  3 +-
 be/src/vec/exprs/vectorized_fn_call.h              |  2 +-
 be/src/vec/exprs/virtual_slot_ref.cpp              | 38 +++--------
 be/src/vec/exprs/virtual_slot_ref.h                |  6 +-
 be/src/vec/olap/vcollect_iterator.cpp              |  1 +
 .../vector_search/ann_topn_descriptor_test.cpp     | 28 +++++---
 .../olap/vector_search/faiss_vector_index_test.cpp |  8 +--
 be/test/olap/vector_search/vector_search_utils.h   | 78 +++++++++++++++++++---
 14 files changed, 123 insertions(+), 78 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp 
b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 7949065421a..caee4851224 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -844,8 +844,7 @@ Status SegmentIterator::_extract_common_expr_columns(const 
vectorized::VExprSPtr
     } else if (node_type == TExprNodeType::VIRTUAL_SLOT_REF) {
         std::shared_ptr<vectorized::VirtualSlotRef> virtual_slot_ref =
                 std::dynamic_pointer_cast<vectorized::VirtualSlotRef>(expr);
-        auto column_expr = virtual_slot_ref->get_virtual_column_expr();
-        RETURN_IF_ERROR(_extract_common_expr_columns(column_expr->root()));
+        
RETURN_IF_ERROR(_extract_common_expr_columns(virtual_slot_ref->get_virtual_column_expr()));
     }
 
     return Status::OK();
@@ -2153,6 +2152,8 @@ Status SegmentIterator::next_batch(vectorized::Block* 
block) {
 
                 return res;
             }
+
+            RETURN_IF_ERROR(res);
             // reverse block row order if read_orderby_key_reverse is true for 
key topn
             // it should be processed for all success _next_batch_internal
             if (_opts.read_orderby_key_reverse) {
diff --git a/be/src/olap/rowset/segment_v2/virtual_column_iterator.cpp 
b/be/src/olap/rowset/segment_v2/virtual_column_iterator.cpp
index 80d230bec31..767e03f89c3 100644
--- a/be/src/olap/rowset/segment_v2/virtual_column_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/virtual_column_iterator.cpp
@@ -72,15 +72,6 @@ Status VirtualColumnIterator::read_by_rowids(const rowid_t* 
rowids, const size_t
         _filter[_row_id_to_idx[rowids[i]]] = 1;
     }
 
-    // Count one of the filter
-    size_t count_one = 0;
-    for (size_t i = 0; i < _size; ++i) {
-        if (_filter[i] == 1) {
-            count_one++;
-        }
-    }
-    std::cout << "count_one: " << count_one << std::endl;
-
     // Apply filter to materialized column
     doris::vectorized::IColumn::Ptr res_col = 
_materialized_column_ptr->filter(_filter, 0);
     // Update dst column
diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp 
b/be/src/pipeline/exec/olap_scan_operator.cpp
index cfbd5c665b3..8ae9b2dd5ef 100644
--- a/be/src/pipeline/exec/olap_scan_operator.cpp
+++ b/be/src/pipeline/exec/olap_scan_operator.cpp
@@ -594,6 +594,8 @@ Status OlapScanLocalState::init(RuntimeState* state, 
LocalStateInfo& info) {
     const TOlapScanNode& olap_scan_node = 
_parent->cast<OlapScanOperatorX>()._olap_scan_node;
 
     if (olap_scan_node.__isset.ann_sort_info || 
olap_scan_node.__isset.ann_sort_limit) {
+        LOG_INFO("Ann sort info: {}",
+                 
apache::thrift::ThriftDebugString(olap_scan_node.ann_sort_info));
         DCHECK(olap_scan_node.__isset.ann_sort_info);
         DCHECK(olap_scan_node.__isset.ann_sort_limit);
         DCHECK(olap_scan_node.ann_sort_info.ordering_exprs.size() == 1);
@@ -604,7 +606,6 @@ Status OlapScanLocalState::init(RuntimeState* state, 
LocalStateInfo& info) {
         size_t limit = olap_scan_node.ann_sort_limit;
         std::shared_ptr<vectorized::VExprContext> ordering_expr_ctx;
         RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(ordering_expr, 
ordering_expr_ctx));
-        LOG_INFO("Ann thrift expr: {}", 
apache::thrift::ThriftDebugString(ordering_expr));
         _ann_topn_descriptor =
                 vectorized::AnnTopNDescriptor::create_shared(limit, 
ordering_expr_ctx);
     }
@@ -626,20 +627,21 @@ Status OlapScanLocalState::open(RuntimeState* state) {
 
             _slot_id_to_virtual_column_expr[slot_desc->id()] = 
virtual_column_expr_ctx;
             _slot_id_to_col_type[slot_desc->id()] = 
slot_desc->get_data_type_ptr();
-            int cid = p.intermediate_row_desc().get_column_id(slot_desc->id());
-            if (cid < 0) {
+            int col_pos = 
p.intermediate_row_desc().get_column_id(slot_desc->id());
+            if (col_pos < 0) {
                 return Status::InternalError(
                         "Invalid virtual slot, can not find its information. 
Slot desc:\n{}\nRow "
                         "desc:\n{}",
                         slot_desc->debug_string(), 
p.row_desc().debug_string());
             } else {
-                _slot_id_to_index_in_block[slot_desc->id()] = cid;
+                _slot_id_to_index_in_block[slot_desc->id()] = col_pos;
             }
 
             LOG_INFO(
-                    "OlapScanLocalState opening, virtual column expr slot id: 
{}, cid: {}, expr: "
+                    "OlapScanLocalState opening, virtual column expr slot id: 
{}, col_pos: {}, "
+                    "expr: "
                     "{}",
-                    slot_desc->id(), cid, 
virtual_column_expr_ctx->root()->debug_string());
+                    slot_desc->id(), col_pos, 
virtual_column_expr_ctx->root()->debug_string());
         }
     }
 
diff --git a/be/src/vec/exec/scan/olap_scanner.cpp 
b/be/src/vec/exec/scan/olap_scanner.cpp
index 3723dd4d4bc..8f49ac0399e 100644
--- a/be/src/vec/exec/scan/olap_scanner.cpp
+++ b/be/src/vec/exec/scan/olap_scanner.cpp
@@ -152,6 +152,7 @@ Status OlapScanner::init() {
     }
 
     for (auto pair : local_state->_slot_id_to_virtual_column_expr) {
+        // Scanner will be executed in a different thread, so we need to clone 
the context.
         VExprContextSPtr context;
         RETURN_IF_ERROR(pair.second->clone(_state, context));
         _slot_id_to_virtual_column_expr[pair.first] = context;
diff --git a/be/src/vec/exec/scan/scanner.cpp b/be/src/vec/exec/scan/scanner.cpp
index 72c0c679d6b..8235be56ef5 100644
--- a/be/src/vec/exec/scan/scanner.cpp
+++ b/be/src/vec/exec/scan/scanner.cpp
@@ -100,7 +100,6 @@ Status Scanner::get_block(RuntimeState* state, Block* 
block, bool* eof) {
                                                 slot_desc->get_data_type_ptr(),
                                                 slot_desc->col_name()));
         }
-        LOG_INFO("Block columns count: {}", block->columns());
     }
 
 #ifndef BE_TEST
@@ -207,9 +206,6 @@ Status Scanner::_do_projections(vectorized::Block* 
origin_block, vectorized::Blo
             reinterpret_cast<ColumnNullable*>(mutable_columns[i].get())
                     ->insert_range_from_not_nullable(*column_ptr, 0, rows);
         } else {
-            LOG_INFO(" Projections {} insert_range_from {} to {}",
-                     _projections[i]->root()->debug_string(), 
column_ptr->get_name(),
-                     mutable_columns[i]->get_name());
             mutable_columns[i]->insert_range_from(*column_ptr, 0, rows);
         }
     }
diff --git a/be/src/vec/exprs/vann_topn_predicate.cpp 
b/be/src/vec/exprs/vann_topn_predicate.cpp
index 85e29961c7a..41098e43a07 100644
--- a/be/src/vec/exprs/vann_topn_predicate.cpp
+++ b/be/src/vec/exprs/vann_topn_predicate.cpp
@@ -70,11 +70,11 @@ Status AnnTopNDescriptor::prepare(RuntimeState* state, 
const RowDescriptor& row_
     _dest_column_idx = vir_slot_ref->column_id();
     auto vir_col_expr = vir_slot_ref->get_virtual_column_expr();
     std::shared_ptr<VectorizedFnCall> distance_fn_call =
-            std::dynamic_pointer_cast<VectorizedFnCall>(vir_col_expr->root());
+            std::dynamic_pointer_cast<VectorizedFnCall>(vir_col_expr);
 
     if (distance_fn_call == nullptr) {
         return Status::InternalError("Ann topn expr expect FuncationCall, 
got\n{}",
-                                     vir_col_expr->root()->debug_string());
+                                     vir_col_expr->debug_string());
     }
 
     std::shared_ptr<VCastExpr> cast_to_array_expr =
diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp 
b/be/src/vec/exprs/vectorized_fn_call.cpp
index bad13b11a25..6fdfeb2a95f 100644
--- a/be/src/vec/exprs/vectorized_fn_call.cpp
+++ b/be/src/vec/exprs/vectorized_fn_call.cpp
@@ -364,9 +364,8 @@ Status VectorizedFnCall::prepare_ann_range_search() {
     auto vir_slot_ref = std::dynamic_pointer_cast<VirtualSlotRef>(left_child);
     if (vir_slot_ref != nullptr) {
         DCHECK(vir_slot_ref->get_virtual_column_expr() != nullptr);
-        DCHECK(vir_slot_ref->get_virtual_column_expr()->root() != nullptr);
         function_call = std::dynamic_pointer_cast<VectorizedFnCall>(
-                vir_slot_ref->get_virtual_column_expr()->root());
+                vir_slot_ref->get_virtual_column_expr());
     } else {
         function_call = 
std::dynamic_pointer_cast<VectorizedFnCall>(left_child);
     }
diff --git a/be/src/vec/exprs/vectorized_fn_call.h 
b/be/src/vec/exprs/vectorized_fn_call.h
index 3c9cb5e7350..4f3f76b5436 100644
--- a/be/src/vec/exprs/vectorized_fn_call.h
+++ b/be/src/vec/exprs/vectorized_fn_call.h
@@ -76,7 +76,7 @@ public:
 
     size_t estimate_memory(const size_t rows) override;
 
-    MOCK_FUNCTION Status evaluate_ann_range_search(
+    Status evaluate_ann_range_search(
             const std::vector<std::unique_ptr<segment_v2::IndexIterator>>& 
cid_to_index_iterators,
             const std::vector<ColumnId>& idx_to_cid,
             const std::vector<std::unique_ptr<segment_v2::ColumnIterator>>& 
column_iterators,
diff --git a/be/src/vec/exprs/virtual_slot_ref.cpp 
b/be/src/vec/exprs/virtual_slot_ref.cpp
index 35f077deff8..d7c56b4735f 100644
--- a/be/src/vec/exprs/virtual_slot_ref.cpp
+++ b/be/src/vec/exprs/virtual_slot_ref.cpp
@@ -35,6 +35,7 @@
 #include "vec/core/column_with_type_and_name.h"
 #include "vec/exprs/vectorized_fn_call.h"
 #include "vec/exprs/vexpr_context.h"
+#include "vec/exprs/vexpr_fwd.h"
 
 namespace doris::vectorized {
 
@@ -94,9 +95,12 @@ Status VirtualSlotRef::prepare(doris::RuntimeState* state, 
const doris::RowDescr
     }
     const TExpr& expr = *slot_desc->get_virtual_column_expr();
     LOG_INFO("Virtual column expr is {}", 
apache::thrift::ThriftDebugString(expr));
-    RETURN_IF_ERROR(VExpr::create_expr_tree(expr, _virtual_column_expr));
-    RETURN_IF_ERROR(_virtual_column_expr->prepare(state, desc));
-
+    // Create a temp_ctx only for create_expr_tree.
+    VExprContextSPtr temp_ctx;
+    RETURN_IF_ERROR(VExpr::create_expr_tree(expr, temp_ctx));
+    _virtual_column_expr = temp_ctx->root();
+    // Virtual column expr should do prepare with original context.
+    RETURN_IF_ERROR(_virtual_column_expr->prepare(state, desc, context));
     _prepare_finished = true;
     return Status::OK();
 }
@@ -104,7 +108,7 @@ Status VirtualSlotRef::prepare(doris::RuntimeState* state, 
const doris::RowDescr
 Status VirtualSlotRef::open(RuntimeState* state, VExprContext* context,
                             FunctionContext::FunctionStateScope scope) {
     DCHECK(_prepare_finished);
-    RETURN_IF_ERROR(_virtual_column_expr->open(state));
+    RETURN_IF_ERROR(_virtual_column_expr->open(state, context, scope));
     RETURN_IF_ERROR(VExpr::open(state, context, scope));
     _open_finished = true;
     return Status::OK();
@@ -136,7 +140,7 @@ Status VirtualSlotRef::execute(VExprContext* context, 
Block* block, int* result_
             // 注意,在执行 execute 之后,后续代码里不能在用 120 行的 column 了,因为 execute 的时候 
vector 可能发生
             // resize,导致之前的引用实效。
             int tmp_column_id = -1;
-            RETURN_IF_ERROR(_virtual_column_expr->execute(block, 
&tmp_column_id));
+            RETURN_IF_ERROR(_virtual_column_expr->execute(context, block, 
&tmp_column_id));
 
             // Maybe do clone.
             block->replace_by_position(_column_id,
@@ -205,34 +209,14 @@ bool VirtualSlotRef::equals(const VExpr& other) {
     return true;
 }
 
-// void VirtualSlotRef::prepare_virtual_slots(
-//         const std::map<SlotId, vectorized::VExprContextSPtr>& 
_slot_id_to_virtual_column_expr) {
-//     DCHECK(_children.empty());
-
-//     if (_slot_id_to_virtual_column_expr.find(_slot_id) != 
_slot_id_to_virtual_column_expr.end()) {
-//         _virtual_column_expr = _slot_id_to_virtual_column_expr.at(_slot_id);
-//     } else {
-//         // TODO: FIX debug_string 函数有core dump 的可能
-//         std::string msg;
-//         for (auto& it : _slot_id_to_virtual_column_expr) {
-//             msg += fmt::format("slot_id: {}, virtual_column_expr: {}, ", 
it.first,
-//                                it.second->root()->debug_string());
-//         }
-//         throw doris::Exception(doris::ErrorCode::INTERNAL_ERROR,
-//                                "VirtualSlotRef {} has no virtual column 
expr, slot_id: {}, "
-//                                "slot_id_to_virtual_column_expr: {}",
-//                                *_column_name, _slot_id, msg);
-//     }
-// }
-
 Status VirtualSlotRef::evaluate_ann_range_search(
         const std::vector<std::unique_ptr<segment_v2::IndexIterator>>& 
cid_to_index_iterators,
         const std::vector<ColumnId>& idx_to_cid,
         const std::vector<std::unique_ptr<segment_v2::ColumnIterator>>& 
column_iterators,
         roaring::Roaring& row_bitmap) {
     if (_virtual_column_expr != nullptr) {
-        return _virtual_column_expr->root()->evaluate_ann_range_search(
-                cid_to_index_iterators, idx_to_cid, column_iterators, 
row_bitmap);
+        return 
_virtual_column_expr->evaluate_ann_range_search(cid_to_index_iterators, 
idx_to_cid,
+                                                               
column_iterators, row_bitmap);
     }
     return Status::OK();
 }
diff --git a/be/src/vec/exprs/virtual_slot_ref.h 
b/be/src/vec/exprs/virtual_slot_ref.h
index 6a7d61517fb..9bd184cd5b7 100644
--- a/be/src/vec/exprs/virtual_slot_ref.h
+++ b/be/src/vec/exprs/virtual_slot_ref.h
@@ -44,7 +44,7 @@ public:
     void collect_slot_column_ids(std::set<int>& column_ids) const override {
         column_ids.insert(_column_id);
     }
-    std::shared_ptr<VExprContext> get_virtual_column_expr() const { return 
_virtual_column_expr; }
+    std::shared_ptr<VExpr> get_virtual_column_expr() const { return 
_virtual_column_expr; }
     // void prepare_virtual_slots(const std::map<SlotId, 
vectorized::VExprContextSPtr>&
     //                                    _slot_id_to_virtual_column_expr) 
override;
 
@@ -89,7 +89,7 @@ public:
     void set_column_data_type(DataTypePtr column_data_type) {
         _column_data_type = std::move(column_data_type);
     }
-    void set_virtual_column_expr(std::shared_ptr<VExprContext> 
virtual_column_expr) {
+    void set_virtual_column_expr(std::shared_ptr<VExpr> virtual_column_expr) {
         _virtual_column_expr = virtual_column_expr;
     }
 #endif
@@ -99,7 +99,7 @@ private:
     int _slot_id;
     const std::string* _column_name;
     const std::string _column_label;
-    std::shared_ptr<VExprContext> _virtual_column_expr;
+    std::shared_ptr<VExpr> _virtual_column_expr;
     DataTypePtr _column_data_type;
 };
 
diff --git a/be/src/vec/olap/vcollect_iterator.cpp 
b/be/src/vec/olap/vcollect_iterator.cpp
index 0127160005a..2b4d6650824 100644
--- a/be/src/vec/olap/vcollect_iterator.cpp
+++ b/be/src/vec/olap/vcollect_iterator.cpp
@@ -484,6 +484,7 @@ void VCollectIterator::Level0Iterator::init_for_union(bool 
get_data_by_ref) {
 
 Status VCollectIterator::Level0Iterator::ensure_first_row_ref() {
     DCHECK(!_get_data_by_ref);
+    // TODO: Why not return error directly?
     auto s = refresh_current_row();
     _ref = {_block, 0, false};
 
diff --git a/be/test/olap/vector_search/ann_topn_descriptor_test.cpp 
b/be/test/olap/vector_search/ann_topn_descriptor_test.cpp
index 8df7b62770a..b08adf73c32 100644
--- a/be/test/olap/vector_search/ann_topn_descriptor_test.cpp
+++ b/be/test/olap/vector_search/ann_topn_descriptor_test.cpp
@@ -39,20 +39,30 @@ namespace doris::vectorized {
 TEST_F(VectorSearchTest, AnnTopNDescriptorConstructor) {
     int limit = 10;
     std::shared_ptr<VExprContext> distanc_calcu_fn_call_ctx;
-    
ASSERT_TRUE(vectorized::VExpr::create_expr_tree(_distance_function_call_thrift,
-                                                    distanc_calcu_fn_call_ctx)
-                        .ok());
+    auto distance_function_call_thrift = 
read_from_json<TExpr>(_distance_function_call_thrift);
+    ASSERT_TRUE(distance_function_call_thrift.nodes.empty() != true);
+    auto st1 = 
vectorized::VExpr::create_expr_tree(distance_function_call_thrift,
+                                                   distanc_calcu_fn_call_ctx);
+    ASSERT_TRUE(st1.ok()) << fmt::format(
+            "st: {}, expr {}", st1.to_string(),
+            apache::thrift::ThriftDebugString(distance_function_call_thrift));
+    ASSERT_TRUE(distanc_calcu_fn_call_ctx != nullptr) << "create expr tree 
failed";
+    ASSERT_TRUE(distanc_calcu_fn_call_ctx->root() != nullptr);
 
     std::shared_ptr<VExprContext> virtual_slot_expr_ctx;
     ASSERT_TRUE(vectorized::VExpr::create_expr_tree(_virtual_slot_ref_expr, 
virtual_slot_expr_ctx)
                         .ok());
+
+    ASSERT_TRUE(virtual_slot_expr_ctx != nullptr) << "create expr tree failed";
+    ASSERT_TRUE(virtual_slot_expr_ctx->root() != nullptr);
+
     std::shared_ptr<VirtualSlotRef> v =
             
std::dynamic_pointer_cast<VirtualSlotRef>(virtual_slot_expr_ctx->root());
     if (v == nullptr) {
         LOG(FATAL) << "VAnnTopNDescriptor::SetUp() failed";
     }
 
-    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx);
+    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx->root());
 
     std::shared_ptr<AnnTopNDescriptor> predicate;
     predicate = AnnTopNDescriptor::create_shared(limit, virtual_slot_expr_ctx);
@@ -62,7 +72,8 @@ TEST_F(VectorSearchTest, AnnTopNDescriptorConstructor) {
 TEST_F(VectorSearchTest, AnnTopNDescriptorPrepare) {
     int limit = 10;
     std::shared_ptr<VExprContext> distanc_calcu_fn_call_ctx;
-    Status st = 
vectorized::VExpr::create_expr_tree(_distance_function_call_thrift,
+    auto distance_function_call_thrift = 
read_from_json<TExpr>(_distance_function_call_thrift);
+    Status st = 
vectorized::VExpr::create_expr_tree(distance_function_call_thrift,
                                                     distanc_calcu_fn_call_ctx);
 
     std::shared_ptr<VExprContext> virtual_slot_expr_ctx;
@@ -73,7 +84,7 @@ TEST_F(VectorSearchTest, AnnTopNDescriptorPrepare) {
         LOG(FATAL) << "VAnnTopNDescriptor::SetUp() failed";
     }
 
-    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx);
+    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx->root());
     std::shared_ptr<AnnTopNDescriptor> predicate;
     predicate = AnnTopNDescriptor::create_shared(limit, virtual_slot_expr_ctx);
     st = predicate->prepare(&_runtime_state, _row_desc);
@@ -86,7 +97,8 @@ TEST_F(VectorSearchTest, AnnTopNDescriptorPrepare) {
 TEST_F(VectorSearchTest, AnnTopNDescriptorEvaluateTopN) {
     int limit = 10;
     std::shared_ptr<VExprContext> distanc_calcu_fn_call_ctx;
-    Status st = 
vectorized::VExpr::create_expr_tree(_distance_function_call_thrift,
+    auto distance_function_call_thrift = 
read_from_json<TExpr>(_distance_function_call_thrift);
+    Status st = 
vectorized::VExpr::create_expr_tree(distance_function_call_thrift,
                                                     distanc_calcu_fn_call_ctx);
 
     std::shared_ptr<VExprContext> virtual_slot_expr_ctx;
@@ -97,7 +109,7 @@ TEST_F(VectorSearchTest, AnnTopNDescriptorEvaluateTopN) {
         LOG(FATAL) << "VAnnTopNDescriptor::SetUp() failed";
     }
 
-    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx);
+    v->set_virtual_column_expr(distanc_calcu_fn_call_ctx->root());
     std::shared_ptr<AnnTopNDescriptor> predicate;
     predicate = AnnTopNDescriptor::create_shared(limit, virtual_slot_expr_ctx);
     st = predicate->prepare(&_runtime_state, _row_desc);
diff --git a/be/test/olap/vector_search/faiss_vector_index_test.cpp 
b/be/test/olap/vector_search/faiss_vector_index_test.cpp
index 802f20b3e9f..7c7b42611ec 100644
--- a/be/test/olap/vector_search/faiss_vector_index_test.cpp
+++ b/be/test/olap/vector_search/faiss_vector_index_test.cpp
@@ -302,7 +302,7 @@ TEST_F(VectorSearchTest, SearchAllVectors) {
     // Add 500 vectors
     const int num_vectors = 500;
     std::vector<float> vectors;
-    for (int i = 0; i < num_vectors * params.m; i++) {
+    for (int i = 0; i < num_vectors; i++) {
         auto vec = generate_random_vector(params.d);
         vectors.insert(vectors.end(), vec.begin(), vec.end());
     }
@@ -384,7 +384,7 @@ TEST_F(VectorSearchTest, CompRangeSearch) {
         for (int i = 0; i < num_vectors; i++) {
             double sum = 0;
             for (int j = 0; j < params.d; j++) {
-                accumulate(vectors[i * params.m + j], query_vec[j], sum);
+                accumulate(vectors[i * params.d + j], query_vec[j], sum);
             }
             distances[i] = std::make_pair(i, finalize(sum));
         }
@@ -462,7 +462,7 @@ TEST_F(VectorSearchTest, RangeSearchNoSelector1) {
         for (int i = 0; i < num_vectors; i++) {
             double sum = 0;
             for (int j = 0; j < params.d; j++) {
-                accumulate(vectors[i * params.m + j], vectors[j], sum);
+                accumulate(vectors[i * params.d + j], vectors[j], sum);
             }
             distances[i] = std::make_pair(i, finalize(sum));
         }
@@ -578,7 +578,7 @@ TEST_F(VectorSearchTest, RangeSearchWithSelector1) {
         for (int i = 0; i < num_vectors; i++) {
             double sum = 0;
             for (int j = 0; j < params.d; j++) {
-                accumulate(vectors[i * params.m + j], vectors[j], sum);
+                accumulate(vectors[i * params.d + j], vectors[j], sum);
             }
             distances[i] = std::make_pair(i, finalize(sum));
         }
diff --git a/be/test/olap/vector_search/vector_search_utils.h 
b/be/test/olap/vector_search/vector_search_utils.h
index 023247fc3a7..bf84e5d7739 100644
--- a/be/test/olap/vector_search/vector_search_utils.h
+++ b/be/test/olap/vector_search/vector_search_utils.h
@@ -42,7 +42,6 @@
 using doris::segment_v2::DorisCompoundReader;
 
 namespace doris::vec_search_mock {
-
 class MockIndexFileReader : public ::doris::segment_v2::IndexFileReader {
 public:
     MockIndexFileReader()
@@ -99,8 +98,6 @@ private:
 } // namespace doris::vec_search_mock
 
 namespace doris::vectorized {
-static std::string order_by_expr_thrift =
-        
R"xxx({"1":{"lst":["rec",12,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"l2_distance"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",2,{"1":{"i32":1},"4":{"tf":1},"5":{"lst":["tf",1,1]}},{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",2,{"1":{"i32":1},"4":{"tf":1},"5":{"lst":["tf",1,1]}},{"1":{"i32":0},"2":{
 [...]
 template <typename T>
 T read_from_json(const std::string& json_str) {
     auto memBufferIn = 
std::make_shared<apache::thrift::transport::TMemoryBuffer>(
@@ -135,11 +132,6 @@ public:
 
 protected:
     void SetUp() override {
-        TExpr _distance_function_call_thrift = 
read_from_json<TExpr>(order_by_expr_thrift);
-        // std::cout << "distance function call thrift:\n"
-        //           << 
apache::thrift::ThriftDebugString(_distance_function_call_thrift)
-        //           << std::endl;
-
         TDescriptorTable thrift_tbl;
         TTableDescriptor thrift_table_desc;
         thrift_table_desc.id = 0;
@@ -161,7 +153,7 @@ protected:
         scalar_type.__set_type(TPrimitiveType::DOUBLE);
         type_node.__set_scalar_type(scalar_type);
         slot_desc.slotType.types.push_back(type_node);
-        slot_desc.virtual_column_expr = _distance_function_call_thrift;
+        slot_desc.virtual_column_expr = 
read_from_json<TExpr>(_distance_function_call_thrift);
         slot_desc.__isset.virtual_column_expr = true;
         thrift_tbl.slotDescriptors.push_back(slot_desc);
         slot_desc.id = 1;
@@ -220,10 +212,76 @@ private:
     RowDescriptor _row_desc;
     std::unique_ptr<vec_search_mock::MockAnnIndexIterator> _ann_index_iterator;
     vectorized::IColumn::MutablePtr _result_column;
-    doris::TExpr _distance_function_call_thrift;
     doris::TExpr _virtual_slot_ref_expr;
     DescriptorTbl* _desc_tbl;
     doris::RuntimeState _runtime_state;
     std::shared_ptr<lucene::store::RAMDirectory> _ram_dir;
+
+    /*
+    [0] TExprNode {
+        num_children = 2
+        fn = TFunctionName {
+            name = "l2_distance"
+        }
+    },
+    [1] TExprNode {
+        num_children = 1
+        fn = TFunctionName {
+            name = "casttoarray"
+        }
+    },
+    [2] TExprNode {
+        num_children = 0,
+        slot_ref = TSlotRef {
+            slot_id = 1
+        }
+    },
+    [3] TExprNode {
+        node_type = ARRAY_LITERAL,
+        num_children = 8,
+    },
+    [4] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 1
+        }
+    },
+    [5] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 2
+        }
+    },
+    [6] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 3
+        }
+    },
+    [7] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 4
+        }
+    },
+    [8] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 5
+        }
+    },
+    [9] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 6
+        }
+    },
+    [10] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 7
+        }
+    },
+    [11] TExprNode {
+        float_literal = TFloatLiteral{
+            value = 20
+        }
+    },
+    */
+    const std::string _distance_function_call_thrift =
+            
R"xxx({"1":{"lst":["rec",12,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}}},"4":{"i32":2},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"l2_distance"}}},"2":{"i32":0},"3":{"lst":["rec",2,{"1":{"lst":["rec",2,{"1":{"i32":1},"4":{"tf":1},"5":{"lst":["tf",1,1]}},{"1":{"i32":0},"2":{"rec":{"1":{"i32":8}}}}]},"3":{"i64":-1}},{"1":{"lst":["rec",2,{"1":{"i32":1},"4":{"tf":1},"5":{"lst":["tf",1,1]}},{"1":{"i32":0},"
 [...]
 };
 } // namespace doris::vectorized
\ No newline at end of file


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

Reply via email to