diff --git a/be/src/common/config.h b/be/src/common/config.h index 6843a88514..bc3c67d92c 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -822,10 +822,6 @@ CONF_Int32(doris_remote_scanner_thread_pool_thread_num, "16"); // number of s3 scanner thread pool queue size CONF_Int32(doris_remote_scanner_thread_pool_queue_size, "10240"); -// If set to true, the new scan node framework will be used. -// This config should be removed when the new scan node is ready. -CONF_Bool(enable_new_scan_node, "true"); - // limit the queue of pending batches which will be sent by a single nodechannel CONF_mInt64(nodechannel_pending_queue_max_bytes, "67108864"); diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp index 38de757f87..cea7068631 100644 --- a/be/src/exec/es_http_scan_node.cpp +++ b/be/src/exec/es_http_scan_node.cpp @@ -456,17 +456,10 @@ void EsHttpScanNode::scanner_worker(int start_idx, int length, std::promise scanner( - new EsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, - scanner_expr_ctxs, &counter, doc_value_mode)); - status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter); - } else { - std::unique_ptr scanner(new vectorized::VEsHttpScanner( - _runtime_state, runtime_profile(), _tuple_id, properties, scanner_expr_ctxs, - &counter, doc_value_mode)); - status = scanner_scan(std::move(scanner)); - } + std::unique_ptr scanner( + new EsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, + scanner_expr_ctxs, &counter, doc_value_mode)); + status = scanner_scan(std::move(scanner), scanner_expr_ctxs, &counter); if (!status.ok()) { LOG(WARNING) << "Scanner[" << start_idx << "] process failed. status=" << status.get_error_msg(); diff --git a/be/src/exec/es_http_scan_node.h b/be/src/exec/es_http_scan_node.h index 0fc9fac376..d07f68af24 100644 --- a/be/src/exec/es_http_scan_node.h +++ b/be/src/exec/es_http_scan_node.h @@ -30,7 +30,6 @@ #include "exec/es_http_scanner.h" #include "exec/scan_node.h" #include "gen_cpp/PaloInternalService_types.h" -#include "vec/exec/ves_http_scanner.h" namespace doris { @@ -85,7 +84,6 @@ protected: std::condition_variable _queue_reader_cond; std::condition_variable _queue_writer_cond; - bool _vectorized = false; private: // Create scanners to do scan job @@ -98,10 +96,6 @@ private: Status scanner_scan(std::unique_ptr scanner, const std::vector& conjunct_ctxs, EsScanCounter* counter); - virtual Status scanner_scan(std::unique_ptr scanner) { - return Status::NotSupported("vectorized scan in EsHttpScanNode is not supported!"); - }; - Status build_conjuncts_list(); std::vector _scanner_threads; diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index ce7bb4d317..9a67ced4cf 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -71,15 +71,11 @@ #include "vec/exec/vbroker_scan_node.h" #include "vec/exec/vdata_gen_scan_node.h" #include "vec/exec/vempty_set_node.h" -#include "vec/exec/ves_http_scan_node.h" #include "vec/exec/vexcept_node.h" #include "vec/exec/vexchange_node.h" #include "vec/exec/vintersect_node.h" -#include "vec/exec/vjdbc_scan_node.h" #include "vec/exec/vmysql_scan_node.h" #include "vec/exec/vnested_loop_join_node.h" -#include "vec/exec/vodbc_scan_node.h" -#include "vec/exec/volap_scan_node.h" #include "vec/exec/vrepeat_node.h" #include "vec/exec/vschema_scan_node.h" #include "vec/exec/vselect_node.h" @@ -231,8 +227,7 @@ Status ExecNode::prepare(RuntimeState* state) { // For vectorized olap scan node, the conjuncts is prepared in _vconjunct_ctx_ptr. // And _conjunct_ctxs is useless. // TODO: Should be removed when non-vec engine is removed. - if (typeid(*this) != typeid(doris::vectorized::VOlapScanNode) && - typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { + if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { RETURN_IF_ERROR(Expr::prepare(_conjunct_ctxs, state, _row_descriptor)); } RETURN_IF_ERROR(vectorized::VExpr::prepare(_projections, state, _row_descriptor)); @@ -250,8 +245,7 @@ Status ExecNode::open(RuntimeState* state) { RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->open(state)); } RETURN_IF_ERROR(vectorized::VExpr::open(_projections, state)); - if (typeid(*this) != typeid(doris::vectorized::VOlapScanNode) && - typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { + if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { return Expr::open(_conjunct_ctxs, state); } else { return Status::OK(); @@ -295,8 +289,7 @@ Status ExecNode::close(RuntimeState* state) { if (_vconjunct_ctx_ptr) { (*_vconjunct_ctx_ptr)->close(state); } - if (typeid(*this) != typeid(doris::vectorized::VOlapScanNode) && - typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { + if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) { Expr::close(_conjunct_ctxs, state); } vectorized::VExpr::close(_projections, state); @@ -449,11 +442,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN #endif case TPlanNodeType::ODBC_SCAN_NODE: if (state->enable_vectorized_exec()) { - if (config::enable_new_scan_node) { - *node = pool->add(new vectorized::NewOdbcScanNode(pool, tnode, descs)); - } else { - *node = pool->add(new vectorized::VOdbcScanNode(pool, tnode, descs)); - } + *node = pool->add(new vectorized::NewOdbcScanNode(pool, tnode, descs)); } else { *node = pool->add(new OdbcScanNode(pool, tnode, descs)); } @@ -462,11 +451,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::JDBC_SCAN_NODE: if (state->enable_vectorized_exec()) { #ifdef LIBJVM - if (config::enable_new_scan_node) { - *node = pool->add(new vectorized::NewJdbcScanNode(pool, tnode, descs)); - } else { - *node = pool->add(new vectorized::VJdbcScanNode(pool, tnode, descs)); - } + *node = pool->add(new vectorized::NewJdbcScanNode(pool, tnode, descs)); #else return Status::InternalError("Jdbc scan node is disabled since no libjvm is found!"); #endif @@ -477,11 +462,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::ES_HTTP_SCAN_NODE: if (state->enable_vectorized_exec()) { - if (config::enable_new_scan_node) { - *node = pool->add(new vectorized::NewEsScanNode(pool, tnode, descs)); - } else { - *node = pool->add(new vectorized::VEsHttpScanNode(pool, tnode, descs)); - } + *node = pool->add(new vectorized::NewEsScanNode(pool, tnode, descs)); } else { *node = pool->add(new EsHttpScanNode(pool, tnode, descs)); } @@ -497,11 +478,7 @@ Status ExecNode::create_node(RuntimeState* state, ObjectPool* pool, const TPlanN case TPlanNodeType::OLAP_SCAN_NODE: if (state->enable_vectorized_exec()) { - if (config::enable_new_scan_node) { - *node = pool->add(new vectorized::NewOlapScanNode(pool, tnode, descs)); - } else { - *node = pool->add(new vectorized::VOlapScanNode(pool, tnode, descs)); - } + *node = pool->add(new vectorized::NewOlapScanNode(pool, tnode, descs)); } else { *node = pool->add(new OlapScanNode(pool, tnode, descs)); } diff --git a/be/src/exec/olap_scan_node.h b/be/src/exec/olap_scan_node.h index 47cf6cd01c..ed018e2d93 100644 --- a/be/src/exec/olap_scan_node.h +++ b/be/src/exec/olap_scan_node.h @@ -30,7 +30,6 @@ #include "runtime/descriptors.h" #include "util/progress_updater.h" #include "util/spinlock.h" -#include "vec/exec/volap_scanner.h" namespace doris { class IRuntimeFilter; diff --git a/be/src/exprs/expr_context.h b/be/src/exprs/expr_context.h index 06a7453540..e61d60fd00 100644 --- a/be/src/exprs/expr_context.h +++ b/be/src/exprs/expr_context.h @@ -35,10 +35,6 @@ USING_DORIS_UDF; namespace doris { -namespace vectorized { -class VOlapScanNode; -} // namespace vectorized - class Expr; class MemPool; class RuntimeState; @@ -164,7 +160,6 @@ private: friend class OlapScanNode; friend class EsPredicate; friend class RowGroupReader; - friend class vectorized::VOlapScanNode; /// FunctionContexts for each registered expression. The FunctionContexts are created /// and owned by this ExprContext. diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 365b89aea3..61cd8e666d 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -88,11 +88,7 @@ set(VEC_FILES data_types/data_type_jsonb.cpp exec/vaggregation_node.cpp exec/varrow_scanner.cpp - exec/ves_http_scan_node.cpp - exec/ves_http_scanner.cpp - exec/volap_scan_node.cpp exec/vsort_node.cpp - exec/volap_scanner.cpp exec/vexchange_node.cpp exec/vset_operation_node.cpp exec/vunion_node.cpp @@ -100,7 +96,6 @@ set(VEC_FILES exec/vexcept_node.cpp exec/vselect_node.cpp exec/vnested_loop_join_node.cpp - exec/vodbc_scan_node.cpp exec/vmysql_scan_node.cpp exec/vschema_scan_node.cpp exec/vempty_set_node.cpp @@ -112,7 +107,6 @@ set(VEC_FILES exec/vbroker_scanner.cpp exec/vjson_scanner.cpp exec/vjdbc_connector.cpp - exec/vjdbc_scan_node.cpp exec/vparquet_scanner.cpp exec/vorc_scanner.cpp exec/join/vhash_join_node.cpp diff --git a/be/src/vec/exec/ves_http_scan_node.cpp b/be/src/vec/exec/ves_http_scan_node.cpp deleted file mode 100644 index d0704a2477..0000000000 --- a/be/src/vec/exec/ves_http_scan_node.cpp +++ /dev/null @@ -1,447 +0,0 @@ -// 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/exec/ves_http_scan_node.h" - -#include "exec/es/es_predicate.h" -#include "exec/es/es_query_builder.h" -#include "exec/es/es_scan_reader.h" -#include "exec/es/es_scroll_query.h" -#include "exprs/expr_context.h" -#include "gen_cpp/PlanNodes_types.h" -#include "runtime/runtime_state.h" -#include "runtime/tuple_row.h" -#include "util/runtime_profile.h" - -namespace doris::vectorized { - -VEsHttpScanNode::VEsHttpScanNode(ObjectPool* pool, const TPlanNode& tnode, - const DescriptorTbl& descs) - : ScanNode(pool, tnode, descs), - _tuple_id(tnode.es_scan_node.tuple_id), - _runtime_state(nullptr), - _tuple_desc(nullptr), - _num_running_scanners(0), - _scan_finished(false), - _eos(false), - _max_buffered_batches(1024), - _wait_scanner_timer(nullptr) {} - -Status VEsHttpScanNode::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(ScanNode::init(tnode, state)); - - // use TEsScanNode - _properties = tnode.es_scan_node.properties; - - if (tnode.es_scan_node.__isset.docvalue_context) { - _docvalue_context = tnode.es_scan_node.docvalue_context; - } - - if (tnode.es_scan_node.__isset.fields_context) { - _fields_context = tnode.es_scan_node.fields_context; - } - return Status::OK(); -} - -Status VEsHttpScanNode::prepare(RuntimeState* state) { - VLOG_QUERY << "VEsHttpScanNode prepare"; - RETURN_IF_ERROR(ScanNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - _scanner_profile.reset(new RuntimeProfile("EsHttpScanNode")); - runtime_profile()->add_child(_scanner_profile.get(), true, nullptr); - - _runtime_state = state; - _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (_tuple_desc == nullptr) { - return Status::InternalError("Failed to get tuple descriptor, _tuple_id=i{}", _tuple_id); - } - - // set up column name vector for ESScrollQueryBuilder - for (auto slot_desc : _tuple_desc->slots()) { - if (!slot_desc->is_materialized()) { - continue; - } - _column_names.push_back(slot_desc->col_name()); - } - - _wait_scanner_timer = ADD_TIMER(runtime_profile(), "WaitScannerTime"); - - return Status::OK(); -} - -// build predicate -Status VEsHttpScanNode::build_conjuncts_list() { - Status status = Status::OK(); - _conjunct_to_predicate.resize(_conjunct_ctxs.size()); - - for (int i = 0; i < _conjunct_ctxs.size(); ++i) { - EsPredicate* predicate = _pool->add(new EsPredicate(_conjunct_ctxs[i], _tuple_desc, _pool)); - predicate->set_field_context(_fields_context); - status = predicate->build_disjuncts_list(); - if (status.ok()) { - _conjunct_to_predicate[i] = _predicate_to_conjunct.size(); - _predicate_to_conjunct.push_back(i); - - _predicates.push_back(predicate); - } else { - _conjunct_to_predicate[i] = -1; - - VLOG_CRITICAL << status.get_error_msg(); - status = predicate->get_es_query_status(); - if (!status.ok()) { - LOG(WARNING) << status.get_error_msg(); - return status; - } - } - } - - return Status::OK(); -} - -Status VEsHttpScanNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VEsHttpScanNode::open"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - RETURN_IF_CANCELLED(state); - - // fe by enable_new_es_dsl to control whether to generate DSL for easy rollback. After the code is stable, can delete the be generation logic - if (_properties.find(ESScanReader::KEY_QUERY_DSL) != _properties.end()) { - RETURN_IF_ERROR(start_scanners()); - return Status::OK(); - } - - // if conjunct is constant, compute direct and set eos = true - for (int conj_idx = 0; conj_idx < _conjunct_ctxs.size(); ++conj_idx) { - if (_conjunct_ctxs[conj_idx]->root()->is_constant()) { - void* value = _conjunct_ctxs[conj_idx]->get_value(nullptr); - if (value == nullptr || *reinterpret_cast(value) == false) { - _eos = true; - } - } - } - - RETURN_IF_ERROR(build_conjuncts_list()); - - // remove those predicates which ES cannot support - std::vector list; - BooleanQueryBuilder::validate(_predicates, &list); - - DCHECK(list.size() == _predicate_to_conjunct.size()); - for (int i = list.size() - 1; i >= 0; i--) { - if (!list[i]) { - _predicate_to_conjunct.erase(_predicate_to_conjunct.begin() + i); - _predicates.erase(_predicates.begin() + i); - } - } - - // filter the conjuncts and ES will process them later - for (int i = _predicate_to_conjunct.size() - 1; i >= 0; i--) { - int conjunct_index = _predicate_to_conjunct[i]; - _conjunct_ctxs[conjunct_index]->close(_runtime_state); - _conjunct_ctxs.erase(_conjunct_ctxs.begin() + conjunct_index); - } - - auto checker = [&](int index) { - return _conjunct_to_predicate[index] != -1 && list[_conjunct_to_predicate[index]]; - }; - _peel_pushed_vconjunct(state, checker); - - RETURN_IF_ERROR(start_scanners()); - - return Status::OK(); -} - -// Prefer to the local host -static std::string get_host_port(const std::vector& es_hosts) { - std::string host_port; - std::string localhost = BackendOptions::get_localhost(); - - TNetworkAddress host = es_hosts[0]; - for (auto& es_host : es_hosts) { - if (es_host.hostname == localhost) { - host = es_host; - break; - } - } - - host_port = host.hostname; - host_port += ":"; - host_port += std::to_string(host.port); - return host_port; -} - -Status VEsHttpScanNode::start_scanners() { - { - std::unique_lock l(_batch_queue_lock); - _num_running_scanners = _scan_ranges.size(); - } - - _scanners_status.resize(_scan_ranges.size()); - for (int i = 0; i < _scan_ranges.size(); i++) { - _scanner_threads.emplace_back( - [this, i, length = _scan_ranges.size(), &p_status = _scanners_status[i], - parent_span = opentelemetry::trace::Tracer::GetCurrentSpan()] { - OpentelemetryScope scope {parent_span}; - this->scanner_worker(i, length, p_status); - }); - } - return Status::OK(); -} - -Status VEsHttpScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* eos) { - INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VEsHttpScanNode::get_next"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - if (state->is_cancelled()) { - std::unique_lock l(_block_queue_lock); - if (update_status(Status::Cancelled("Cancelled"))) { - _queue_writer_cond.notify_all(); - } - } - - if (_scan_finished.load() || _eos) { - *eos = true; - return Status::OK(); - } - - std::shared_ptr scanner_block; - { - std::unique_lock l(_block_queue_lock); - while (_process_status.ok() && !_runtime_state->is_cancelled() && - _num_running_scanners > 0 && _block_queue.empty()) { - SCOPED_TIMER(_wait_scanner_timer); - _queue_reader_cond.wait_for(l, std::chrono::seconds(1)); - } - if (!_process_status.ok()) { - // Some scanner process failed. - return _process_status; - } - if (_runtime_state->is_cancelled()) { - if (update_status(Status::Cancelled("Cancelled"))) { - _queue_writer_cond.notify_all(); - } - return _process_status; - } - if (!_block_queue.empty()) { - scanner_block = _block_queue.front(); - _block_queue.pop_front(); - } - } - - // All scanner has been finished, and all cached batch has been read - if (scanner_block == nullptr) { - _scan_finished.store(true); - *eos = true; - return Status::OK(); - } - - // notify one scanner - _queue_writer_cond.notify_one(); - - reached_limit(scanner_block.get(), eos); - *block = *scanner_block; - - // This is first time reach limit. - // Only valid when query 'select * from table1 limit 20' - if (*eos) { - _scan_finished.store(true); - _queue_writer_cond.notify_all(); - LOG(INFO) << "VEsHttpScanNode ReachedLimit."; - *eos = true; - } else { - *eos = false; - } - - return Status::OK(); -} - -Status VEsHttpScanNode::scanner_scan(std::unique_ptr scanner) { - RETURN_IF_ERROR(scanner->open()); - bool scanner_eof = false; - - const int batch_size = _runtime_state->batch_size(); - std::unique_ptr tuple_pool(new MemPool(mem_tracker())); - size_t slot_num = _tuple_desc->slots().size(); - - while (!scanner_eof) { - std::shared_ptr block(new vectorized::Block()); - std::vector columns(slot_num); - for (int i = 0; i < slot_num; i++) { - columns[i] = _tuple_desc->slots()[i]->get_empty_mutable_column(); - } - while (columns[0]->size() < batch_size && !scanner_eof) { - RETURN_IF_CANCELLED(_runtime_state); - - // If we have finished all works - if (_scan_finished.load()) { - return Status::OK(); - } - - // Get from scanner - RETURN_IF_ERROR( - scanner->get_next(columns, tuple_pool.get(), &scanner_eof, _docvalue_context)); - } - - if (columns[0]->size() > 0) { - auto n_columns = 0; - for (const auto slot_desc : _tuple_desc->slots()) { - block->insert(ColumnWithTypeAndName(std::move(columns[n_columns++]), - slot_desc->get_data_type_ptr(), - slot_desc->col_name())); - } - - RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block.get(), - _tuple_desc->slots().size())); - - std::unique_lock l(_block_queue_lock); - while (_process_status.ok() && !_scan_finished.load() && - !_runtime_state->is_cancelled() && - _block_queue.size() >= _max_buffered_batches) { - _queue_writer_cond.wait_for(l, std::chrono::seconds(1)); - } - // Process already set failed, so we just return OK - if (!_process_status.ok()) { - return Status::OK(); - } - // Scan already finished, just return - if (_scan_finished.load()) { - return Status::OK(); - } - // Runtime state is canceled, just return cancel - if (_runtime_state->is_cancelled()) { - return Status::Cancelled("Cancelled"); - } - _block_queue.push_back(block); - - // Notify reader to process - _queue_reader_cond.notify_one(); - } - } - - return Status::OK(); -} - -Status VEsHttpScanNode::close(RuntimeState* state) { - if (is_closed()) { - return Status::OK(); - } - SCOPED_TIMER(_runtime_profile->total_time_counter()); - _scan_finished.store(true); - _queue_writer_cond.notify_all(); - _queue_reader_cond.notify_all(); - for (int i = 0; i < _scanner_threads.size(); ++i) { - _scanner_threads[i].join(); - } - - //don't need to hold lock to update_status in close function - //collect scanners status - update_status(collect_scanners_status()); - - _batch_queue.clear(); - _block_queue.clear(); - - RETURN_IF_ERROR(ScanNode::close(state)); - return _process_status; -} - -Status VEsHttpScanNode::collect_scanners_status() { - // NOTE. if open() was called, but set_range() was NOT called for some reason. - // then close() was called. - // there would cause a core because _scanners_status's iterator was in [0, _scan_ranges) other than [0, _scanners_status) - // it is said that the fragment-call-frame is calling scan-node in this way.... - // in my options, it's better fixed in fragment-call-frame. e.g. call close() according the return value of open() - for (int i = 0; i < _scanners_status.size(); i++) { - std::future f = _scanners_status[i].get_future(); - RETURN_IF_ERROR(f.get()); - } - return Status::OK(); -} - -// This function is called after plan node has been prepared. -Status VEsHttpScanNode::set_scan_ranges(const std::vector& scan_ranges) { - _scan_ranges = scan_ranges; - return Status::OK(); -} - -void VEsHttpScanNode::debug_string(int ident_level, std::stringstream* out) const { - (*out) << "VEsHttpScanNode"; -} - -void VEsHttpScanNode::scanner_worker(int start_idx, int length, std::promise& p_status) { - START_AND_SCOPE_SPAN(_runtime_state->get_tracer(), span, "VEsHttpScanNode::scanner_worker"); - SCOPED_ATTACH_TASK(_runtime_state); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_shared()); - // Clone expr context - std::vector scanner_expr_ctxs; - DCHECK(start_idx < length); - auto status = Expr::clone_if_not_exists(_conjunct_ctxs, _runtime_state, &scanner_expr_ctxs); - if (!status.ok()) { - LOG(WARNING) << "Clone conjuncts failed."; - } - - EsScanCounter counter; - const TEsScanRange& es_scan_range = _scan_ranges[start_idx].scan_range.es_scan_range; - - // Collect the information from scan range to properties - std::map properties(_properties); - properties[ESScanReader::KEY_INDEX] = es_scan_range.index; - if (es_scan_range.__isset.type) { - properties[ESScanReader::KEY_TYPE] = es_scan_range.type; - } - properties[ESScanReader::KEY_SHARD] = std::to_string(es_scan_range.shard_id); - properties[ESScanReader::KEY_BATCH_SIZE] = std::to_string(_runtime_state->batch_size()); - properties[ESScanReader::KEY_HOST_PORT] = get_host_port(es_scan_range.es_hosts); - // push down limit to Elasticsearch - // if predicate in _conjunct_ctxs can not be processed by Elasticsearch, we can not push down limit operator to Elasticsearch - if (limit() != -1 && limit() <= _runtime_state->batch_size() && _conjunct_ctxs.empty()) { - properties[ESScanReader::KEY_TERMINATE_AFTER] = std::to_string(limit()); - } - - bool doc_value_mode = false; - properties[ESScanReader::KEY_QUERY] = ESScrollQueryBuilder::build( - properties, _column_names, _predicates, _docvalue_context, &doc_value_mode); - - // start scanner to scan - std::unique_ptr scanner( - new vectorized::VEsHttpScanner(_runtime_state, runtime_profile(), _tuple_id, properties, - scanner_expr_ctxs, &counter, doc_value_mode)); - status = scanner_scan(std::move(scanner)); - if (!status.ok()) { - LOG(WARNING) << "Scanner[" << start_idx - << "] process failed. status=" << status.get_error_msg(); - } - - // scanner is going to finish - { - std::lock_guard l(_batch_queue_lock); - if (!status.ok()) { - update_status(status); - } - // This scanner will finish - _num_running_scanners--; - } - _queue_reader_cond.notify_all(); - // If one scanner failed, others don't need scan any more - if (!status.ok()) { - _queue_writer_cond.notify_all(); - } - - p_status.set_value(status); -} - -} // namespace doris::vectorized diff --git a/be/src/vec/exec/ves_http_scan_node.h b/be/src/vec/exec/ves_http_scan_node.h deleted file mode 100644 index cd42a573e4..0000000000 --- a/be/src/vec/exec/ves_http_scan_node.h +++ /dev/null @@ -1,113 +0,0 @@ -// 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 - -#include "exec/es_http_scan_node.h" -#include "exec/scan_node.h" -#include "runtime/descriptors.h" -#include "vec/exec/ves_http_scanner.h" - -namespace doris { - -class RuntimeState; -class Status; - -namespace vectorized { - -class VEsHttpScanNode : public ScanNode { -public: - VEsHttpScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - ~VEsHttpScanNode() = default; - - virtual Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos) override; - - virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; - virtual Status prepare(RuntimeState* state) override; - virtual Status open(RuntimeState* state) override; - Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override { - return Status::NotSupported("Not Implemented VEsHttpScanNode Node::get_next scalar"); - } - virtual Status close(RuntimeState* state) override; - virtual Status set_scan_ranges(const std::vector& scan_ranges) override; - -private: - // Write debug string of this into out. - virtual void debug_string(int indentation_level, std::stringstream* out) const override; - - // Update process status to one failed status, - // NOTE: Must hold the mutex of this scan node - bool update_status(const Status& new_status) { - if (_process_status.ok()) { - _process_status = new_status; - return true; - } - return false; - } - // One scanner worker, This scanner will handle 'length' ranges start from start_idx - virtual void scanner_worker(int start_idx, int length, std::promise& p_status); - - TupleId _tuple_id; - RuntimeState* _runtime_state; - TupleDescriptor* _tuple_desc; - - int _num_running_scanners; - std::atomic _scan_finished; - bool _eos; - int _max_buffered_batches; - RuntimeProfile::Counter* _wait_scanner_timer; - - Status _process_status; - - std::map _docvalue_context; - - std::condition_variable _queue_reader_cond; - std::condition_variable _queue_writer_cond; - - // Create scanners to do scan job - Status start_scanners(); - - // Collect all scanners 's status - Status collect_scanners_status(); - - Status build_conjuncts_list(); - - std::vector _scanner_threads; - std::vector> _scanners_status; - std::map _properties; - std::map _fields_context; - std::vector _scan_ranges; - std::vector _column_names; - - std::mutex _batch_queue_lock; - std::deque> _batch_queue; - std::vector _predicates; - - std::vector _predicate_to_conjunct; - std::vector _conjunct_to_predicate; - - std::unique_ptr _scanner_profile; - - Status scanner_scan(std::unique_ptr scanner); - - std::deque> _block_queue; - std::mutex _block_queue_lock; -}; -} // namespace vectorized -} // namespace doris \ No newline at end of file diff --git a/be/src/vec/exec/ves_http_scanner.cpp b/be/src/vec/exec/ves_http_scanner.cpp deleted file mode 100644 index 11e3e87f94..0000000000 --- a/be/src/vec/exec/ves_http_scanner.cpp +++ /dev/null @@ -1,52 +0,0 @@ -// 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/exec/ves_http_scanner.h" - -namespace doris::vectorized { - -Status VEsHttpScanner::get_next(std::vector& columns, - MemPool* tuple_pool, bool* eof, - const std::map& docvalue_context) { - SCOPED_TIMER(_read_timer); - if (_line_eof && _batch_eof) { - *eof = true; - return Status::OK(); - } - - while (!_batch_eof) { - if (_line_eof || _es_scroll_parser == nullptr) { - RETURN_IF_ERROR(_es_reader->get_next(&_batch_eof, _es_scroll_parser)); - if (_batch_eof) { - *eof = true; - return Status::OK(); - } - } - - COUNTER_UPDATE(_rows_read_counter, 1); - SCOPED_TIMER(_materialize_timer); - RETURN_IF_ERROR(_es_scroll_parser->fill_columns(_tuple_desc, columns, tuple_pool, - &_line_eof, docvalue_context)); - if (!_line_eof) { - break; - } - } - - return Status::OK(); -} - -} // namespace doris::vectorized diff --git a/be/src/vec/exec/ves_http_scanner.h b/be/src/vec/exec/ves_http_scanner.h deleted file mode 100644 index a315e3e776..0000000000 --- a/be/src/vec/exec/ves_http_scanner.h +++ /dev/null @@ -1,38 +0,0 @@ - -// 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 - -namespace doris::vectorized { - -class VEsHttpScanner : public EsHttpScanner { -public: - VEsHttpScanner(RuntimeState* state, RuntimeProfile* profile, TupleId tuple_id, - const std::map& properties, - const std::vector& conjunct_ctxs, EsScanCounter* counter, - bool doc_value_mode) - : EsHttpScanner(state, profile, tuple_id, properties, conjunct_ctxs, counter, - doc_value_mode) {}; - - Status get_next(std::vector& columns, MemPool* tuple_pool, - bool* eof, const std::map& docvalue_context); -}; - -} // namespace doris::vectorized diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp index 5cff86bf46..283aa3e2a7 100644 --- a/be/src/vec/exec/vjdbc_connector.cpp +++ b/be/src/vec/exec/vjdbc_connector.cpp @@ -410,4 +410,4 @@ FUNC_IMPL_TO_CONVERT_DATA(double, double, D, Double) } // namespace vectorized } // namespace doris -#endif \ No newline at end of file +#endif diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h index e6f957746f..ba2f19c7b8 100644 --- a/be/src/vec/exec/vjdbc_connector.h +++ b/be/src/vec/exec/vjdbc_connector.h @@ -105,4 +105,4 @@ private: } // namespace vectorized } // namespace doris -#endif \ No newline at end of file +#endif diff --git a/be/src/vec/exec/vjdbc_scan_node.cpp b/be/src/vec/exec/vjdbc_scan_node.cpp deleted file mode 100644 index 7d15603992..0000000000 --- a/be/src/vec/exec/vjdbc_scan_node.cpp +++ /dev/null @@ -1,173 +0,0 @@ -// 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/exec/vjdbc_scan_node.h" -#ifdef LIBJVM -#include - -#include "common/status.h" - -namespace doris { -namespace vectorized { - -VJdbcScanNode::VJdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ScanNode(pool, tnode, descs), - _is_init(false), - _table_name(tnode.jdbc_scan_node.table_name), - _tuple_id(tnode.jdbc_scan_node.tuple_id), - _query_string(tnode.jdbc_scan_node.query_string), - _tuple_desc(nullptr) {} - -Status VJdbcScanNode::prepare(RuntimeState* state) { - VLOG_CRITICAL << "VJdbcScanNode::Prepare"; - if (_is_init) { - return Status::OK(); - } - - if (state == nullptr) { - return Status::InternalError("input pointer is NULL of VJdbcScanNode::prepare."); - } - - RETURN_IF_ERROR(ScanNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - // get tuple desc - _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - if (_tuple_desc == nullptr) { - return Status::InternalError("Failed to get tuple descriptor."); - } - - // get jdbc table info - const JdbcTableDescriptor* jdbc_table = - static_cast(_tuple_desc->table_desc()); - if (jdbc_table == nullptr) { - return Status::InternalError("jdbc table pointer is NULL of VJdbcScanNode::prepare."); - } - _jdbc_param.driver_class = jdbc_table->jdbc_driver_class(); - _jdbc_param.driver_path = jdbc_table->jdbc_driver_url(); - _jdbc_param.resource_name = jdbc_table->jdbc_resource_name(); - _jdbc_param.driver_checksum = jdbc_table->jdbc_driver_checksum(); - _jdbc_param.jdbc_url = jdbc_table->jdbc_url(); - _jdbc_param.user = jdbc_table->jdbc_user(); - _jdbc_param.passwd = jdbc_table->jdbc_passwd(); - _jdbc_param.tuple_desc = _tuple_desc; - _jdbc_param.query_string = std::move(_query_string); - - _jdbc_connector.reset(new (std::nothrow) JdbcConnector(_jdbc_param)); - if (_jdbc_connector == nullptr) { - return Status::InternalError("new a jdbc scanner failed."); - } - - _is_init = true; - return Status::OK(); -} - -Status VJdbcScanNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VJdbcScanNode::open"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - VLOG_CRITICAL << "VJdbcScanNode::open"; - - if (state == nullptr) { - return Status::InternalError("input pointer is NULL of VJdbcScanNode::open."); - } - - if (!_is_init) { - return Status::InternalError("used before initialize of VJdbcScanNode::open."); - } - - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(_jdbc_connector->open(state, true)); - RETURN_IF_ERROR(_jdbc_connector->query()); - return Status::OK(); -} - -Status VJdbcScanNode::get_next(RuntimeState* state, vectorized::Block* block, bool* eos) { - VLOG_CRITICAL << "VJdbcScanNode::get_next"; - INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VJdbcScanNode::get_next"); - if (nullptr == state || nullptr == block || nullptr == eos) { - return Status::InternalError("input is NULL pointer"); - } - - if (!_is_init) { - return Status::InternalError("used before initialize of VJdbcScanNode::get_next."); - } - - auto column_size = _tuple_desc->slots().size(); - std::vector columns(column_size); - bool mem_reuse = block->mem_reuse(); - // only empty block should be here - DCHECK(block->rows() == 0); - - bool jdbc_eos = false; - do { - RETURN_IF_CANCELLED(state); - - columns.resize(column_size); - for (auto i = 0; i < column_size; i++) { - if (mem_reuse) { - columns[i] = std::move(*block->get_by_position(i).column).mutate(); - } else { - columns[i] = _tuple_desc->slots()[i]->get_empty_mutable_column(); - } - } - - RETURN_IF_ERROR(_jdbc_connector->get_next(&jdbc_eos, columns, state->batch_size())); - - if (jdbc_eos) { - *eos = true; - break; - } - - // Before really use the Block, must clear other ptr of column in block - // So here need do std::move and clear in `columns` - if (!mem_reuse) { - int column_index = 0; - for (const auto slot_desc : _tuple_desc->slots()) { - block->insert(ColumnWithTypeAndName(std::move(columns[column_index++]), - slot_desc->get_data_type_ptr(), - slot_desc->col_name())); - } - } else { - columns.clear(); - } - VLOG_ROW << "VJdbcScanNode output rows: " << block->rows(); - } while (block->rows() == 0 && !(*eos)); - - RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns())); - reached_limit(block, eos); - return Status::OK(); -} - -Status VJdbcScanNode::close(RuntimeState* state) { - if (is_closed()) { - return Status::OK(); - } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VJdbcScanNode::close"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - return ExecNode::close(state); -} - -// No use -Status VJdbcScanNode::set_scan_ranges(const std::vector& scan_ranges) { - return Status::OK(); -} - -} // namespace vectorized -} // namespace doris - -#endif \ No newline at end of file diff --git a/be/src/vec/exec/vjdbc_scan_node.h b/be/src/vec/exec/vjdbc_scan_node.h deleted file mode 100644 index c9f13a544e..0000000000 --- a/be/src/vec/exec/vjdbc_scan_node.h +++ /dev/null @@ -1,68 +0,0 @@ -// 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 -#ifdef LIBJVM - -#include "exec/exec_node.h" -#include "exec/scan_node.h" -#include "vec/exec/vjdbc_connector.h" -namespace doris { - -namespace vectorized { -class VJdbcScanNode final : public ScanNode { -public: - VJdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - - ~VJdbcScanNode() override = default; - - Status prepare(RuntimeState* state) override; - - Status open(RuntimeState* state) override; - - Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override { - return Status::NotSupported("Not Implemented JdbcScanNode::get_next."); - } - - Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos) override; - - Status close(RuntimeState* state) override; - - // No use - Status set_scan_ranges(const std::vector& scan_ranges) override; - -private: - std::string get_query_stmt(const std::string& table, const std::vector& fields, - const std::vector& filters, int64_t limit); - - bool _is_init; - std::string _table_name; - - // Tuple id resolved in prepare() to set _tuple_desc; - TupleId _tuple_id; - //SQL - std::string _query_string; - // Descriptor of tuples read from JDBC table. - const TupleDescriptor* _tuple_desc; - - // Scanner of JDBC. - std::unique_ptr _jdbc_connector; - JdbcConnectorParam _jdbc_param; -}; -} // namespace vectorized -} // namespace doris -#endif \ No newline at end of file diff --git a/be/src/vec/exec/vodbc_scan_node.cpp b/be/src/vec/exec/vodbc_scan_node.cpp deleted file mode 100644 index 994eedc926..0000000000 --- a/be/src/vec/exec/vodbc_scan_node.cpp +++ /dev/null @@ -1,249 +0,0 @@ -// 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/exec/vodbc_scan_node.h" - -#include "exec/text_converter.h" -#include "exec/text_converter.hpp" - -namespace doris { -namespace vectorized { - -VOdbcScanNode::VOdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string scan_node_type) - : ScanNode(pool, tnode, descs), - _is_init(false), - _scan_node_type(std::move(scan_node_type)), - _table_name(tnode.odbc_scan_node.table_name), - _connect_string(std::move(tnode.odbc_scan_node.connect_string)), - _query_string(std::move(tnode.odbc_scan_node.query_string)), - _tuple_id(tnode.odbc_scan_node.tuple_id), - _tuple_desc(nullptr), - _slot_num(0) {} - -Status VOdbcScanNode::prepare(RuntimeState* state) { - VLOG_CRITICAL << _scan_node_type << "::Prepare"; - - if (_is_init) { - return Status::OK(); - } - - if (nullptr == state) { - return Status::InternalError("input pointer is null."); - } - - RETURN_IF_ERROR(ScanNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - // get tuple desc - _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - - if (nullptr == _tuple_desc) { - return Status::InternalError("Failed to get tuple descriptor."); - } - - _slot_num = _tuple_desc->slots().size(); - - _odbc_param.connect_string = std::move(_connect_string); - _odbc_param.query_string = std::move(_query_string); - _odbc_param.tuple_desc = _tuple_desc; - - _odbc_scanner.reset(new (std::nothrow) ODBCConnector(_odbc_param)); - - if (_odbc_scanner.get() == nullptr) { - return Status::InternalError("new a odbc scanner failed."); - } - - _tuple_pool.reset(new (std::nothrow) MemPool()); - - if (_tuple_pool.get() == nullptr) { - return Status::InternalError("new a mem pool failed."); - } - - _text_converter.reset(new (std::nothrow) TextConverter('\\')); - - if (_text_converter.get() == nullptr) { - return Status::InternalError("new a text convertor failed."); - } - - _is_init = true; - - return Status::OK(); -} - -Status VOdbcScanNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VOdbcScanNode::open"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - VLOG_CRITICAL << _scan_node_type << "::Open"; - - if (nullptr == state) { - return Status::InternalError("input pointer is null."); - } - - if (!_is_init) { - return Status::InternalError("used before initialize."); - } - - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(_odbc_scanner->open(state)); - RETURN_IF_ERROR(_odbc_scanner->query()); - // check materialize slot num - - return Status::OK(); -} - -Status VOdbcScanNode::get_next(RuntimeState* state, Block* block, bool* eos) { - INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VOdbcScanNode::get_next"); - VLOG_CRITICAL << get_scan_node_type() << "::GetNext"; - - if (nullptr == state || nullptr == block || nullptr == eos) { - return Status::InternalError("input is NULL pointer"); - } - - if (!is_init()) { - return Status::InternalError("used before initialize."); - } - RETURN_IF_CANCELLED(state); - - auto odbc_scanner = get_odbc_scanner(); - auto tuple_desc = get_tuple_desc(); - auto text_converter = get_text_converter(); - - auto column_size = tuple_desc->slots().size(); - std::vector columns(column_size); - - bool mem_reuse = block->mem_reuse(); - // only empty block should be here - DCHECK(block->rows() == 0); - - // Indicates whether there are more rows to process. Set in _odbc_scanner.next(). - bool odbc_eos = false; - - do { - RETURN_IF_CANCELLED(state); - - columns.resize(column_size); - for (auto i = 0; i < column_size; i++) { - if (mem_reuse) { - columns[i] = std::move(*block->get_by_position(i).column).mutate(); - } else { - columns[i] = tuple_desc->slots()[i]->get_empty_mutable_column(); - } - } - - while (true) { - // block is full, break - if (state->batch_size() <= columns[0]->size()) { - break; - } - - RETURN_IF_ERROR(odbc_scanner->get_next_row(&odbc_eos)); - - if (odbc_eos) { - *eos = true; - break; - } - - // Read one row from reader - for (int column_index = 0, materialized_column_index = 0; column_index < column_size; - ++column_index) { - auto slot_desc = tuple_desc->slots()[column_index]; - // because the fe planner filter the non_materialize column - if (!slot_desc->is_materialized()) { - continue; - } - const auto& column_data = odbc_scanner->get_column_data(materialized_column_index); - - char* value_data = static_cast(column_data.target_value_ptr); - int value_len = column_data.strlen_or_ind; - - if (value_len == SQL_NULL_DATA) { - if (slot_desc->is_nullable()) { - columns[column_index]->insert_default(); - } else { - return Status::InternalError( - "nonnull column contains nullptr. table={}, column={}", _table_name, - slot_desc->col_name()); - } - } else if (value_len > column_data.buffer_length) { - return Status::InternalError( - "column value length longer than buffer length. " - "table={}, column={}, buffer_length", - _table_name, slot_desc->col_name(), column_data.buffer_length); - } else { - if (!text_converter->write_column(slot_desc, &columns[column_index], value_data, - value_len, true, false)) { - std::stringstream ss; - ss << "Fail to convert odbc value:'" << value_data << "' to " - << slot_desc->type() << " on column:`" << slot_desc->col_name() + "`"; - return Status::InternalError(ss.str()); - } - } - materialized_column_index++; - } - } - - // Before really use the Block, muse clear other ptr of column in block - // So here need do std::move and clear in `columns` - if (!mem_reuse) { - int column_index = 0; - for (const auto slot_desc : tuple_desc->slots()) { - block->insert(ColumnWithTypeAndName(std::move(columns[column_index++]), - slot_desc->get_data_type_ptr(), - slot_desc->col_name())); - } - } else { - columns.clear(); - } - VLOG_ROW << "VOdbcScanNode output rows: " << block->rows(); - } while (block->rows() == 0 && !(*eos)); - - RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns())); - reached_limit(block, eos); - - return Status::OK(); -} - -Status VOdbcScanNode::close(RuntimeState* state) { - if (is_closed()) { - return Status::OK(); - } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VOdbcScanNode::close"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - - _tuple_pool.reset(); - - return ExecNode::close(state); -} - -void VOdbcScanNode::debug_string(int indentation_level, std::stringstream* out) const { - *out << string(indentation_level * 2, ' '); - *out << _scan_node_type << "(tupleid=" << _tuple_id << " table=" << _table_name; - *out << ")" << std::endl; - - for (int i = 0; i < _children.size(); ++i) { - _children[i]->debug_string(indentation_level + 1, out); - } -} - -Status VOdbcScanNode::set_scan_ranges(const std::vector& scan_ranges) { - return Status::OK(); -} - -} // namespace vectorized -} // namespace doris diff --git a/be/src/vec/exec/vodbc_scan_node.h b/be/src/vec/exec/vodbc_scan_node.h deleted file mode 100644 index 6261bbed9c..0000000000 --- a/be/src/vec/exec/vodbc_scan_node.h +++ /dev/null @@ -1,87 +0,0 @@ -// 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 "exec/odbc_connector.h" -#include "exec/scan_node.h" -#include "exec/text_converter.hpp" - -namespace doris { -namespace vectorized { - -class VOdbcScanNode : public ScanNode { -public: - VOdbcScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, - std::string scan_node_type = "VOdbcScanNode"); - ~VOdbcScanNode() = default; - - // initialize odbc_scanner, and create text_converter. - virtual Status prepare(RuntimeState* state) override; - - // Start ODBC scan using odbc_scanner. - virtual Status open(RuntimeState* state) override; - - Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override { - return Status::NotSupported("Not Implemented VOdbcScanNode Node::get_next scalar"); - } - Status get_next(RuntimeState* state, Block* block, bool* eos) override; - - // Close the odbc_scanner, and report errors. - virtual Status close(RuntimeState* state) override; - // No use - virtual Status set_scan_ranges(const std::vector& scan_ranges) override; - const TupleDescriptor* get_tuple_desc() { return _tuple_desc; } - TextConverter* get_text_converter() { return _text_converter.get(); } - ODBCConnector* get_odbc_scanner() { return _odbc_scanner.get(); } - const std::string& get_scan_node_type() { return _scan_node_type; } - - bool is_init() { return _is_init; } - -protected: - // Write debug string of this into out. - virtual void debug_string(int indentation_level, std::stringstream* out) const override; - -private: - bool _is_init; - - std::string _scan_node_type; - - // Name of Odbc table - std::string _table_name; - - std::string _connect_string; - - std::string _query_string; - // Tuple id resolved in prepare() to set _tuple_desc; - TupleId _tuple_id; - - // Descriptor of tuples read from ODBC table. - const TupleDescriptor* _tuple_desc; - // Tuple index in tuple row. - int _slot_num; - // Pool for allocating tuple data, including all varying-length slots. - std::unique_ptr _tuple_pool; - - // Scanner of ODBC. - std::unique_ptr _odbc_scanner; - ODBCConnectorParam _odbc_param; - // Helper class for converting text to other types; - std::unique_ptr _text_converter; -}; -} // namespace vectorized -} // namespace doris diff --git a/be/src/vec/exec/volap_scan_node.cpp b/be/src/vec/exec/volap_scan_node.cpp deleted file mode 100644 index 7e0abc1e77..0000000000 --- a/be/src/vec/exec/volap_scan_node.cpp +++ /dev/null @@ -1,1841 +0,0 @@ -// 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/exec/volap_scan_node.h" - -#include "common/resource_tls.h" -#include "exec/scan_node.h" -#include "gen_cpp/PlanNodes_types.h" -#include "olap/storage_engine.h" -#include "runtime/descriptors.h" -#include "runtime/exec_env.h" -#include "runtime/large_int_value.h" -#include "runtime/runtime_filter_mgr.h" -#include "util/priority_thread_pool.hpp" -#include "util/to_string.h" -#include "vec/core/block.h" -#include "vec/data_types/data_type_decimal.h" -#include "vec/exec/volap_scanner.h" -#include "vec/exprs/vbloom_predicate.h" -#include "vec/exprs/vcompound_pred.h" -#include "vec/exprs/vexpr.h" -#include "vec/exprs/vruntimefilter_wrapper.h" -#include "vec/functions/in.h" - -namespace doris::vectorized { -using doris::operator<<; - -#define RETURN_IF_PUSH_DOWN(stmt) \ - if (!push_down) { \ - stmt; \ - } else { \ - return; \ - } - -VOlapScanNode::VOlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : ScanNode(pool, tnode, descs), - _tuple_id(tnode.olap_scan_node.tuple_id), - _olap_scan_node(tnode.olap_scan_node), - _tuple_desc(nullptr), - _tuple_idx(0), - _eos(false), - _max_materialized_row_batches(config::doris_scanner_queue_size), - _start(false), - _scanner_done(false), - _transfer_done(false), - _status(Status::OK()), - _resource_info(nullptr), - _buffered_bytes(0), - _eval_conjuncts_fn(nullptr), - _runtime_filter_descs(tnode.runtime_filters), - _max_materialized_blocks(config::doris_scanner_queue_size) { - _materialized_blocks.reserve(_max_materialized_blocks); - _free_blocks.reserve(_max_materialized_blocks); - // if sort_info is set, push _limit to each olap scanner - if (_olap_scan_node.__isset.sort_info && _olap_scan_node.__isset.sort_limit) { - _limit_per_scanner = _olap_scan_node.sort_limit; - } -} - -Status VOlapScanNode::init(const TPlanNode& tnode, RuntimeState* state) { - RETURN_IF_ERROR(ExecNode::init(tnode, state)); - - const TQueryOptions& query_options = state->query_options(); - if (query_options.__isset.max_scan_key_num) { - _max_scan_key_num = query_options.max_scan_key_num; - } else { - _max_scan_key_num = config::doris_max_scan_key_num; - } - - if (query_options.__isset.max_pushdown_conditions_per_column) { - _max_pushdown_conditions_per_column = query_options.max_pushdown_conditions_per_column; - } else { - _max_pushdown_conditions_per_column = config::max_pushdown_conditions_per_column; - } - - _max_scanner_queue_size_bytes = query_options.mem_limit / 20; //TODO: session variable percent - - /// TODO: could one filter used in the different scan_node ? - int filter_size = _runtime_filter_descs.size(); - _runtime_filter_ctxs.resize(filter_size); - _runtime_filter_ready_flag.resize(filter_size); - for (int i = 0; i < filter_size; ++i) { - IRuntimeFilter* runtime_filter = nullptr; - const auto& filter_desc = _runtime_filter_descs[i]; - RETURN_IF_ERROR(state->runtime_filter_mgr()->regist_filter( - RuntimeFilterRole::CONSUMER, filter_desc, state->query_options(), id())); - RETURN_IF_ERROR(state->runtime_filter_mgr()->get_consume_filter(filter_desc.filter_id, - &runtime_filter)); - - _runtime_filter_ctxs[i].runtimefilter = runtime_filter; - _runtime_filter_ready_flag[i] = false; - } - - return Status::OK(); -} - -void VOlapScanNode::init_scan_profile() { - std::string scanner_profile_name = "VOlapScanner"; - if (_olap_scan_node.__isset.table_name) { - scanner_profile_name = fmt::format("VOlapScanner({0})", _olap_scan_node.table_name); - } - _scanner_profile.reset(new RuntimeProfile(scanner_profile_name)); - runtime_profile()->add_child(_scanner_profile.get(), true, nullptr); - - _segment_profile.reset(new RuntimeProfile("SegmentIterator")); - _scanner_profile->add_child(_segment_profile.get(), true, nullptr); -} - -void VOlapScanNode::_init_counter(RuntimeState* state) { - ADD_TIMER(_scanner_profile, "ShowHintsTime_V1"); - - _reader_init_timer = ADD_TIMER(_scanner_profile, "ReaderInitTime"); - _read_compressed_counter = ADD_COUNTER(_segment_profile, "CompressedBytesRead", TUnit::BYTES); - _read_uncompressed_counter = - ADD_COUNTER(_segment_profile, "UncompressedBytesRead", TUnit::BYTES); - _block_load_timer = ADD_TIMER(_segment_profile, "BlockLoadTime"); - _block_load_counter = ADD_COUNTER(_segment_profile, "BlocksLoad", TUnit::UNIT); - _block_fetch_timer = ADD_TIMER(_scanner_profile, "BlockFetchTime"); - _raw_rows_counter = ADD_COUNTER(_segment_profile, "RawRowsRead", TUnit::UNIT); - _block_convert_timer = ADD_TIMER(_scanner_profile, "BlockConvertTime"); - // Will be delete after non-vectorized code is removed - _block_seek_timer = ADD_TIMER(_segment_profile, "BlockSeekTime"); - _block_seek_counter = ADD_COUNTER(_segment_profile, "BlockSeekCount", TUnit::UNIT); - _block_init_timer = ADD_TIMER(_segment_profile, "BlockInitTime"); - _block_init_seek_timer = ADD_TIMER(_segment_profile, "BlockInitSeekTime"); - _block_init_seek_counter = ADD_COUNTER(_segment_profile, "BlockInitSeekCount", TUnit::UNIT); - - _rows_vec_cond_counter = ADD_COUNTER(_segment_profile, "RowsVectorPredFiltered", TUnit::UNIT); - _vec_cond_timer = ADD_TIMER(_segment_profile, "VectorPredEvalTime"); - _short_cond_timer = ADD_TIMER(_segment_profile, "ShortPredEvalTime"); - _first_read_timer = ADD_TIMER(_segment_profile, "FirstReadTime"); - _first_read_seek_timer = ADD_TIMER(_segment_profile, "FirstReadSeekTime"); - _first_read_seek_counter = ADD_COUNTER(_segment_profile, "FirstReadSeekCount", TUnit::UNIT); - - _lazy_read_timer = ADD_TIMER(_segment_profile, "LazyReadTime"); - _lazy_read_seek_timer = ADD_TIMER(_segment_profile, "LazyReadSeekTime"); - _lazy_read_seek_counter = ADD_COUNTER(_segment_profile, "LazyReadSeekCount", TUnit::UNIT); - - _output_col_timer = ADD_TIMER(_segment_profile, "OutputColumnTime"); - - _stats_filtered_counter = ADD_COUNTER(_segment_profile, "RowsStatsFiltered", TUnit::UNIT); - _bf_filtered_counter = ADD_COUNTER(_segment_profile, "RowsBloomFilterFiltered", TUnit::UNIT); - _del_filtered_counter = ADD_COUNTER(_scanner_profile, "RowsDelFiltered", TUnit::UNIT); - _conditions_filtered_counter = - ADD_COUNTER(_segment_profile, "RowsConditionsFiltered", TUnit::UNIT); - _key_range_filtered_counter = - ADD_COUNTER(_segment_profile, "RowsKeyRangeFiltered", TUnit::UNIT); - - _io_timer = ADD_TIMER(_segment_profile, "IOTimer"); - _decompressor_timer = ADD_TIMER(_segment_profile, "DecompressorTimer"); - _index_load_timer = ADD_TIMER(_segment_profile, "IndexLoadTime_V1"); - - _scan_timer = ADD_TIMER(_scanner_profile, "ScanTime"); - _scan_cpu_timer = ADD_TIMER(_scanner_profile, "ScanCpuTime"); - - _total_pages_num_counter = ADD_COUNTER(_segment_profile, "TotalPagesNum", TUnit::UNIT); - _cached_pages_num_counter = ADD_COUNTER(_segment_profile, "CachedPagesNum", TUnit::UNIT); - - _bitmap_index_filter_counter = - ADD_COUNTER(_segment_profile, "RowsBitmapIndexFiltered", TUnit::UNIT); - _bitmap_index_filter_timer = ADD_TIMER(_segment_profile, "BitmapIndexFilterTimer"); - - _num_scanners = ADD_COUNTER(_runtime_profile, "NumScanners", TUnit::UNIT); - - _filtered_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentFiltered", TUnit::UNIT); - _total_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentTotal", TUnit::UNIT); - - // time of transfer thread to wait for row batch from scan thread - _scanner_wait_batch_timer = ADD_TIMER(_runtime_profile, "ScannerBatchWaitTime"); - // time of scan thread to wait for worker thread of the thread pool - _scanner_wait_worker_timer = ADD_TIMER(_runtime_profile, "ScannerWorkerWaitTime"); - - // time of node to wait for batch/block queue - _olap_wait_batch_queue_timer = ADD_TIMER(_runtime_profile, "BatchQueueWaitTime"); - - // for the purpose of debugging or profiling - for (int i = 0; i < GENERAL_DEBUG_COUNT; ++i) { - char name[64]; - snprintf(name, sizeof(name), "GeneralDebugTimer%d", i); - _general_debug_timer[i] = ADD_TIMER(_segment_profile, name); - } -} - -Status VOlapScanNode::prepare(RuntimeState* state) { - init_scan_profile(); - RETURN_IF_ERROR(ScanNode::prepare(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - // create scanner profile - // create timer - _tablet_counter = ADD_COUNTER(runtime_profile(), "TabletCount ", TUnit::UNIT); - _scanner_sched_counter = ADD_COUNTER(runtime_profile(), "ScannerSchedCount ", TUnit::UNIT); - - _rows_pushed_cond_filtered_counter = - ADD_COUNTER(_scanner_profile, "RowsPushedCondFiltered", TUnit::UNIT); - _init_counter(state); - _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id); - - if (_tuple_desc == nullptr) { - // TODO: make sure we print all available diagnostic output to our error log - return Status::InternalError("Failed to get tuple descriptor."); - } - - const std::vector& slots = _tuple_desc->slots(); - - for (int i = 0; i < slots.size(); ++i) { - if (!slots[i]->is_materialized()) { - continue; - } - - if (slots[i]->type().is_collection_type()) { - _collection_slots.push_back(slots[i]); - } - - if (slots[i]->type().is_string_type()) { - _string_slots.push_back(slots[i]); - } - } - - _runtime_state = state; - for (size_t i = 0; i < _runtime_filter_descs.size(); ++i) { - _runtime_filter_ctxs[i].runtimefilter->init_profile(_runtime_profile.get()); - } - return Status::OK(); -} - -Status VOlapScanNode::open(RuntimeState* state) { - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VOlapScanNode::open"); - VLOG_CRITICAL << "VOlapScanNode::Open"; - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(ExecNode::open(state)); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - _resource_info = ResourceTls::get_resource_tls(); - - // acquire runtime filter - _runtime_filter_ctxs.resize(_runtime_filter_descs.size()); - - std::vector vexprs; - for (size_t i = 0; i < _runtime_filter_descs.size(); ++i) { - IRuntimeFilter* runtime_filter = _runtime_filter_ctxs[i].runtimefilter; - // If all targets are local, scan node will use hash node's runtime filter, and we don't - // need to allocate memory again - if (runtime_filter->has_remote_target()) { - if (auto bf = runtime_filter->get_bloomfilter()) { - RETURN_IF_ERROR(bf->init_with_fixed_length()); - } - } - bool ready = runtime_filter->is_ready(); - if (!ready) { - ready = runtime_filter->await(); - } - if (ready) { - RETURN_IF_ERROR(runtime_filter->get_push_expr_ctxs(&vexprs)); - _runtime_filter_ctxs[i].apply_mark = true; - _runtime_filter_ctxs[i].runtimefilter = runtime_filter; - } - } - RETURN_IF_ERROR(_append_rf_into_conjuncts(state, vexprs)); - - return Status::OK(); -} - -void VOlapScanNode::transfer_thread(RuntimeState* state) { - // scanner open pushdown to scanThread - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VOlapScanNode::transfer_thread"); - SCOPED_ATTACH_TASK(state); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker_shared()); - Status status = Status::OK(); - - if (_vconjunct_ctx_ptr) { - for (auto scanner : _volap_scanners) { - status = (*_vconjunct_ctx_ptr)->clone(state, scanner->vconjunct_ctx_ptr()); - if (!status.ok()) { - std::lock_guard guard(_status_mutex); - _status = status; - break; - } - } - } - - /********************************* - * 优先级调度基本策略: - * 1. 通过查询拆分的Range个数来确定初始nice值 - * Range个数越多,越倾向于认定为大查询,nice值越小 - * 2. 通过查询累计读取的数据量来调整nice值 - * 读取的数据越多,越倾向于认定为大查询,nice值越小 - * 3. 通过nice值来判断查询的优先级 - * nice值越大的,越优先获得的查询资源 - * 4. 定期提高队列内残留任务的优先级,避免大查询完全饿死 - *********************************/ - _total_assign_num = 0; - _nice = 18 + std::max(0, 2 - (int)_volap_scanners.size() / 5); - - auto doris_scanner_row_num = - _limit == -1 ? config::doris_scanner_row_num - : std::min(static_cast(config::doris_scanner_row_num), _limit); - _block_size = _limit == -1 ? state->batch_size() - : std::min(static_cast(state->batch_size()), _limit); - auto block_per_scanner = (doris_scanner_row_num + (_block_size - 1)) / _block_size; - auto pre_block_count = - std::min(_volap_scanners.size(), - static_cast(config::doris_scanner_thread_pool_thread_num)) * - block_per_scanner; - - for (int i = 0; i < pre_block_count; ++i) { - auto block = new Block(_tuple_desc->slots(), _block_size); - _free_blocks.emplace_back(block); - _buffered_bytes += block->allocated_bytes(); - } - - // read from scanner - while (LIKELY(status.ok())) { - int assigned_thread_num = _start_scanner_thread_task(state, block_per_scanner); - - std::vector blocks; - { - // 1 scanner idle task not empty, assign new scanner task - std::unique_lock l(_scan_blocks_lock); - - // scanner_row_num = 16k - // 16k * 10 * 12 * 8 = 15M(>2s) --> nice=10 - // 16k * 20 * 22 * 8 = 55M(>6s) --> nice=0 - while (_nice > 0 && _total_assign_num > (22 - _nice) * (20 - _nice) * 6) { - --_nice; - } - - // 2 wait when all scanner are running & no result in queue - while (UNLIKELY(_running_thread == assigned_thread_num && _scan_blocks.empty() && - !_scanner_done)) { - SCOPED_TIMER(_scanner_wait_batch_timer); - _scan_block_added_cv.wait(l); - } - - // 3 transfer result block when queue is not empty - if (LIKELY(!_scan_blocks.empty())) { - blocks.swap(_scan_blocks); - for (auto b : blocks) { - _scan_row_batches_bytes -= b->allocated_bytes(); - } - // delete scan_block if transfer thread should be stopped - // because scan_block wouldn't be useful anymore - if (UNLIKELY(_transfer_done)) { - std::for_each(blocks.begin(), blocks.end(), std::default_delete()); - blocks.clear(); - } - } else { - if (_scanner_done) { - // We should close eof scanners before transfer done, otherwise, - // they are closed until scannode is closed. Because plan is closed - // after the plan is finished, so query profile would leak stats from - // scanners closed by scannode::close. - while (!_volap_scanners.empty()) { - auto scanner = _volap_scanners.front(); - _volap_scanners.pop_front(); - DCHECK(scanner->need_to_close()); - scanner->close(state); - } - break; - } - } - } - - if (!blocks.empty()) { - _add_blocks(blocks); - } - } - - VLOG_CRITICAL << "TransferThread finish."; - { - std::unique_lock l(_blocks_lock); - _transfer_done = true; - } - _block_added_cv.notify_all(); - { - std::unique_lock l(_scan_blocks_lock); - _scan_thread_exit_cv.wait(l, [this] { return _running_thread == 0; }); - } - VLOG_CRITICAL << "Scanner threads have been exited. TransferThread exit."; -} - -void VOlapScanNode::scanner_thread(VOlapScanner* scanner) { - SCOPED_ATTACH_TASK(_runtime_state); - SCOPED_CONSUME_MEM_TRACKER(_runtime_state->scanner_mem_tracker().get()); - Thread::set_self_name("volap_scanner"); - int64_t wait_time = scanner->update_wait_worker_timer(); - // Do not use ScopedTimer. There is no guarantee that, the counter - // (_scan_cpu_timer, the class member) is not destroyed after `_running_thread==0`. - ThreadCpuStopWatch cpu_watch; - cpu_watch.start(); - Status status = Status::OK(); - bool eos = false; - RuntimeState* state = scanner->runtime_state(); - DCHECK(nullptr != state); - if (!scanner->is_open()) { - status = scanner->open(); - if (!status.ok()) { - std::lock_guard guard(_status_mutex); - _status = status; - eos = true; - } - scanner->set_opened(); - } - - std::vector vexprs; - auto& scanner_filter_apply_marks = *scanner->mutable_runtime_filter_marks(); - DCHECK(scanner_filter_apply_marks.size() == _runtime_filter_descs.size()); - for (size_t i = 0; i < scanner_filter_apply_marks.size(); i++) { - if (!scanner_filter_apply_marks[i] && !_runtime_filter_ctxs[i].apply_mark) { - /// When runtime filters are ready during running, we should use them to filter data - /// in VOlapScanner. - /// New arrival rf will be processed as below: - /// 1. convert these runtime filters to vectorized expressions - /// 2. if this is the first scanner thread to receive this rf, construct a new - /// VExprContext and update `_vconjunct_ctx_ptr` in scan node. Notice that we use - /// `_runtime_filter_ready_flag` to ensure `_vconjunct_ctx_ptr` will be updated only - /// once after any runtime_filters are ready. - /// 3. finally, just copy this new VExprContext to scanner and use it to filter data. - IRuntimeFilter* runtime_filter = _runtime_filter_ctxs[i].runtimefilter; - DCHECK(runtime_filter != nullptr); - bool ready = runtime_filter->is_ready(); - if (ready) { - runtime_filter->get_prepared_vexprs(&vexprs, _row_descriptor); - scanner_filter_apply_marks[i] = true; - if (!_runtime_filter_ready_flag[i] && !vexprs.empty()) { - std::lock_guard l(_rf_lock); - if (!_runtime_filter_ready_flag[i]) { - // Use all conjuncts and new arrival runtime filters to construct a new - // expression tree here. - _append_rf_into_conjuncts(state, vexprs); - _runtime_filter_ready_flag[i] = true; - } - } - } - } - } - - if (!vexprs.empty()) { - if (*scanner->vconjunct_ctx_ptr()) { - scanner->discard_conjuncts(); - } - { - std::shared_lock l(_rf_lock); - WARN_IF_ERROR((*_vconjunct_ctx_ptr)->clone(state, scanner->vconjunct_ctx_ptr()), - "Something wrong for runtime filters: "); - } - } - - std::vector blocks; - - // Because we use thread pool to scan data from storage. One scanner can't - // use this thread too long, this can starve other query's scanner. So, we - // need yield this thread when we do enough work. However, OlapStorage read - // data in pre-aggregate mode, then we can't use storage returned data to - // judge if we need to yield. So we record all raw data read in this round - // scan, if this exceed row number or bytes threshold, we yield this thread. - int64_t raw_rows_read = scanner->raw_rows_read(); - int64_t raw_rows_threshold = raw_rows_read + config::doris_scanner_row_num; - int64_t raw_bytes_read = 0; - int64_t raw_bytes_threshold = config::doris_scanner_row_bytes; - bool get_free_block = true; - int num_rows_in_block = 0; - - // Has to wait at least one full block, or it will cause a lot of schedule task in priority - // queue, it will affect query latency and query concurrency for example ssb 3.3. - while (!eos && raw_bytes_read < raw_bytes_threshold && - ((raw_rows_read < raw_rows_threshold && get_free_block) || - num_rows_in_block < _runtime_state->batch_size())) { - if (UNLIKELY(_transfer_done)) { - eos = true; - status = Status::Cancelled( - "Scan thread cancelled, cause query done, maybe reach limit."); - break; - } - - auto block = _alloc_block(get_free_block); - status = scanner->get_block(_runtime_state, block, &eos); - VLOG_ROW << "VOlapScanNode input rows: " << block->rows(); - if (!status.ok()) { - LOG(WARNING) << "Scan thread read VOlapScanner failed: " << status.to_string(); - // Add block ptr in blocks, prevent mem leak in read failed - blocks.push_back(block); - eos = true; - break; - } - - raw_bytes_read += block->bytes(); - num_rows_in_block += block->rows(); - // 4. if status not ok, change status_. - if (UNLIKELY(block->rows() == 0)) { - std::lock_guard l(_free_blocks_lock); - _free_blocks.emplace_back(block); - } else { - if (!blocks.empty() && - blocks.back()->rows() + block->rows() <= _runtime_state->batch_size()) { - MutableBlock(blocks.back()).merge(*block); - block->clear_column_data(); - std::lock_guard l(_free_blocks_lock); - _free_blocks.emplace_back(block); - } else { - blocks.push_back(block); - } - } - raw_rows_read = scanner->raw_rows_read(); - } - - { - // if we failed, check status. - if (UNLIKELY(!status.ok())) { - _transfer_done = true; - std::lock_guard guard(_status_mutex); - if (LIKELY(_status.ok())) { - _status = status; - } - } - - bool global_status_ok = false; - { - std::lock_guard guard(_status_mutex); - global_status_ok = _status.ok(); - } - if (UNLIKELY(!global_status_ok)) { - eos = true; - std::for_each(blocks.begin(), blocks.end(), std::default_delete()); - } else { - std::lock_guard l(_scan_blocks_lock); - _scan_blocks.insert(_scan_blocks.end(), blocks.begin(), blocks.end()); - for (auto b : blocks) { - _scan_row_batches_bytes += b->allocated_bytes(); - } - } - // If eos is true, we will process out of this lock block. - if (eos) { - scanner->mark_to_need_to_close(); - } - std::lock_guard l(_volap_scanners_lock); - _volap_scanners.push_front(scanner); - } - if (eos) { - std::lock_guard l(_scan_blocks_lock); - _progress.update(1); - if (_progress.done()) { - // this is the right out - _scanner_done = true; - } - } - _scan_cpu_timer->update(cpu_watch.elapsed_time()); - _scanner_wait_worker_timer->update(wait_time); - - std::unique_lock l(_scan_blocks_lock); - _running_thread--; - - // The transfer thead will wait for `_running_thread==0`, to make sure all scanner threads won't access class members. - // Do not access class members after this code. - _scan_block_added_cv.notify_one(); - _scan_thread_exit_cv.notify_one(); -} - -Status VOlapScanNode::_add_blocks(std::vector& block) { - { - std::unique_lock l(_blocks_lock); - - // check queue limit for both block queue size and bytes - while (UNLIKELY((_materialized_blocks.size() >= _max_materialized_blocks || - _materialized_row_batches_bytes >= _max_scanner_queue_size_bytes / 2) && - !_transfer_done)) { - _block_consumed_cv.wait(l); - } - - VLOG_CRITICAL << "Push block to materialized_blocks"; - _materialized_blocks.insert(_materialized_blocks.end(), block.cbegin(), block.cend()); - for (auto b : block) { - _materialized_row_batches_bytes += b->allocated_bytes(); - } - } - // remove one block, notify main thread - _block_added_cv.notify_one(); - return Status::OK(); -} - -Status VOlapScanNode::normalize_conjuncts() { - std::vector slots = _tuple_desc->slots(); - - for (int slot_idx = 0; slot_idx < slots.size(); ++slot_idx) { - switch (slots[slot_idx]->type().type) { -#define M(NAME) \ - case TYPE_##NAME: { \ - ColumnValueRange range(slots[slot_idx]->col_name(), \ - slots[slot_idx]->type().precision, \ - slots[slot_idx]->type().scale); \ - _id_to_slot_column_value_range[slots[slot_idx]->id()] = \ - std::pair {slots[slot_idx], range}; \ - break; \ - } -#define APPLY_FOR_PRIMITIVE_TYPE(M) \ - M(TINYINT) \ - M(SMALLINT) \ - M(INT) \ - M(BIGINT) \ - M(LARGEINT) \ - M(CHAR) \ - M(DATE) \ - M(DATETIME) \ - M(DATEV2) \ - M(DATETIMEV2) \ - M(VARCHAR) \ - M(STRING) \ - M(HLL) \ - M(DECIMAL32) \ - M(DECIMAL64) \ - M(DECIMAL128) \ - M(DECIMALV2) \ - M(BOOLEAN) - APPLY_FOR_PRIMITIVE_TYPE(M) -#undef M - default: { - VLOG_CRITICAL << "Unsupported Normalize Slot [ColName=" << slots[slot_idx]->col_name() - << "]"; - break; - } - } - } - if (_vconjunct_ctx_ptr) { - if ((*_vconjunct_ctx_ptr)->root()) { - VExpr* new_root = _normalize_predicate(_runtime_state, (*_vconjunct_ctx_ptr)->root()); - if (new_root) { - (*_vconjunct_ctx_ptr)->set_root(new_root); - } else { - (*(_vconjunct_ctx_ptr.get()))->mark_as_stale(); - _stale_vexpr_ctxs.push_back(std::move(_vconjunct_ctx_ptr)); - _vconjunct_ctx_ptr.reset(nullptr); - } - } - } - for (auto& it : _id_to_slot_column_value_range) { - std::visit( - [&](auto&& range) { - if (range.is_empty_value_range()) { - _eos = true; - } - }, - it.second.second); - _column_value_ranges[it.second.first->col_name()] = it.second.second; - } - - return Status::OK(); -} - -static std::string olap_filter_to_string(const doris::TCondition& condition) { - auto op_name = condition.condition_op; - if (condition.condition_op == "*=") { - op_name = "IN"; - } else if (condition.condition_op == "!*=") { - op_name = "NOT IN"; - } - return fmt::format("{{{} {} {}}}", condition.column_name, op_name, - to_string(condition.condition_values)); -} - -static std::string olap_filters_to_string(const std::vector& filters) { - std::string filters_string; - filters_string += "["; - for (auto it = filters.cbegin(); it != filters.cend(); it++) { - if (it != filters.cbegin()) { - filters_string += ","; - } - filters_string += olap_filter_to_string(*it); - } - filters_string += "]"; - return filters_string; -} - -Status VOlapScanNode::build_key_ranges_and_filters() { - const std::vector& column_names = _olap_scan_node.key_column_name; - const std::vector& column_types = _olap_scan_node.key_column_type; - DCHECK(column_types.size() == column_names.size()); - - // 1. construct scan key except last olap engine short key - _scan_keys.set_is_convertible(limit() == -1); - - // we use `exact_range` to identify a key range is an exact range or not when we convert - // it to `_scan_keys`. If `exact_range` is true, we can just discard it from `_olap_filter`. - bool exact_range = true; - for (int column_index = 0; column_index < column_names.size() && !_scan_keys.has_range_value(); - ++column_index) { - auto iter = _column_value_ranges.find(column_names[column_index]); - if (_column_value_ranges.end() == iter) { - break; - } - - RETURN_IF_ERROR(std::visit( - [&](auto&& range) { - // make a copy or range and pass to extend_scan_key, keep the range unchanged - // because extend_scan_key method may change the first parameter. - // but the original range may be converted to olap filters, if it's not a exact_range. - auto temp_range = range; - RETURN_IF_ERROR(_scan_keys.extend_scan_key(temp_range, _max_scan_key_num, - &exact_range)); - if (exact_range) { - _column_value_ranges.erase(iter->first); - } - return Status::OK(); - }, - iter->second)); - } - - for (auto& iter : _column_value_ranges) { - std::vector filters; - std::visit([&](auto&& range) { range.to_olap_filter(filters); }, iter.second); - - for (const auto& filter : filters) { - _olap_filter.push_back(std::move(filter)); - } - } - - _runtime_profile->add_info_string("PushdownPredicate", olap_filters_to_string(_olap_filter)); - - _runtime_profile->add_info_string("KeyRanges", _scan_keys.debug_string()); - - VLOG_CRITICAL << _scan_keys.debug_string(); - - return Status::OK(); -} - -Status VOlapScanNode::start_scan(RuntimeState* state) { - RETURN_IF_CANCELLED(state); - - VLOG_CRITICAL << "NormalizeConjuncts"; - RETURN_IF_ERROR(normalize_conjuncts()); - - if (_eos) { - return Status::OK(); - } - - VLOG_CRITICAL << "BuildKeyRangesAndFilters"; - RETURN_IF_ERROR(build_key_ranges_and_filters()); - - VLOG_CRITICAL << "StartScanThread"; - RETURN_IF_ERROR(start_scan_thread(state)); - - return Status::OK(); -} - -static bool ignore_cast(SlotDescriptor* slot, VExpr* expr) { - if (slot->type().is_date_type() && expr->type().is_date_type()) { - return true; - } - if (slot->type().is_string_type() && expr->type().is_string_type()) { - return true; - } - return false; -} - -template -Status VOlapScanNode::change_value_range(ColumnValueRange& temp_range, void* value, - const ChangeFixedValueRangeFunc& func, - const std::string& fn_name, int slot_ref_child) { - if constexpr (PrimitiveType == TYPE_DATE) { - DateTimeValue date_value; - reinterpret_cast(value)->convert_vec_dt_to_dt(&date_value); - if constexpr (IsFixed) { - if (!date_value.check_loss_accuracy_cast_to_date()) { - func(temp_range, - reinterpret_cast::CppType*>( - &date_value)); - } - } else { - if (date_value.check_loss_accuracy_cast_to_date()) { - if (fn_name == "lt" || fn_name == "ge") { - ++date_value; - } - } - func(temp_range, to_olap_filter_type(fn_name, slot_ref_child), - reinterpret_cast::CppType*>( - &date_value)); - } - } else if constexpr (PrimitiveType == TYPE_DATETIME) { - DateTimeValue date_value; - reinterpret_cast(value)->convert_vec_dt_to_dt(&date_value); - if constexpr (IsFixed) { - func(temp_range, - reinterpret_cast::CppType*>( - &date_value)); - } else { - func(temp_range, to_olap_filter_type(fn_name, slot_ref_child), - reinterpret_cast::CppType*>( - reinterpret_cast(&date_value))); - } - } else if constexpr ((PrimitiveType == TYPE_DECIMALV2) || (PrimitiveType == TYPE_CHAR) || - (PrimitiveType == TYPE_VARCHAR) || (PrimitiveType == TYPE_HLL) || - (PrimitiveType == TYPE_DATETIMEV2) || (PrimitiveType == TYPE_TINYINT) || - (PrimitiveType == TYPE_SMALLINT) || (PrimitiveType == TYPE_INT) || - (PrimitiveType == TYPE_BIGINT) || (PrimitiveType == TYPE_LARGEINT) || - (PrimitiveType == TYPE_DECIMAL32) || (PrimitiveType == TYPE_DECIMAL64) || - (PrimitiveType == TYPE_DECIMAL128) || (PrimitiveType == TYPE_STRING) || - (PrimitiveType == TYPE_BOOLEAN) || (PrimitiveType == TYPE_DATEV2)) { - if constexpr (IsFixed) { - func(temp_range, - reinterpret_cast::CppType*>(value)); - } else { - func(temp_range, to_olap_filter_type(fn_name, slot_ref_child), - reinterpret_cast::CppType*>(value)); - } - } else { - static_assert(always_false_v); - } - - return Status::OK(); -} - -bool VOlapScanNode::is_key_column(const std::string& key_name) { - // all column in dup_keys table or unique_keys with merge on write table olap scan node threat - // as key column - if (_olap_scan_node.keyType == TKeysType::DUP_KEYS || - (_olap_scan_node.keyType == TKeysType::UNIQUE_KEYS && - _olap_scan_node.__isset.enable_unique_key_merge_on_write && - _olap_scan_node.enable_unique_key_merge_on_write)) { - return true; - } - - auto res = std::find(_olap_scan_node.key_column_name.begin(), - _olap_scan_node.key_column_name.end(), key_name); - return res != _olap_scan_node.key_column_name.end(); -} - -Status VOlapScanNode::start_scan_thread(RuntimeState* state) { - if (_scan_ranges.empty()) { - _transfer_done = true; - return Status::OK(); - } - - // ranges constructed from scan keys - std::vector> cond_ranges; - RETURN_IF_ERROR(_scan_keys.get_key_range(&cond_ranges)); - // if we can't get ranges from conditions, we give it a total range - if (cond_ranges.empty()) { - cond_ranges.emplace_back(new OlapScanRange()); - } - bool need_split = true; - // If we have ranges more than 64, there is no need to call - // ShowHint to split ranges - if (limit() != -1 || cond_ranges.size() > 64) { - need_split = false; - } - int scanners_per_tablet = std::max(1, 64 / (int)_scan_ranges.size()); - - std::unordered_set disk_set; - for (auto& scan_range : _scan_ranges) { - auto tablet_id = scan_range->tablet_id; - std::string err; - TabletSharedPtr tablet = - StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, true, &err); - if (tablet == nullptr) { - std::stringstream ss; - ss << "failed to get tablet: " << tablet_id << ", reason: " << err; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - - std::vector>* ranges = &cond_ranges; - std::vector> split_ranges; - if (need_split && !tablet->all_beta()) { - auto st = get_hints(tablet, *scan_range, config::doris_scan_range_row_count, - _scan_keys.begin_include(), _scan_keys.end_include(), cond_ranges, - &split_ranges, _runtime_profile.get()); - if (st.ok()) { - ranges = &split_ranges; - } - } - int size_based_scanners_per_tablet = 1; - - if (config::doris_scan_range_max_mb > 0) { - size_based_scanners_per_tablet = std::max( - 1, (int)(tablet->tablet_footprint() / (config::doris_scan_range_max_mb << 20))); - } - - int ranges_per_scanner = - std::max(1, (int)ranges->size() / - std::min(scanners_per_tablet, size_based_scanners_per_tablet)); - int num_ranges = ranges->size(); - for (int i = 0; i < num_ranges;) { - std::vector scanner_ranges; - scanner_ranges.push_back((*ranges)[i].get()); - ++i; - for (int j = 1; i < num_ranges && j < ranges_per_scanner && - (*ranges)[i]->end_include == (*ranges)[i - 1]->end_include; - ++j, ++i) { - scanner_ranges.push_back((*ranges)[i].get()); - } - VOlapScanner* scanner = new VOlapScanner(state, this, _olap_scan_node.is_preaggregation, - _need_agg_finalize, *scan_range); - // add scanner to pool before doing prepare. - // so that scanner can be automatically deconstructed if prepare failed. - _scanner_pool.add(scanner); - RETURN_IF_ERROR(scanner->prepare(*scan_range, scanner_ranges, _olap_filter, - _bloom_filters_push_down, _push_down_functions)); - - _volap_scanners.push_back(scanner); - disk_set.insert(scanner->scan_disk()); - } - } - COUNTER_SET(_num_disks_accessed_counter, static_cast(disk_set.size())); - COUNTER_SET(_num_scanners, static_cast(_volap_scanners.size())); - - // init progress - std::stringstream ss; - ss << "ScanThread complete (node=" << id() << "):"; - _progress = ProgressUpdater(ss.str(), _volap_scanners.size(), 1); - - _transfer_thread.reset(new std::thread( - [this, state, parent_span = opentelemetry::trace::Tracer::GetCurrentSpan()] { - opentelemetry::trace::Scope scope {parent_span}; - transfer_thread(state); - })); - - return Status::OK(); -} - -Status VOlapScanNode::close(RuntimeState* state) { - if (is_closed()) { - return Status::OK(); - } - START_AND_SCOPE_SPAN(state->get_tracer(), span, "VOlapScanNode::close"); - // change done status - { - std::unique_lock l(_blocks_lock); - _transfer_done = true; - } - // notify all scanner thread - _block_consumed_cv.notify_all(); - _block_added_cv.notify_all(); - _scan_block_added_cv.notify_all(); - - // join transfer thread - if (_transfer_thread) { - _transfer_thread->join(); - } - - // clear some block in queue - // TODO: The presence of transfer_thread here may cause Block's memory alloc and be released not in a thread, - // which may lead to potential performance problems. we should rethink whether to delete the transfer thread - std::for_each(_materialized_blocks.begin(), _materialized_blocks.end(), - std::default_delete()); - _materialized_row_batches_bytes = 0; - std::for_each(_scan_blocks.begin(), _scan_blocks.end(), std::default_delete()); - _scan_row_batches_bytes = 0; - std::for_each(_free_blocks.begin(), _free_blocks.end(), std::default_delete()); - - // OlapScanNode terminate by exception - // so that initiative close the Scanner - for (auto scanner : _volap_scanners) { - scanner->close(state); - } - - for (auto& filter_ctx : _runtime_filter_ctxs) { - filter_ctx.runtimefilter->consumer_close(); - } - - for (auto& ctx : _stale_vexpr_ctxs) { - (*ctx)->close(state); - } - - VLOG_CRITICAL << "VOlapScanNode::close()"; - return ScanNode::close(state); -} - -Status VOlapScanNode::get_next(RuntimeState* state, Block* block, bool* eos) { - INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VOlapScanNode::get_next"); - SCOPED_TIMER(_runtime_profile->total_time_counter()); - SCOPED_CONSUME_MEM_TRACKER(mem_tracker()); - - // check if Canceled. - if (state->is_cancelled()) { - std::unique_lock l(_blocks_lock); - _transfer_done = true; - std::lock_guard guard(_status_mutex); - if (LIKELY(_status.ok())) { - _status = Status::Cancelled("Cancelled"); - } - return _status; - } - - // check if started. - if (!_start) { - Status status = start_scan(state); - - if (!status.ok()) { - LOG(ERROR) << "StartScan Failed cause " << status.get_error_msg(); - *eos = true; - return status; - } - - _start = true; - } - - // some conjuncts will be disposed in start_scan function, so - // we should check _eos after call start_scan - if (_eos) { - *eos = true; - return Status::OK(); - } - - // wait for block from queue - Block* materialized_block = nullptr; - { - std::unique_lock l(_blocks_lock); - SCOPED_TIMER(_olap_wait_batch_queue_timer); - while (_materialized_blocks.empty() && !_transfer_done) { - if (state->is_cancelled()) { - _transfer_done = true; - } - - // use wait_for, not wait, in case to capture the state->is_cancelled() - _block_added_cv.wait_for(l, std::chrono::seconds(1)); - } - - if (!_materialized_blocks.empty()) { - materialized_block = _materialized_blocks.back(); - DCHECK(materialized_block != nullptr); - _materialized_blocks.pop_back(); - _materialized_row_batches_bytes -= materialized_block->allocated_bytes(); - } - } - - // return block - if (nullptr != materialized_block) { - // notify scanner - _block_consumed_cv.notify_one(); - // get scanner's block memory - block->swap(*materialized_block); - VLOG_ROW << "VOlapScanNode output rows: " << block->rows(); - reached_limit(block, eos); - - // reach scan node limit - if (*eos) { - { - std::unique_lock l(_blocks_lock); - _transfer_done = true; - } - - _block_consumed_cv.notify_all(); - *eos = true; - } else { - *eos = false; - } - - { - // ReThink whether the SpinLock Better - std::lock_guard l(_free_blocks_lock); - _free_blocks.emplace_back(materialized_block); - } - return Status::OK(); - } - - // all scanner done, change *eos to true - *eos = true; - std::lock_guard guard(_status_mutex); - return _status; -} - -Block* VOlapScanNode::_alloc_block(bool& get_free_block) { - { - std::lock_guard l(_free_blocks_lock); - if (!_free_blocks.empty()) { - auto block = _free_blocks.back(); - _free_blocks.pop_back(); - return block; - } - } - - get_free_block = false; - - auto block = new Block(_tuple_desc->slots(), _block_size); - _buffered_bytes += block->allocated_bytes(); - return block; -} - -int VOlapScanNode::_start_scanner_thread_task(RuntimeState* state, int block_per_scanner) { - std::list olap_scanners; - int assigned_thread_num = _running_thread; - size_t max_thread = config::doris_scanner_queue_size; - if (config::doris_scanner_row_num > state->batch_size()) { - max_thread /= config::doris_scanner_row_num / state->batch_size(); - if (max_thread <= 0) max_thread = 1; - } - // copy to local - { - // How many thread can apply to this query - size_t thread_slot_num = 0; - { - if (_scan_row_batches_bytes < _max_scanner_queue_size_bytes / 2) { - std::lock_guard l(_free_blocks_lock); - thread_slot_num = _free_blocks.size() / block_per_scanner; - thread_slot_num += (_free_blocks.size() % block_per_scanner != 0); - thread_slot_num = std::min(thread_slot_num, max_thread - assigned_thread_num); - if (thread_slot_num <= 0) { - thread_slot_num = 1; - } - } else { - std::lock_guard l(_scan_blocks_lock); - if (_scan_blocks.empty()) { - // Just for notify if _scan_blocks is empty and no running thread - if (assigned_thread_num == 0) { - thread_slot_num = 1; - // NOTE: if olap_scanners_ is empty, scanner_done_ should be true - } - } - } - } - - { - std::lock_guard l(_volap_scanners_lock); - thread_slot_num = std::min(thread_slot_num, _volap_scanners.size()); - for (int i = 0; i < thread_slot_num && !_volap_scanners.empty();) { - auto scanner = _volap_scanners.front(); - _volap_scanners.pop_front(); - - if (scanner->need_to_close()) { - scanner->close(state); - } else { - olap_scanners.push_back(scanner); - _running_thread++; - assigned_thread_num++; - i++; - } - } - } - } - - // post volap scanners to thread-pool - ThreadPoolToken* thread_token = state->get_query_fragments_ctx()->get_token(); - auto iter = olap_scanners.begin(); - if (thread_token != nullptr) { - while (iter != olap_scanners.end()) { - auto s = thread_token->submit_func( - [this, scanner = *iter] { this->scanner_thread(scanner); }); - if (s.ok()) { - (*iter)->start_wait_worker_timer(); - COUNTER_UPDATE(_scanner_sched_counter, 1); - olap_scanners.erase(iter++); - } else { - LOG(FATAL) << "Failed to assign scanner task to thread pool! " << s.get_error_msg(); - } - ++_total_assign_num; - } - } else { - PriorityThreadPool* thread_pool = state->exec_env()->scan_thread_pool(); - PriorityThreadPool* remote_thread_pool = state->exec_env()->remote_scan_thread_pool(); - while (iter != olap_scanners.end()) { - PriorityThreadPool::Task task; - task.work_function = [this, scanner = *iter] { this->scanner_thread(scanner); }; - task.priority = _nice; - task.queue_id = state->exec_env()->store_path_to_index((*iter)->scan_disk()); - (*iter)->start_wait_worker_timer(); - - TabletStorageType type = (*iter)->get_storage_type(); - bool ret = false; - COUNTER_UPDATE(_scanner_sched_counter, 1); - if (type == TabletStorageType::STORAGE_TYPE_LOCAL) { - ret = thread_pool->offer(task); - } else { - ret = remote_thread_pool->offer(task); - } - - if (ret) { - olap_scanners.erase(iter++); - } else { - LOG(FATAL) << "Failed to assign scanner task to thread pool!"; - } - ++_total_assign_num; - } - } - - return assigned_thread_num; -} - -// PlanFragmentExecutor will call this method to set scan range -// Doris scan range is defined in thrift file like this -// struct TPaloScanRange { -// 1: required list hosts -// 2: required string schema_hash -// 3: required string version -// 5: required Types.TTabletId tablet_id -// 6: required string db_name -// 7: optional list partition_column_ranges -// 8: optional string index_name -// 9: optional string table_name -//} -// every doris_scan_range is related with one tablet so that one olap scan node contains multiple tablet -Status VOlapScanNode::set_scan_ranges(const std::vector& scan_ranges) { - for (auto& scan_range : scan_ranges) { - DCHECK(scan_range.scan_range.__isset.palo_scan_range); - _scan_ranges.emplace_back(new TPaloScanRange(scan_range.scan_range.palo_scan_range)); - COUNTER_UPDATE(_tablet_counter, 1); - } - - return Status::OK(); -} - -Status VOlapScanNode::get_hints(TabletSharedPtr table, const TPaloScanRange& scan_range, - int block_row_count, bool is_begin_include, bool is_end_include, - const std::vector>& scan_key_range, - std::vector>* sub_scan_range, - RuntimeProfile* profile) { - RuntimeProfile::Counter* show_hints_timer = profile->get_counter("ShowHintsTime_V1"); - std::vector> ranges; - bool have_valid_range = false; - for (auto& key_range : scan_key_range) { - if (key_range->begin_scan_range.size() == 1 && - key_range->begin_scan_range.get_value(0) == NEGATIVE_INFINITY) { - continue; - } - SCOPED_TIMER(show_hints_timer); - - Status res = Status::OK(); - std::vector range; - res = table->split_range(key_range->begin_scan_range, key_range->end_scan_range, - block_row_count, &range); - if (!res.ok()) { - return Status::InternalError("fail to show hints"); - } - ranges.emplace_back(std::move(range)); - have_valid_range = true; - } - - if (!have_valid_range) { - std::vector range; - auto res = table->split_range({}, {}, block_row_count, &range); - if (!res.ok()) { - return Status::InternalError("fail to show hints"); - } - ranges.emplace_back(std::move(range)); - } - - for (int i = 0; i < ranges.size(); ++i) { - for (int j = 0; j < ranges[i].size(); j += 2) { - std::unique_ptr range(new OlapScanRange); - range->begin_scan_range.reset(); - range->begin_scan_range = ranges[i][j]; - range->end_scan_range.reset(); - range->end_scan_range = ranges[i][j + 1]; - - if (0 == j) { - range->begin_include = is_begin_include; - } else { - range->begin_include = true; - } - - if (j + 2 == ranges[i].size()) { - range->end_include = is_end_include; - } else { - range->end_include = false; - } - - sub_scan_range->emplace_back(std::move(range)); - } - } - - return Status::OK(); -} - -template -bool VOlapScanNode::_should_push_down_in_predicate(VInPredicate* pred, VExprContext* expr_ctx) { - if (pred->is_not_in() != IsNotIn) { - return false; - } - InState* state = reinterpret_cast( - expr_ctx->fn_context(pred->fn_context_index()) - ->get_function_state(FunctionContext::FRAGMENT_LOCAL)); - HybridSetBase* set = state->hybrid_set.get(); - - // if there are too many elements in InPredicate, exceed the limit, - // we will not push any condition of this column to storage engine. - // because too many conditions pushed down to storage engine may even - // slow down the query process. - // ATTN: This is just an experience value. You may need to try - // different thresholds to improve performance. - if (set->size() > _max_pushdown_conditions_per_column) { - VLOG_NOTICE << "Predicate value num " << set->size() << " exceed limit " - << _max_pushdown_conditions_per_column; - return false; - } - return true; -} - -bool VOlapScanNode::_should_push_down_function_filter(VectorizedFnCall* fn_call, - VExprContext* expr_ctx, - StringVal* constant_str, - doris_udf::FunctionContext** fn_ctx) { - // Now only `like` function filters is supported to push down - if (fn_call->fn().name.function_name != "like") { - return false; - } - - const auto& children = fn_call->children(); - doris_udf::FunctionContext* func_cxt = expr_ctx->fn_context(fn_call->fn_context_index()); - DCHECK(func_cxt != nullptr); - DCHECK(children.size() == 2); - for (size_t i = 0; i < children.size(); i++) { - if (VExpr::expr_without_cast(children[i])->node_type() != TExprNodeType::SLOT_REF) { - // not a slot ref(column) - continue; - } - if (!children[1 - i]->is_constant()) { - // only handle constant value - return false; - } else { - DCHECK(children[1 - i]->type().is_string_type()); - if (const ColumnConst* const_column = check_and_get_column( - children[1 - i]->get_const_col(expr_ctx)->column_ptr)) { - *constant_str = const_column->get_data_at(0).to_string_val(); - } else { - return false; - } - } - } - *fn_ctx = func_cxt; - return true; -} - -bool VOlapScanNode::_should_push_down_binary_predicate( - VectorizedFnCall* fn_call, VExprContext* expr_ctx, StringRef* constant_val, - int* slot_ref_child, const std::function& fn_checker) { - if (!fn_checker(fn_call->fn().name.function_name)) { - return false; - } - - const auto& children = fn_call->children(); - DCHECK(children.size() == 2); - for (size_t i = 0; i < children.size(); i++) { - if (VExpr::expr_without_cast(children[i])->node_type() != TExprNodeType::SLOT_REF) { - // not a slot ref(column) - continue; - } - if (!children[1 - i]->is_constant()) { - // only handle constant value - return false; - } else { - if (const ColumnConst* const_column = check_and_get_column( - children[1 - i]->get_const_col(expr_ctx)->column_ptr)) { - *slot_ref_child = i; - *constant_val = const_column->get_data_at(0); - } else { - return false; - } - } - } - return true; -} - -bool VOlapScanNode::_is_predicate_acting_on_slot( - VExpr* expr, - const std::function&, const VSlotRef**, VExpr**)>& checker, - SlotDescriptor** slot_desc, ColumnValueRangeType** range) { - const VSlotRef* slot_ref = nullptr; - VExpr* child_contains_slot = nullptr; - if (!checker(expr->children(), &slot_ref, &child_contains_slot)) { - // not a slot ref(column) - return false; - } - - auto entry = _id_to_slot_column_value_range.find(slot_ref->slot_id()); - if (_id_to_slot_column_value_range.end() == entry) { - return false; - } - *slot_desc = entry->second.first; - DCHECK(child_contains_slot != nullptr); - if (child_contains_slot->type().type != (*slot_desc)->type().type) { - if (!ignore_cast(*slot_desc, child_contains_slot)) { - // the type of predicate not match the slot's type - return false; - } - } - *range = &(entry->second.second); - return true; -} - -template -Status VOlapScanNode::_normalize_in_and_eq_predicate(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, bool* push_down) { - auto temp_range = ColumnValueRange::create_empty_column_value_range(slot->type().precision, - slot->type().scale); - bool effect = false; - // 1. Normalize in conjuncts like 'where col in (v1, v2, v3)' - if (TExprNodeType::IN_PRED == expr->node_type()) { - VInPredicate* pred = static_cast(expr); - if (!_should_push_down_in_predicate(pred, expr_ctx)) { - return Status::OK(); - } - - // begin to push InPredicate value into ColumnValueRange - InState* state = reinterpret_cast( - expr_ctx->fn_context(pred->fn_context_index()) - ->get_function_state(FunctionContext::FRAGMENT_LOCAL)); - HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); - auto fn_name = std::string(""); - while (iter->has_next()) { - // column in (nullptr) is always false so continue to - // dispose next item - if (nullptr == iter->get_value()) { - iter->next(); - continue; - } - auto value = const_cast(iter->get_value()); - RETURN_IF_ERROR(change_value_range( - temp_range, value, ColumnValueRange::add_fixed_value_range, fn_name)); - iter->next(); - } - - range.intersection(temp_range); - effect = true; - } else if (TExprNodeType::BINARY_PRED == expr->node_type()) { - DCHECK(expr->children().size() == 2); - auto eq_checker = [](const std::string& fn_name) { return fn_name == "eq"; }; - - StringRef value; - int slot_ref_child = -1; - if (_should_push_down_binary_predicate(reinterpret_cast(expr), expr_ctx, - &value, &slot_ref_child, eq_checker)) { - DCHECK(slot_ref_child >= 0); - // where A = nullptr should return empty result set - auto fn_name = std::string(""); - if (value.data != nullptr) { - if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING || - T == TYPE_HLL) { - auto val = StringValue(value.data, value.size); - RETURN_IF_ERROR(change_value_range( - temp_range, reinterpret_cast(&val), - ColumnValueRange::add_fixed_value_range, fn_name)); - } else { - RETURN_IF_ERROR(change_value_range( - temp_range, reinterpret_cast(const_cast(value.data)), - ColumnValueRange::add_fixed_value_range, fn_name)); - } - range.intersection(temp_range); - effect = true; - } - } - } - - // exceed limit, no conditions will be pushed down to storage engine. - if (range.get_fixed_value_size() > _max_pushdown_conditions_per_column) { - range.set_whole_value_range(); - } else { - *push_down = effect; - } - return Status::OK(); -} - -template -Status VOlapScanNode::_normalize_not_in_and_not_eq_predicate(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, - bool* push_down) { - bool is_fixed_range = range.is_fixed_value_range(); - auto not_in_range = ColumnValueRange::create_empty_column_value_range(range.column_name()); - bool effect = false; - // 1. Normalize in conjuncts like 'where col in (v1, v2, v3)' - if (TExprNodeType::IN_PRED == expr->node_type()) { - VInPredicate* pred = static_cast(expr); - if (!_should_push_down_in_predicate(pred, expr_ctx)) { - return Status::OK(); - } - - // begin to push InPredicate value into ColumnValueRange - InState* state = reinterpret_cast( - expr_ctx->fn_context(pred->fn_context_index()) - ->get_function_state(FunctionContext::FRAGMENT_LOCAL)); - HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); - auto fn_name = std::string(""); - while (iter->has_next()) { - // column not in (nullptr) is always true - if (nullptr == iter->get_value()) { - continue; - } - auto value = const_cast(iter->get_value()); - if (is_fixed_range) { - RETURN_IF_ERROR(change_value_range( - range, value, ColumnValueRange::remove_fixed_value_range, fn_name)); - } else { - RETURN_IF_ERROR(change_value_range( - not_in_range, value, ColumnValueRange::add_fixed_value_range, fn_name)); - } - iter->next(); - } - effect = true; - } else if (TExprNodeType::BINARY_PRED == expr->node_type()) { - DCHECK(expr->children().size() == 2); - - auto ne_checker = [](const std::string& fn_name) { return fn_name == "ne"; }; - StringRef value; - int slot_ref_child = -1; - if (_should_push_down_binary_predicate(reinterpret_cast(expr), expr_ctx, - &value, &slot_ref_child, ne_checker)) { - DCHECK(slot_ref_child >= 0); - // where A = nullptr should return empty result set - if (value.data != nullptr) { - auto fn_name = std::string(""); - if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING || - T == TYPE_HLL) { - auto val = StringValue(value.data, value.size); - if (is_fixed_range) { - RETURN_IF_ERROR(change_value_range( - range, reinterpret_cast(&val), - ColumnValueRange::remove_fixed_value_range, fn_name)); - } else { - RETURN_IF_ERROR(change_value_range( - not_in_range, reinterpret_cast(&val), - ColumnValueRange::add_fixed_value_range, fn_name)); - } - } else { - if (is_fixed_range) { - RETURN_IF_ERROR(change_value_range( - range, reinterpret_cast(const_cast(value.data)), - ColumnValueRange::remove_fixed_value_range, fn_name)); - } else { - RETURN_IF_ERROR(change_value_range( - not_in_range, - reinterpret_cast(const_cast(value.data)), - ColumnValueRange::add_fixed_value_range, fn_name)); - } - } - effect = true; - } - } - } - - if (is_fixed_range || - not_in_range.get_fixed_value_size() <= _max_pushdown_conditions_per_column) { - if (!is_fixed_range) { - // push down not in condition to storage engine - not_in_range.to_in_condition(_olap_filter, false); - } - *push_down = effect; - } - return Status::OK(); -} - -template -Status VOlapScanNode::_normalize_is_null_predicate(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - bool* push_down) { - if (TExprNodeType::FUNCTION_CALL == expr->node_type()) { - if (reinterpret_cast(expr)->fn().name.function_name == "is_null_pred") { - auto temp_range = ColumnValueRange::create_empty_column_value_range( - slot->type().precision, slot->type().scale); - temp_range.set_contain_null(true); - range.intersection(temp_range); - *push_down = true; - } else if (reinterpret_cast(expr)->fn().name.function_name == - "is_not_null_pred") { - auto temp_range = ColumnValueRange::create_empty_column_value_range( - slot->type().precision, slot->type().scale); - temp_range.set_contain_null(false); - range.intersection(temp_range); - *push_down = true; - } - } - return Status::OK(); -} - -template -Status VOlapScanNode::_normalize_noneq_binary_predicate(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, - ColumnValueRange& range, - bool* push_down) { - if (TExprNodeType::BINARY_PRED == expr->node_type()) { - DCHECK(expr->children().size() == 2); - - auto noneq_checker = [](const std::string& fn_name) { - return fn_name != "ne" && fn_name != "eq"; - }; - StringRef value; - int slot_ref_child = -1; - if (_should_push_down_binary_predicate(reinterpret_cast(expr), expr_ctx, - &value, &slot_ref_child, noneq_checker)) { - DCHECK(slot_ref_child >= 0); - const std::string& fn_name = - reinterpret_cast(expr)->fn().name.function_name; - - // where A = nullptr should return empty result set - if (value.data != nullptr) { - *push_down = true; - if constexpr (T == TYPE_CHAR || T == TYPE_VARCHAR || T == TYPE_STRING || - T == TYPE_HLL) { - auto val = StringValue(value.data, value.size); - RETURN_IF_ERROR(change_value_range(range, reinterpret_cast(&val), - ColumnValueRange::add_value_range, - fn_name, slot_ref_child)); - } else { - RETURN_IF_ERROR(change_value_range( - range, reinterpret_cast(const_cast(value.data)), - ColumnValueRange::add_value_range, fn_name, slot_ref_child)); - } - } - } - } - return Status::OK(); -} - -Status VOlapScanNode::_normalize_bloom_filter(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, bool* push_down) { - if (TExprNodeType::BLOOM_PRED == expr->node_type()) { - DCHECK(expr->children().size() == 1); - _bloom_filters_push_down.emplace_back(slot->col_name(), expr->get_bloom_filter_func()); - *push_down = true; - } - return Status::OK(); -} - -Status VOlapScanNode::_normalize_function_filters(VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, bool* push_down) { - bool opposite = false; - VExpr* fn_expr = expr; - if (TExprNodeType::COMPOUND_PRED == expr->node_type() && - expr->fn().name.function_name == "not") { - fn_expr = fn_expr->children()[0]; - opposite = true; - } - - if (TExprNodeType::FUNCTION_CALL == fn_expr->node_type()) { - doris_udf::FunctionContext* fn_ctx = nullptr; - StringVal val; - if (_should_push_down_function_filter(reinterpret_cast(fn_expr), - expr_ctx, &val, &fn_ctx)) { - std::string col = slot->col_name(); - _push_down_functions.emplace_back(opposite, col, fn_ctx, val); - *push_down = true; - } - } - return Status::OK(); -} - -void VOlapScanNode::eval_const_conjuncts(VExpr* vexpr, VExprContext* expr_ctx, bool* push_down) { - char* constant_val = nullptr; - if (vexpr->is_constant()) { - if (const ColumnConst* const_column = - check_and_get_column(vexpr->get_const_col(expr_ctx)->column_ptr)) { - constant_val = const_cast(const_column->get_data_at(0).data); - if (constant_val == nullptr || *reinterpret_cast(constant_val) == false) { - *push_down = true; - _eos = true; - } - } else if (const ColumnVector* bool_column = - check_and_get_column>( - vexpr->get_const_col(expr_ctx)->column_ptr)) { - // TODO: If `vexpr->is_constant()` is true, a const column is expected here. - // But now we still don't cover all predicates for const expression. - // For example, for query `SELECT col FROM tbl WHERE 'PROMOTION' LIKE 'AAA%'`, - // predicate `like` will return a ColumnVector which contains a single value. - LOG(WARNING) << "Expr[" << vexpr->debug_string() - << "] should return a const column but actually is " - << vexpr->get_const_col(expr_ctx)->column_ptr->get_name(); - DCHECK_EQ(bool_column->size(), 1); - if (bool_column->size() == 1) { - constant_val = const_cast(bool_column->get_data_at(0).data); - if (constant_val == nullptr || *reinterpret_cast(constant_val) == false) { - *push_down = true; - _eos = true; - } - } else { - LOG(WARNING) << "Constant predicate in scan node should return a bool column with " - "`size == 1` but actually is " - << bool_column->size(); - } - } else { - LOG(WARNING) << "Expr[" << vexpr->debug_string() - << "] should return a boolean column but actually is " - << vexpr->get_const_col(expr_ctx)->column_ptr->get_name(); - } - } -} - -VExpr* VOlapScanNode::_normalize_predicate(RuntimeState* state, VExpr* conjunct_expr_root) { - static constexpr auto is_leaf = [](VExpr* expr) { return !expr->is_and_expr(); }; - auto in_predicate_checker = [](const std::vector& children, const VSlotRef** slot, - VExpr** child_contains_slot) { - if (children.empty() || - VExpr::expr_without_cast(children[0])->node_type() != TExprNodeType::SLOT_REF) { - // not a slot ref(column) - return false; - } - *slot = reinterpret_cast(VExpr::expr_without_cast(children[0])); - *child_contains_slot = children[0]; - return true; - }; - auto eq_predicate_checker = [](const std::vector& children, const VSlotRef** slot, - VExpr** child_contains_slot) { - for (const VExpr* child : children) { - if (VExpr::expr_without_cast(child)->node_type() != TExprNodeType::SLOT_REF) { - // not a slot ref(column) - continue; - } - *slot = reinterpret_cast(VExpr::expr_without_cast(child)); - *child_contains_slot = const_cast(child); - return true; - } - return false; - }; - - if (conjunct_expr_root != nullptr) { - if (is_leaf(conjunct_expr_root)) { - auto impl = conjunct_expr_root->get_impl(); - VExpr* cur_expr = impl ? const_cast(impl) : conjunct_expr_root; - SlotDescriptor* slot = nullptr; - ColumnValueRangeType* range = nullptr; - bool push_down = false; - eval_const_conjuncts(cur_expr, *(_vconjunct_ctx_ptr.get()), &push_down); - if (push_down) { - return nullptr; - } - if (_is_predicate_acting_on_slot(cur_expr, in_predicate_checker, &slot, &range) || - _is_predicate_acting_on_slot(cur_expr, eq_predicate_checker, &slot, &range)) { - std::visit( - [&](auto& value_range) { - RETURN_IF_PUSH_DOWN(_normalize_in_and_eq_predicate( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, value_range, - &push_down)); - RETURN_IF_PUSH_DOWN(_normalize_not_in_and_not_eq_predicate( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, value_range, - &push_down)); - RETURN_IF_PUSH_DOWN(_normalize_is_null_predicate( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, value_range, - &push_down)); - RETURN_IF_PUSH_DOWN(_normalize_noneq_binary_predicate( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, value_range, - &push_down)); - if (is_key_column(slot->col_name())) { - RETURN_IF_PUSH_DOWN(_normalize_bloom_filter( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, &push_down)); - if (state->enable_function_pushdown()) { - RETURN_IF_PUSH_DOWN(_normalize_function_filters( - cur_expr, *(_vconjunct_ctx_ptr.get()), slot, - &push_down)); - } - } - }, - *range); - } - if (push_down && is_key_column(slot->col_name())) { - return nullptr; - } else { - return conjunct_expr_root; - } - } else { - VExpr* left_child = _normalize_predicate(state, conjunct_expr_root->children()[0]); - VExpr* right_child = _normalize_predicate(state, conjunct_expr_root->children()[1]); - - if (left_child != nullptr && right_child != nullptr) { - conjunct_expr_root->set_children({left_child, right_child}); - return conjunct_expr_root; - } else { - // here only close the and expr self, do not close the child - conjunct_expr_root->set_children({}); - conjunct_expr_root->close(state, *_vconjunct_ctx_ptr, - (*_vconjunct_ctx_ptr)->get_function_state_scope()); - } - - // here do not close Expr* now - return left_child != nullptr ? left_child : right_child; - } - } - return conjunct_expr_root; -} - -Status VOlapScanNode::_append_rf_into_conjuncts(RuntimeState* state, std::vector& vexprs) { - if (!vexprs.empty()) { - VExpr* last_expr = nullptr; - if (_vconjunct_ctx_ptr) { - last_expr = (*_vconjunct_ctx_ptr)->root(); - } else { - DCHECK(_rf_vexpr_set.find(vexprs[0]) == _rf_vexpr_set.end()); - last_expr = vexprs[0]; - _rf_vexpr_set.insert(vexprs[0]); - } - for (size_t j = _vconjunct_ctx_ptr ? 0 : 1; j < vexprs.size(); j++) { - if (_rf_vexpr_set.find(vexprs[j]) != _rf_vexpr_set.end()) { - continue; - } - TFunction fn; - TFunctionName fn_name; - fn_name.__set_db_name(""); - fn_name.__set_function_name("and"); - fn.__set_name(fn_name); - fn.__set_binary_type(TFunctionBinaryType::BUILTIN); - std::vector arg_types; - arg_types.push_back(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); - arg_types.push_back(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); - fn.__set_arg_types(arg_types); - fn.__set_ret_type(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); - fn.__set_has_var_args(false); - - TExprNode texpr_node; - texpr_node.__set_type(create_type_desc(PrimitiveType::TYPE_BOOLEAN)); - texpr_node.__set_node_type(TExprNodeType::COMPOUND_PRED); - texpr_node.__set_opcode(TExprOpcode::COMPOUND_AND); - texpr_node.__set_fn(fn); - texpr_node.__set_is_nullable(last_expr->is_nullable() || vexprs[j]->is_nullable()); - VExpr* new_node = _pool->add(new VcompoundPred(texpr_node)); - new_node->add_child(last_expr); - DCHECK((vexprs[j])->get_impl() != nullptr); - new_node->add_child(vexprs[j]); - last_expr = new_node; - _rf_vexpr_set.insert(vexprs[j]); - } - auto new_vconjunct_ctx_ptr = _pool->add(new VExprContext(last_expr)); - if (_vconjunct_ctx_ptr) { - (*_vconjunct_ctx_ptr)->clone_fn_contexts(new_vconjunct_ctx_ptr); - } - RETURN_IF_ERROR(new_vconjunct_ctx_ptr->prepare(state, _row_descriptor)); - RETURN_IF_ERROR(new_vconjunct_ctx_ptr->open(state)); - if (_vconjunct_ctx_ptr) { - (*(_vconjunct_ctx_ptr.get()))->mark_as_stale(); - _stale_vexpr_ctxs.push_back(std::move(_vconjunct_ctx_ptr)); - } - _vconjunct_ctx_ptr.reset(new doris::vectorized::VExprContext*); - *(_vconjunct_ctx_ptr.get()) = new_vconjunct_ctx_ptr; - } - return Status::OK(); -} - -std::string VOlapScanNode::get_name() { - return fmt::format("VOlapScanNode({0})", _olap_scan_node.table_name); -} - -} // namespace doris::vectorized diff --git a/be/src/vec/exec/volap_scan_node.h b/be/src/vec/exec/volap_scan_node.h deleted file mode 100644 index 51f4ed97c0..0000000000 --- a/be/src/vec/exec/volap_scan_node.h +++ /dev/null @@ -1,346 +0,0 @@ -// 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 "exec/olap_common.h" -#include "exec/scan_node.h" -#include "exprs/bloomfilter_predicate.h" -#include "exprs/function_filter.h" -#include "exprs/in_predicate.h" -#include "exprs/runtime_filter.h" -#include "gen_cpp/PlanNodes_types.h" -#include "olap/tablet.h" -#include "util/progress_updater.h" -#include "vec/exprs/vectorized_fn_call.h" -#include "vec/exprs/vin_predicate.h" -#include "vec/exprs/vslot_ref.h" - -namespace doris { -class ObjectPool; -class TPlanNode; -class DescriptorTbl; -class RowBatch; -namespace vectorized { - -class VOlapScanner; - -class VOlapScanNode final : public ScanNode { -public: - VOlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - friend class VOlapScanner; - - Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override; - Status prepare(RuntimeState* state) override; - Status open(RuntimeState* state) override; - - Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override { - return Status::NotSupported("Not Implemented VOlapScanNode Node::get_next scalar"); - } - Status get_next(RuntimeState* state, Block* block, bool* eos) override; - Status close(RuntimeState* state) override; - - Status set_scan_ranges(const std::vector& scan_ranges) override; - - Status get_hints(TabletSharedPtr table, const TPaloScanRange& scan_range, int block_row_count, - bool is_begin_include, bool is_end_include, - const std::vector>& scan_key_range, - std::vector>* sub_scan_range, - RuntimeProfile* profile); - - std::string get_name() override; - -private: - // In order to ensure the accuracy of the query result - // only key column conjuncts will be remove as idle conjunct - bool is_key_column(const std::string& key_name); - - Status start_scan(RuntimeState* state); - Status normalize_conjuncts(); - Status build_key_ranges_and_filters(); - - template - static Status change_value_range(ColumnValueRange& range, void* value, - const ChangeFixedValueRangeFunc& func, - const std::string& fn_name, int slot_ref_child = -1); - - void transfer_thread(RuntimeState* state); - void scanner_thread(VOlapScanner* scanner); - Status start_scan_thread(RuntimeState* state); - - Status _add_blocks(std::vector& block); - int _start_scanner_thread_task(RuntimeState* state, int block_per_scanner); - Block* _alloc_block(bool& get_free_block); - - void _init_counter(RuntimeState* state); - // OLAP_SCAN_NODE profile layering: OLAP_SCAN_NODE, OlapScanner, and SegmentIterator - // according to the calling relationship - void init_scan_profile(); - const std::vector& runtime_filter_descs() const { - return _runtime_filter_descs; - } - - template - Status _normalize_in_and_eq_predicate(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - bool* push_down); - template - Status _normalize_not_in_and_not_eq_predicate(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - bool* push_down); - - template - Status _normalize_noneq_binary_predicate(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - bool* push_down); - - template - Status _normalize_is_null_predicate(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, ColumnValueRange& range, - bool* push_down); - - Status _normalize_bloom_filter(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, bool* push_down); - - void eval_const_conjuncts(VExpr* vexpr, VExprContext* expr_ctx, bool* push_down); - - vectorized::VExpr* _normalize_predicate(RuntimeState* state, - vectorized::VExpr* conjunct_expr_root); - - Status _normalize_function_filters(vectorized::VExpr* expr, VExprContext* expr_ctx, - SlotDescriptor* slot, bool* push_down); - - template - bool _should_push_down_in_predicate(VInPredicate* in_pred, VExprContext* expr_ctx); - - bool _is_predicate_acting_on_slot(VExpr* expr, - const std::function&, - const VSlotRef**, VExpr**)>& checker, - SlotDescriptor** slot_desc, ColumnValueRangeType** range); - - bool _should_push_down_binary_predicate( - VectorizedFnCall* fn_call, VExprContext* expr_ctx, StringRef* constant_val, - int* slot_ref_child, const std::function& fn_checker); - - bool _should_push_down_function_filter(VectorizedFnCall* fn_call, VExprContext* expr_ctx, - StringVal* constant_str, - doris_udf::FunctionContext** fn_ctx); - - Status _append_rf_into_conjuncts(RuntimeState* state, std::vector& vexprs); - - // Tuple id resolved in prepare() to set _tuple_desc; - TupleId _tuple_id; - // doris scan node used to scan doris - TOlapScanNode _olap_scan_node; - // tuple descriptors - const TupleDescriptor* _tuple_desc; - // tuple index - int _tuple_idx; - // string slots - std::vector _string_slots; - // collection slots - std::vector _collection_slots; - - bool _eos; - - // column -> ColumnValueRange map - std::map _column_value_ranges; - - OlapScanKeys _scan_keys; - - std::vector> _scan_ranges; - - std::vector _olap_filter; - // push down bloom filters to storage engine. - // 1. std::pair.first :: column name - // 2. std::pair.second :: shared_ptr of BloomFilterFuncBase - std::vector>> - _bloom_filters_push_down; - - // push down functions to storage engine - // only support scalar functions, now just support like / not like - std::vector _push_down_functions; - - // Pool for storing allocated scanner objects. We don't want to use the - // runtime pool to ensure that the scanner objects are deleted before this - // object is. - ObjectPool _scanner_pool; - - std::shared_ptr _transfer_thread; - - // Keeps track of total splits and the number finished. - ProgressUpdater _progress; - - // to limit _materialized_row_batches_bytes < _max_scanner_queue_size_bytes / 2 - std::atomic_size_t _materialized_row_batches_bytes = 0; - - std::atomic_int _running_thread = 0; - std::condition_variable _scan_thread_exit_cv; - - // to limit _scan_row_batches_bytes < _max_scanner_queue_size_bytes / 2 - std::atomic_size_t _scan_row_batches_bytes = 0; - - std::list _olap_scanners; - - int _max_materialized_row_batches; - // to limit _materialized_row_batches_bytes and _scan_row_batches_bytes - size_t _max_scanner_queue_size_bytes; - bool _start; - // Used in Scan thread to ensure thread-safe - std::atomic_bool _scanner_done; - std::atomic_bool _transfer_done; - - int _total_assign_num; - int _nice; - - // protect _status, for many thread may change _status - SpinLock _status_mutex; - Status _status; - RuntimeState* _runtime_state; - - RuntimeProfile::Counter* _scan_timer; - RuntimeProfile::Counter* _scan_cpu_timer = nullptr; - RuntimeProfile::Counter* _tablet_counter; - RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr; - RuntimeProfile::Counter* _reader_init_timer = nullptr; - RuntimeProfile::Counter* _scanner_sched_counter = nullptr; - TResourceInfo* _resource_info; - - int64_t _buffered_bytes; - EvalConjunctsFn _eval_conjuncts_fn; - - // the max num of scan keys of this scan request. - // it will set as BE's config `doris_max_scan_key_num`, - // or be overwritten by value in TQueryOptions - int32_t _max_scan_key_num = 1024; - // The max number of conditions in InPredicate that can be pushed down - // into OlapEngine. - // If conditions in InPredicate is larger than this, all conditions in - // InPredicate will not be pushed to the OlapEngine. - // it will set as BE's config `max_pushdown_conditions_per_column`, - // or be overwritten by value in TQueryOptions - int32_t _max_pushdown_conditions_per_column = 1024; - - struct RuntimeFilterContext { - RuntimeFilterContext() : apply_mark(false), runtimefilter(nullptr) {} - bool apply_mark; - IRuntimeFilter* runtimefilter; - }; - std::vector _runtime_filter_descs; - std::vector _runtime_filter_ctxs; - std::vector _runtime_filter_ready_flag; - std::shared_mutex _rf_lock; - std::map _conjunctid_to_runtime_filter_ctxs; - - std::unique_ptr _scanner_profile; - std::unique_ptr _segment_profile; - - // Counters - RuntimeProfile::Counter* _io_timer = nullptr; - RuntimeProfile::Counter* _read_compressed_counter = nullptr; - RuntimeProfile::Counter* _decompressor_timer = nullptr; - RuntimeProfile::Counter* _read_uncompressed_counter = nullptr; - RuntimeProfile::Counter* _raw_rows_counter = nullptr; - - RuntimeProfile::Counter* _rows_vec_cond_counter = nullptr; - RuntimeProfile::Counter* _vec_cond_timer = nullptr; - RuntimeProfile::Counter* _short_cond_timer = nullptr; - RuntimeProfile::Counter* _output_col_timer = nullptr; - - RuntimeProfile::Counter* _stats_filtered_counter = nullptr; - RuntimeProfile::Counter* _bf_filtered_counter = nullptr; - RuntimeProfile::Counter* _del_filtered_counter = nullptr; - RuntimeProfile::Counter* _conditions_filtered_counter = nullptr; - RuntimeProfile::Counter* _key_range_filtered_counter = nullptr; - - RuntimeProfile::Counter* _block_fetch_timer = nullptr; - RuntimeProfile::Counter* _block_load_timer = nullptr; - RuntimeProfile::Counter* _block_load_counter = nullptr; - // Not used any more, will be removed after non-vectorized code is removed - RuntimeProfile::Counter* _block_seek_timer = nullptr; - // Not used any more, will be removed after non-vectorized code is removed - RuntimeProfile::Counter* _block_seek_counter = nullptr; - // Add more detail seek timer and counter profile - // Read process is split into 3 stages: init, first read, lazy read - RuntimeProfile::Counter* _block_init_timer = nullptr; - RuntimeProfile::Counter* _block_init_seek_timer = nullptr; - RuntimeProfile::Counter* _block_init_seek_counter = nullptr; - RuntimeProfile::Counter* _first_read_timer = nullptr; - RuntimeProfile::Counter* _first_read_seek_timer = nullptr; - RuntimeProfile::Counter* _first_read_seek_counter = nullptr; - RuntimeProfile::Counter* _lazy_read_timer = nullptr; - RuntimeProfile::Counter* _lazy_read_seek_timer = nullptr; - RuntimeProfile::Counter* _lazy_read_seek_counter = nullptr; - - RuntimeProfile::Counter* _block_convert_timer = nullptr; - - RuntimeProfile::Counter* _index_load_timer = nullptr; - - // total pages read - // used by segment v2 - RuntimeProfile::Counter* _total_pages_num_counter = nullptr; - // page read from cache - // used by segment v2 - RuntimeProfile::Counter* _cached_pages_num_counter = nullptr; - - // row count filtered by bitmap inverted index - RuntimeProfile::Counter* _bitmap_index_filter_counter = nullptr; - // time fro bitmap inverted index read and filter - RuntimeProfile::Counter* _bitmap_index_filter_timer = nullptr; - // number of created olap scanners - RuntimeProfile::Counter* _num_scanners = nullptr; - - // number of segment filtered by column stat when creating seg iterator - RuntimeProfile::Counter* _filtered_segment_counter = nullptr; - // total number of segment related to this scan node - RuntimeProfile::Counter* _total_segment_counter = nullptr; - - RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; - RuntimeProfile::Counter* _scanner_wait_worker_timer = nullptr; - - RuntimeProfile::Counter* _olap_wait_batch_queue_timer = nullptr; - - // for debugging or profiling, record any info as you want - RuntimeProfile::Counter* _general_debug_timer[GENERAL_DEBUG_COUNT] = {}; - - std::vector _scan_blocks; - std::vector _materialized_blocks; - std::mutex _blocks_lock; - std::condition_variable _block_added_cv; - std::condition_variable _block_consumed_cv; - - std::mutex _scan_blocks_lock; - std::condition_variable _scan_block_added_cv; - - std::vector _free_blocks; - std::mutex _free_blocks_lock; - - std::list _volap_scanners; - std::mutex _volap_scanners_lock; - - int _max_materialized_blocks; - - size_t _block_size = 0; - - phmap::flat_hash_set _rf_vexpr_set; - std::vector> _stale_vexpr_ctxs; - int64_t _limit_per_scanner = -1; - phmap::flat_hash_map> - _id_to_slot_column_value_range; -}; -} // namespace vectorized -} // namespace doris diff --git a/be/src/vec/exec/volap_scanner.cpp b/be/src/vec/exec/volap_scanner.cpp deleted file mode 100644 index 2b313ad132..0000000000 --- a/be/src/vec/exec/volap_scanner.cpp +++ /dev/null @@ -1,477 +0,0 @@ -// 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/exec/volap_scanner.h" - -#include - -#include "olap/storage_engine.h" -#include "runtime/runtime_state.h" -#include "vec/core/block.h" -#include "vec/exec/volap_scan_node.h" -#include "vec/exprs/vexpr_context.h" - -namespace doris::vectorized { - -VOlapScanner::VOlapScanner(RuntimeState* runtime_state, VOlapScanNode* parent, bool aggregation, - bool need_agg_finalize, const TPaloScanRange& scan_range) - : _runtime_state(runtime_state), - _parent(parent), - _tuple_desc(parent->_tuple_desc), - _id(-1), - _is_open(false), - _aggregation(aggregation), - _need_agg_finalize(need_agg_finalize), - _version(-1) { - _tablet_schema = std::make_shared(); -} - -Status VOlapScanner::prepare( - const TPaloScanRange& scan_range, const std::vector& key_ranges, - const std::vector& filters, - const std::vector>>& bloom_filters, - const std::vector& function_filters) { - set_tablet_reader(); - // set limit to reduce end of rowset and segment mem use - _tablet_reader->set_batch_size( - _parent->limit() == -1 - ? _parent->_runtime_state->batch_size() - : std::min(static_cast(_parent->_runtime_state->batch_size()), - _parent->limit())); - - // Get olap table - TTabletId tablet_id = scan_range.tablet_id; - SchemaHash schema_hash = strtoul(scan_range.schema_hash.c_str(), nullptr, 10); - _version = strtoul(scan_range.version.c_str(), nullptr, 10); - { - std::string err; - _tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, true, &err); - if (_tablet.get() == nullptr) { - std::stringstream ss; - ss << "failed to get tablet. tablet_id=" << tablet_id - << ", with schema_hash=" << schema_hash << ", reason=" << err; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - _tablet_schema->copy_from(*_tablet->tablet_schema()); - if (_parent->_olap_scan_node.__isset.columns_desc && - !_parent->_olap_scan_node.columns_desc.empty() && - _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) { - // Originally scanner get TabletSchema from tablet object in BE. - // To support lightweight schema change for adding / dropping columns, - // tabletschema is bounded to rowset and tablet's schema maybe outdated, - // so we have to use schema from a query plan witch FE puts it in query plans. - _tablet_schema->clear_columns(); - for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) { - _tablet_schema->append_column(TabletColumn(column_desc)); - } - } - { - std::shared_lock rdlock(_tablet->get_header_lock()); - const RowsetSharedPtr rowset = _tablet->rowset_with_max_version(); - if (rowset == nullptr) { - std::stringstream ss; - ss << "fail to get latest version of tablet: " << tablet_id; - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - - // acquire tablet rowset readers at the beginning of the scan node - // to prevent this case: when there are lots of olap scanners to run for example 10000 - // the rowsets maybe compacted when the last olap scanner starts - Version rd_version(0, _version); - Status acquire_reader_st = - _tablet->capture_rs_readers(rd_version, &_tablet_reader_params.rs_readers); - if (!acquire_reader_st.ok()) { - LOG(WARNING) << "fail to init reader.res=" << acquire_reader_st; - std::stringstream ss; - ss << "failed to initialize storage reader. tablet=" << _tablet->full_name() - << ", res=" << acquire_reader_st - << ", backend=" << BackendOptions::get_localhost(); - return Status::InternalError(ss.str()); - } - - // Initialize tablet_reader_params - RETURN_IF_ERROR(_init_tablet_reader_params(key_ranges, filters, bloom_filters, - function_filters)); - } - } - - return Status::OK(); -} - -Status VOlapScanner::open() { - SCOPED_TIMER(_parent->_reader_init_timer); - - _runtime_filter_marks.resize(_parent->runtime_filter_descs().size(), false); - - auto res = _tablet_reader->init(_tablet_reader_params); - if (!res.ok()) { - std::stringstream ss; - ss << "failed to initialize storage reader. tablet=" - << _tablet_reader_params.tablet->full_name() << ", res=" << res - << ", backend=" << BackendOptions::get_localhost(); - return Status::InternalError(ss.str()); - } - return Status::OK(); -} - -TabletStorageType VOlapScanner::get_storage_type() { - int local_reader = 0; - for (const auto& reader : _tablet_reader_params.rs_readers) { - if (reader->rowset()->rowset_meta()->resource_id().empty()) { - local_reader++; - } - } - int total_reader = _tablet_reader_params.rs_readers.size(); - - if (local_reader == total_reader) { - return TabletStorageType::STORAGE_TYPE_LOCAL; - } else if (local_reader == 0) { - return TabletStorageType::STORAGE_TYPE_REMOTE; - } - return TabletStorageType::STORAGE_TYPE_REMOTE_AND_LOCAL; -} - -// it will be called under tablet read lock because capture rs readers need -Status VOlapScanner::_init_tablet_reader_params( - const std::vector& key_ranges, const std::vector& filters, - const std::vector>>& bloom_filters, - const std::vector& function_filters) { - // if the table with rowset [0-x] or [0-1] [2-y], and [0-1] is empty - bool single_version = - (_tablet_reader_params.rs_readers.size() == 1 && - _tablet_reader_params.rs_readers[0]->rowset()->start_version() == 0 && - !_tablet_reader_params.rs_readers[0] - ->rowset() - ->rowset_meta() - ->is_segments_overlapping()) || - (_tablet_reader_params.rs_readers.size() == 2 && - _tablet_reader_params.rs_readers[0]->rowset()->rowset_meta()->num_rows() == 0 && - _tablet_reader_params.rs_readers[1]->rowset()->start_version() == 2 && - !_tablet_reader_params.rs_readers[1] - ->rowset() - ->rowset_meta() - ->is_segments_overlapping()); - - if (_runtime_state->skip_storage_engine_merge()) { - _tablet_reader_params.direct_mode = true; - _aggregation = true; - } else { - _tablet_reader_params.direct_mode = _aggregation || single_version || - _parent->_olap_scan_node.__isset.push_down_agg_type_opt; - } - RETURN_IF_ERROR(_init_return_columns()); - - _tablet_reader_params.tablet = _tablet; - _tablet_reader_params.tablet_schema = _tablet_schema; - _tablet_reader_params.reader_type = READER_QUERY; - _tablet_reader_params.aggregation = _aggregation; - if (_parent->_olap_scan_node.__isset.push_down_agg_type_opt) - _tablet_reader_params.push_down_agg_type_opt = - _parent->_olap_scan_node.push_down_agg_type_opt; - _tablet_reader_params.version = Version(0, _version); - - // Condition - for (auto& filter : filters) { - _tablet_reader_params.conditions.push_back(filter); - } - std::copy(bloom_filters.cbegin(), bloom_filters.cend(), - std::inserter(_tablet_reader_params.bloom_filters, - _tablet_reader_params.bloom_filters.begin())); - - std::copy(function_filters.cbegin(), function_filters.cend(), - std::inserter(_tablet_reader_params.function_filters, - _tablet_reader_params.function_filters.begin())); - if (!_runtime_state->skip_delete_predicate()) { - auto& delete_preds = _tablet->delete_predicates(); - std::copy(delete_preds.cbegin(), delete_preds.cend(), - std::inserter(_tablet_reader_params.delete_predicates, - _tablet_reader_params.delete_predicates.begin())); - } - - // Merge the columns in delete predicate that not in latest schema in to current tablet schema - for (auto& del_pred_rs : _tablet_reader_params.delete_predicates) { - _tablet_schema->merge_dropped_columns(_tablet->tablet_schema(del_pred_rs->version())); - } - - // Range - for (auto key_range : key_ranges) { - if (key_range->begin_scan_range.size() == 1 && - key_range->begin_scan_range.get_value(0) == NEGATIVE_INFINITY) { - continue; - } - - _tablet_reader_params.start_key_include = key_range->begin_include; - _tablet_reader_params.end_key_include = key_range->end_include; - - _tablet_reader_params.start_key.push_back(key_range->begin_scan_range); - _tablet_reader_params.end_key.push_back(key_range->end_scan_range); - } - - _tablet_reader_params.profile = _parent->runtime_profile(); - _tablet_reader_params.runtime_state = _runtime_state; - - _tablet_reader_params.origin_return_columns = &_return_columns; - _tablet_reader_params.tablet_columns_convert_to_null_set = &_tablet_columns_convert_to_null_set; - - if (_tablet_reader_params.direct_mode) { - _tablet_reader_params.return_columns = _return_columns; - } else { - // we need to fetch all key columns to do the right aggregation on storage engine side. - for (size_t i = 0; i < _tablet_schema->num_key_columns(); ++i) { - _tablet_reader_params.return_columns.push_back(i); - } - for (auto index : _return_columns) { - if (_tablet_schema->column(index).is_key()) { - continue; - } else { - _tablet_reader_params.return_columns.push_back(index); - } - } - // expand the sequence column - if (_tablet_schema->has_sequence_col()) { - bool has_replace_col = false; - for (auto col : _return_columns) { - if (_tablet_schema->column(col).aggregation() == - FieldAggregationMethod::OLAP_FIELD_AGGREGATION_REPLACE) { - has_replace_col = true; - break; - } - } - if (auto sequence_col_idx = _tablet_schema->sequence_col_idx(); - has_replace_col && std::find(_return_columns.begin(), _return_columns.end(), - sequence_col_idx) == _return_columns.end()) { - _tablet_reader_params.return_columns.push_back(sequence_col_idx); - } - } - } - - // If a agg node is this scan node direct parent - // we will not call agg object finalize method in scan node, - // to avoid the unnecessary SerDe and improve query performance - _tablet_reader_params.need_agg_finalize = _need_agg_finalize; - - if (!config::disable_storage_page_cache) { - _tablet_reader_params.use_page_cache = true; - } - - if (_tablet->enable_unique_key_merge_on_write()) { - _tablet_reader_params.delete_bitmap = &_tablet->tablet_meta()->delete_bitmap(); - } - - if (!_runtime_state->skip_storage_engine_merge()) { - if (_parent->_olap_scan_node.__isset.sort_info && - _parent->_olap_scan_node.sort_info.is_asc_order.size() > 0) { - _limit = _parent->_limit_per_scanner; - _tablet_reader_params.read_orderby_key = true; - if (!_parent->_olap_scan_node.sort_info.is_asc_order[0]) { - _tablet_reader_params.read_orderby_key_reverse = true; - } - _tablet_reader_params.read_orderby_key_num_prefix_columns = - _parent->_olap_scan_node.sort_info.is_asc_order.size(); - } - } - - return Status::OK(); -} - -Status VOlapScanner::_init_return_columns() { - for (auto slot : _tuple_desc->slots()) { - if (!slot->is_materialized()) { - continue; - } - - int32_t index = slot->col_unique_id() >= 0 - ? _tablet_schema->field_index(slot->col_unique_id()) - : _tablet_schema->field_index(slot->col_name()); - if (index < 0) { - std::stringstream ss; - ss << "field name is invalid. field=" << slot->col_name(); - LOG(WARNING) << ss.str(); - return Status::InternalError(ss.str()); - } - _return_columns.push_back(index); - if (slot->is_nullable() && !_tablet_schema->column(index).is_nullable()) { - _tablet_columns_convert_to_null_set.emplace(index); - } - } - - if (_return_columns.empty()) { - return Status::InternalError("failed to build storage scanner, no materialized slot!"); - } - return Status::OK(); -} - -Status VOlapScanner::get_block(RuntimeState* state, vectorized::Block* block, bool* eof) { - // only empty block should be here - DCHECK(block->rows() == 0); - - int64_t raw_rows_threshold = raw_rows_read() + config::doris_scanner_row_num; - if (!block->mem_reuse()) { - for (const auto slot_desc : _tuple_desc->slots()) { - block->insert(ColumnWithTypeAndName(slot_desc->get_empty_mutable_column(), - slot_desc->get_data_type_ptr(), - slot_desc->col_name())); - } - } - - { - SCOPED_TIMER(_parent->_scan_timer); - do { - // Read one block from block reader - auto res = _tablet_reader->next_block_with_aggregation(block, nullptr, nullptr, eof); - if (!res) { - std::stringstream ss; - ss << "Internal Error: read storage fail. res=" << res - << ", tablet=" << _tablet->full_name() - << ", backend=" << BackendOptions::get_localhost(); - return res; - } - _num_rows_read += block->rows(); - _update_realtime_counter(); - RETURN_IF_ERROR( - VExprContext::filter_block(_vconjunct_ctx, block, _tuple_desc->slots().size())); - // record rows return (after filter) for _limit check - _num_rows_return += block->rows(); - } while (block->rows() == 0 && !(*eof) && raw_rows_read() < raw_rows_threshold); - } - // NOTE: - // There is no need to check raw_bytes_threshold since block->rows() == 0 is checked first. - // But checking raw_bytes_threshold is still added here for consistency with raw_rows_threshold - // and olap_scanner.cpp. - - // set eof to true if per scanner limit is reached - // currently for query: ORDER BY key LIMIT n - if (_limit > 0 && _num_rows_return > _limit) { - *eof = true; - } - - return Status::OK(); -} - -void VOlapScanner::_update_realtime_counter() { - auto& stats = _tablet_reader->stats(); - COUNTER_UPDATE(_parent->_read_compressed_counter, stats.compressed_bytes_read); - _compressed_bytes_read += stats.compressed_bytes_read; - _tablet_reader->mutable_stats()->compressed_bytes_read = 0; - - COUNTER_UPDATE(_parent->_raw_rows_counter, stats.raw_rows_read); - // if raw_rows_read is reset, scanNode will scan all table rows which may cause BE crash - _raw_rows_read += stats.raw_rows_read; - _tablet_reader->mutable_stats()->raw_rows_read = 0; -} - -Status VOlapScanner::close(RuntimeState* state) { - if (_is_closed) { - return Status::OK(); - } - for (auto& ctx : _stale_vexpr_ctxs) { - ctx->close(state); - } - if (_vconjunct_ctx) { - _vconjunct_ctx->close(state); - } - // olap scan node will call scanner.close() when finished - // will release resources here - // if not clear rowset readers in read_params here - // readers will be release when runtime state deconstructed but - // deconstructor in reader references runtime state - // so that it will core - _tablet_reader_params.rs_readers.clear(); - update_counter(); - _tablet_reader.reset(); - _is_closed = true; - return Status::OK(); -} - -void VOlapScanner::update_counter() { - if (_has_update_counter) { - return; - } - auto& stats = _tablet_reader->stats(); - - COUNTER_UPDATE(_parent->rows_read_counter(), _num_rows_read); - COUNTER_UPDATE(_parent->_rows_pushed_cond_filtered_counter, _num_rows_pushed_cond_filtered); - - COUNTER_UPDATE(_parent->_io_timer, stats.io_ns); - COUNTER_UPDATE(_parent->_read_compressed_counter, stats.compressed_bytes_read); - _compressed_bytes_read += stats.compressed_bytes_read; - COUNTER_UPDATE(_parent->_decompressor_timer, stats.decompress_ns); - COUNTER_UPDATE(_parent->_read_uncompressed_counter, stats.uncompressed_bytes_read); - COUNTER_UPDATE(_parent->bytes_read_counter(), stats.bytes_read); - - COUNTER_UPDATE(_parent->_block_load_timer, stats.block_load_ns); - COUNTER_UPDATE(_parent->_block_load_counter, stats.blocks_load); - COUNTER_UPDATE(_parent->_block_fetch_timer, stats.block_fetch_ns); - COUNTER_UPDATE(_parent->_block_seek_timer, stats.block_seek_ns); - COUNTER_UPDATE(_parent->_block_seek_counter, stats.block_seek_num); - COUNTER_UPDATE(_parent->_block_convert_timer, stats.block_convert_ns); - - COUNTER_UPDATE(_parent->_raw_rows_counter, stats.raw_rows_read); - // if raw_rows_read is reset, scanNode will scan all table rows which may cause BE crash - _raw_rows_read += _tablet_reader->mutable_stats()->raw_rows_read; - // COUNTER_UPDATE(_parent->_filtered_rows_counter, stats.num_rows_filtered); - COUNTER_UPDATE(_parent->_vec_cond_timer, stats.vec_cond_ns); - COUNTER_UPDATE(_parent->_short_cond_timer, stats.short_cond_ns); - COUNTER_UPDATE(_parent->_block_init_timer, stats.block_init_ns); - COUNTER_UPDATE(_parent->_block_init_seek_timer, stats.block_init_seek_ns); - COUNTER_UPDATE(_parent->_block_init_seek_counter, stats.block_init_seek_num); - COUNTER_UPDATE(_parent->_first_read_timer, stats.first_read_ns); - COUNTER_UPDATE(_parent->_first_read_seek_timer, stats.block_first_read_seek_ns); - COUNTER_UPDATE(_parent->_first_read_seek_counter, stats.block_first_read_seek_num); - COUNTER_UPDATE(_parent->_lazy_read_timer, stats.lazy_read_ns); - COUNTER_UPDATE(_parent->_lazy_read_seek_timer, stats.block_lazy_read_seek_ns); - COUNTER_UPDATE(_parent->_lazy_read_seek_counter, stats.block_lazy_read_seek_num); - COUNTER_UPDATE(_parent->_output_col_timer, stats.output_col_ns); - COUNTER_UPDATE(_parent->_rows_vec_cond_counter, stats.rows_vec_cond_filtered); - - COUNTER_UPDATE(_parent->_stats_filtered_counter, stats.rows_stats_filtered); - COUNTER_UPDATE(_parent->_bf_filtered_counter, stats.rows_bf_filtered); - COUNTER_UPDATE(_parent->_del_filtered_counter, stats.rows_del_filtered); - COUNTER_UPDATE(_parent->_del_filtered_counter, stats.rows_del_by_bitmap); - COUNTER_UPDATE(_parent->_del_filtered_counter, stats.rows_vec_del_cond_filtered); - - COUNTER_UPDATE(_parent->_conditions_filtered_counter, stats.rows_conditions_filtered); - COUNTER_UPDATE(_parent->_key_range_filtered_counter, stats.rows_key_range_filtered); - - COUNTER_UPDATE(_parent->_index_load_timer, stats.index_load_ns); - - size_t timer_count = sizeof(stats.general_debug_ns) / sizeof(*stats.general_debug_ns); - for (size_t i = 0; i < timer_count; ++i) { - COUNTER_UPDATE(_parent->_general_debug_timer[i], stats.general_debug_ns[i]); - } - - COUNTER_UPDATE(_parent->_total_pages_num_counter, stats.total_pages_num); - COUNTER_UPDATE(_parent->_cached_pages_num_counter, stats.cached_pages_num); - - COUNTER_UPDATE(_parent->_bitmap_index_filter_counter, stats.rows_bitmap_index_filtered); - COUNTER_UPDATE(_parent->_bitmap_index_filter_timer, stats.bitmap_index_filter_timer); - - COUNTER_UPDATE(_parent->_filtered_segment_counter, stats.filtered_segment_number); - COUNTER_UPDATE(_parent->_total_segment_counter, stats.total_segment_number); - - DorisMetrics::instance()->query_scan_bytes->increment(_compressed_bytes_read); - DorisMetrics::instance()->query_scan_rows->increment(_raw_rows_read); - - _tablet->query_scan_bytes->increment(_compressed_bytes_read); - _tablet->query_scan_rows->increment(_raw_rows_read); - _tablet->query_scan_count->increment(1); - - _has_update_counter = true; -} -} // namespace doris::vectorized diff --git a/be/src/vec/exec/volap_scanner.h b/be/src/vec/exec/volap_scanner.h deleted file mode 100644 index e5dec37469..0000000000 --- a/be/src/vec/exec/volap_scanner.h +++ /dev/null @@ -1,153 +0,0 @@ -// 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 "exec/exec_node.h" -#include "exec/olap_utils.h" -#include "gen_cpp/PaloInternalService_types.h" -#include "gen_cpp/PlanNodes_types.h" -#include "olap/tablet.h" -#include "vec/exprs/vexpr_context.h" -#include "vec/olap/block_reader.h" - -namespace doris { -class RuntimeProfile; -class Field; -class RowBatch; - -namespace vectorized { -class VOlapScanNode; - -class VOlapScanner { -public: - VOlapScanner(RuntimeState* runtime_state, VOlapScanNode* parent, bool aggregation, - bool need_agg_finalize, const TPaloScanRange& scan_range); - virtual ~VOlapScanner() = default; - - Status prepare(const TPaloScanRange& scan_range, const std::vector& key_ranges, - const std::vector& filters, - const std::vector>>& - bloom_filters, - const std::vector& function_filters); - - Status open(); - - Status get_block(RuntimeState* state, vectorized::Block* block, bool* eof); - - Status close(RuntimeState* state); - - Status get_batch(RuntimeState* state, RowBatch* row_batch, bool* eos) { - return Status::NotSupported("Not Implemented VOlapScanNode Node::get_next scalar"); - } - - RuntimeState* runtime_state() { return _runtime_state; } - - VExprContext** vconjunct_ctx_ptr() { return &_vconjunct_ctx; } - - void discard_conjuncts() { - _vconjunct_ctx->mark_as_stale(); - _stale_vexpr_ctxs.push_back(_vconjunct_ctx); - _vconjunct_ctx = nullptr; - } - - void mark_to_need_to_close() { _need_to_close = true; } - - bool need_to_close() { return _need_to_close; } - - int id() const { return _id; } - void set_id(int id) { _id = id; } - bool is_open() const { return _is_open; } - void set_opened() { _is_open = true; } - - int64_t raw_rows_read() const { return _raw_rows_read; } - - void update_counter(); - - const std::string& scan_disk() const { return _tablet->data_dir()->path(); } - - void start_wait_worker_timer() { - _watcher.reset(); - _watcher.start(); - } - - int64_t update_wait_worker_timer() const { return _watcher.elapsed_time(); } - - std::vector* mutable_runtime_filter_marks() { return &_runtime_filter_marks; } - - TabletStorageType get_storage_type(); - -private: - Status _init_tablet_reader_params( - const std::vector& key_ranges, const std::vector& filters, - const std::vector>>& - bloom_filters, - const std::vector& function_filters); - Status _init_return_columns(); - - // Update profile that need to be reported in realtime. - void _update_realtime_counter(); - - virtual void set_tablet_reader() { _tablet_reader = std::make_unique(); } - - RuntimeState* _runtime_state; - VOlapScanNode* _parent; - const TupleDescriptor* _tuple_desc; /**< tuple descriptor */ - - // to record which runtime filters have been used - std::vector _runtime_filter_marks; - - int64_t _limit = -1; - - int _id; - bool _is_open; - bool _aggregation; - bool _need_agg_finalize = true; - bool _has_update_counter = false; - - TabletReader::ReaderParams _tablet_reader_params; - std::unique_ptr _tablet_reader; - - TabletSharedPtr _tablet; - int64_t _version; - - std::vector _return_columns; - std::unordered_set _tablet_columns_convert_to_null_set; - - // time costed and row returned statistics - int64_t _num_rows_read = 0; - int64_t _raw_rows_read = 0; - int64_t _num_rows_return = 0; - int64_t _compressed_bytes_read = 0; - - // number rows filtered by pushed condition - int64_t _num_rows_pushed_cond_filtered = 0; - - bool _is_closed = false; - - MonotonicStopWatch _watcher; - - VExprContext* _vconjunct_ctx = nullptr; - bool _need_to_close = false; - - TabletSchemaSPtr _tablet_schema; - - std::vector _stale_vexpr_ctxs; -}; - -} // namespace vectorized -} // namespace doris