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