| // 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/scan/scanner.h" |
| |
| #include <glog/logging.h> |
| |
| #include "common/config.h" |
| #include "common/status.h" |
| #include "pipeline/exec/scan_operator.h" |
| #include "runtime/descriptors.h" |
| #include "util/defer_op.h" |
| #include "util/runtime_profile.h" |
| #include "vec/columns/column_nothing.h" |
| #include "vec/core/column_with_type_and_name.h" |
| #include "vec/exec/scan/scan_node.h" |
| #include "vec/exprs/vexpr_context.h" |
| |
| namespace doris::vectorized { |
| |
| Scanner::Scanner(RuntimeState* state, pipeline::ScanLocalStateBase* local_state, int64_t limit, |
| RuntimeProfile* profile) |
| : _state(state), |
| _local_state(local_state), |
| _limit(limit), |
| _profile(profile), |
| _output_tuple_desc(_local_state->output_tuple_desc()), |
| _output_row_descriptor(_local_state->_parent->output_row_descriptor()), |
| _has_prepared(false) { |
| DorisMetrics::instance()->scanner_cnt->increment(1); |
| } |
| |
| Status Scanner::init(RuntimeState* state, const VExprContextSPtrs& conjuncts) { |
| if (!conjuncts.empty()) { |
| _conjuncts.resize(conjuncts.size()); |
| for (size_t i = 0; i != conjuncts.size(); ++i) { |
| RETURN_IF_ERROR(conjuncts[i]->clone(state, _conjuncts[i])); |
| } |
| } |
| |
| const auto& projections = _local_state->_projections; |
| if (!projections.empty()) { |
| _projections.resize(projections.size()); |
| for (size_t i = 0; i != projections.size(); ++i) { |
| RETURN_IF_ERROR(projections[i]->clone(state, _projections[i])); |
| } |
| } |
| |
| const auto& intermediate_projections = _local_state->_intermediate_projections; |
| if (!intermediate_projections.empty()) { |
| _intermediate_projections.resize(intermediate_projections.size()); |
| for (int i = 0; i < intermediate_projections.size(); i++) { |
| _intermediate_projections[i].resize(intermediate_projections[i].size()); |
| for (int j = 0; j < intermediate_projections[i].size(); j++) { |
| RETURN_IF_ERROR(intermediate_projections[i][j]->clone( |
| state, _intermediate_projections[i][j])); |
| } |
| } |
| } |
| |
| return Status::OK(); |
| } |
| |
| Status Scanner::get_block_after_projects(RuntimeState* state, vectorized::Block* block, bool* eos) { |
| auto& row_descriptor = _local_state->_parent->row_descriptor(); |
| if (_output_row_descriptor) { |
| if (_alreay_eos) { |
| *eos = true; |
| _padding_block.swap(_origin_block); |
| } else { |
| _origin_block.clear_column_data(row_descriptor.num_materialized_slots()); |
| while (_padding_block.rows() < state->batch_size() / 4 && !*eos) { |
| RETURN_IF_ERROR(get_block(state, &_origin_block, eos)); |
| if (_origin_block.rows() >= state->batch_size() / 4) { |
| break; |
| } |
| |
| if (_origin_block.rows() + _padding_block.rows() <= state->batch_size()) { |
| _merge_padding_block(); |
| _origin_block.clear_column_data(row_descriptor.num_materialized_slots()); |
| } else { |
| if (_origin_block.rows() < _padding_block.rows()) { |
| _padding_block.swap(_origin_block); |
| } |
| break; |
| } |
| } |
| } |
| |
| // first output the origin block change eos = false, next time output padding block |
| // set the eos to true |
| if (*eos && !_padding_block.empty() && !_origin_block.empty()) { |
| _alreay_eos = true; |
| *eos = false; |
| } |
| if (_origin_block.empty() && !_padding_block.empty()) { |
| _padding_block.swap(_origin_block); |
| } |
| return _do_projections(&_origin_block, block); |
| } else { |
| return get_block(state, block, eos); |
| } |
| } |
| |
| Status Scanner::get_block(RuntimeState* state, Block* block, bool* eof) { |
| // only empty block should be here |
| DCHECK(block->rows() == 0); |
| // scanner running time |
| SCOPED_RAW_TIMER(&_per_scanner_timer); |
| int64_t rows_read_threshold = _num_rows_read + config::doris_scanner_row_num; |
| if (!block->mem_reuse()) { |
| for (auto* const slot_desc : _output_tuple_desc->slots()) { |
| block->insert(ColumnWithTypeAndName(slot_desc->get_empty_mutable_column(), |
| slot_desc->get_data_type_ptr(), |
| slot_desc->col_name())); |
| } |
| } |
| |
| { |
| do { |
| // 1. Get input block from scanner |
| { |
| // get block time |
| SCOPED_TIMER(_local_state->_scan_timer); |
| RETURN_IF_ERROR(_get_block_impl(state, block, eof)); |
| if (*eof) { |
| DCHECK(block->rows() == 0); |
| break; |
| } |
| _num_rows_read += block->rows(); |
| _num_byte_read += block->allocated_bytes(); |
| } |
| |
| // 2. Filter the output block finally. |
| { |
| SCOPED_TIMER(_local_state->_filter_timer); |
| RETURN_IF_ERROR(_filter_output_block(block)); |
| } |
| // record rows return (after filter) for _limit check |
| _num_rows_return += block->rows(); |
| } while (!_should_stop && !state->is_cancelled() && block->rows() == 0 && !(*eof) && |
| _num_rows_read < rows_read_threshold); |
| } |
| |
| if (state->is_cancelled()) { |
| // TODO: Should return the specific ErrorStatus instead of just Cancelled. |
| return Status::Cancelled("cancelled"); |
| } |
| *eof = *eof || _should_stop; |
| // set eof to true if per scanner limit is reached |
| // currently for query: ORDER BY key LIMIT n |
| *eof = *eof || (_limit > 0 && _num_rows_return >= _limit); |
| |
| return Status::OK(); |
| } |
| |
| Status Scanner::_filter_output_block(Block* block) { |
| auto old_rows = block->rows(); |
| Status st = VExprContext::filter_block(_conjuncts, block, block->columns()); |
| _counter.num_rows_unselected += old_rows - block->rows(); |
| return st; |
| } |
| |
| Status Scanner::_do_projections(vectorized::Block* origin_block, vectorized::Block* output_block) { |
| SCOPED_RAW_TIMER(&_per_scanner_timer); |
| SCOPED_RAW_TIMER(&_projection_timer); |
| |
| const size_t rows = origin_block->rows(); |
| if (rows == 0) { |
| return Status::OK(); |
| } |
| vectorized::Block input_block = *origin_block; |
| |
| std::vector<int> result_column_ids; |
| for (auto& projections : _intermediate_projections) { |
| result_column_ids.resize(projections.size()); |
| for (int i = 0; i < projections.size(); i++) { |
| RETURN_IF_ERROR(projections[i]->execute(&input_block, &result_column_ids[i])); |
| } |
| input_block.shuffle_columns(result_column_ids); |
| } |
| |
| DCHECK_EQ(rows, input_block.rows()); |
| MutableBlock mutable_block = |
| VectorizedUtils::build_mutable_mem_reuse_block(output_block, *_output_row_descriptor); |
| |
| auto& mutable_columns = mutable_block.mutable_columns(); |
| |
| DCHECK_EQ(mutable_columns.size(), _projections.size()); |
| |
| for (int i = 0; i < mutable_columns.size(); ++i) { |
| ColumnPtr column_ptr; |
| RETURN_IF_ERROR(_projections[i]->execute(&input_block, column_ptr)); |
| column_ptr = column_ptr->convert_to_full_column_if_const(); |
| if (mutable_columns[i]->is_nullable() != column_ptr->is_nullable()) { |
| throw Exception(ErrorCode::INTERNAL_ERROR, "Nullable mismatch"); |
| } |
| mutable_columns[i] = column_ptr->assume_mutable(); |
| } |
| |
| output_block->set_columns(std::move(mutable_columns)); |
| |
| // origin columns was moved into output_block, so we need to set origin_block to empty columns |
| auto empty_columns = origin_block->clone_empty_columns(); |
| origin_block->set_columns(std::move(empty_columns)); |
| DCHECK_EQ(output_block->rows(), rows); |
| |
| return Status::OK(); |
| } |
| |
| Status Scanner::try_append_late_arrival_runtime_filter() { |
| if (_applied_rf_num == _total_rf_num) { |
| return Status::OK(); |
| } |
| DCHECK(_applied_rf_num < _total_rf_num); |
| |
| int arrived_rf_num = 0; |
| RETURN_IF_ERROR(_local_state->_helper.try_append_late_arrival_runtime_filter( |
| _state, &arrived_rf_num, _local_state->_conjuncts, |
| _local_state->_parent->row_descriptor())); |
| |
| if (arrived_rf_num == _applied_rf_num) { |
| // No newly arrived runtime filters, just return; |
| return Status::OK(); |
| } |
| |
| // There are newly arrived runtime filters, |
| // renew the _conjuncts |
| if (!_conjuncts.empty()) { |
| _discard_conjuncts(); |
| } |
| // Notice that the number of runtime filters may be larger than _applied_rf_num. |
| // But it is ok because it will be updated at next time. |
| RETURN_IF_ERROR(_local_state->clone_conjunct_ctxs(_conjuncts)); |
| _applied_rf_num = arrived_rf_num; |
| return Status::OK(); |
| } |
| |
| Status Scanner::close(RuntimeState* state) { |
| #ifndef BE_TEST |
| COUNTER_UPDATE(_local_state->_scanner_wait_worker_timer, _scanner_wait_worker_timer); |
| #endif |
| return Status::OK(); |
| } |
| |
| bool Scanner::_try_close() { |
| bool expected = false; |
| return _is_closed.compare_exchange_strong(expected, true); |
| } |
| |
| void Scanner::_collect_profile_before_close() { |
| COUNTER_UPDATE(_local_state->_scan_cpu_timer, _scan_cpu_timer); |
| COUNTER_UPDATE(_local_state->_rows_read_counter, _num_rows_read); |
| |
| // Update stats for load |
| _state->update_num_rows_load_filtered(_counter.num_rows_filtered); |
| _state->update_num_rows_load_unselected(_counter.num_rows_unselected); |
| } |
| |
| void Scanner::update_scan_cpu_timer() { |
| int64_t cpu_time = _cpu_watch.elapsed_time(); |
| _scan_cpu_timer += cpu_time; |
| if (_state && _state->get_query_ctx()) { |
| _state->get_query_ctx()->resource_ctx()->cpu_context()->update_cpu_cost_ms(cpu_time); |
| } |
| } |
| |
| } // namespace doris::vectorized |