diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index c1d3038050fbd4..d3bd0f0a3a2436 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -305,24 +305,22 @@ struct OlapReaderStatistics { // block_load_ns // block_init_ns // block_init_seek_ns - // block_conditions_filtered_ns - // first_read_ns - // block_first_read_seek_ns + // generate_row_ranges_ns + // predicate_column_read_ns + // predicate_column_read_seek_ns // lazy_read_ns // block_lazy_read_seek_ns int64_t block_init_ns = 0; int64_t block_init_seek_num = 0; int64_t block_init_seek_ns = 0; - int64_t first_read_ns = 0; - int64_t second_read_ns = 0; - int64_t block_first_read_seek_num = 0; - int64_t block_first_read_seek_ns = 0; + int64_t predicate_column_read_ns = 0; + int64_t non_predicate_read_ns = 0; + int64_t predicate_column_read_seek_num = 0; + int64_t predicate_column_read_seek_ns = 0; int64_t lazy_read_ns = 0; int64_t block_lazy_read_seek_num = 0; int64_t block_lazy_read_seek_ns = 0; - int64_t block_convert_ns = 0; - int64_t raw_rows_read = 0; int64_t rows_vec_cond_filtered = 0; @@ -351,11 +349,10 @@ struct OlapReaderStatistics { int64_t rows_del_by_bitmap = 0; // the number of rows filtered by various column indexes. int64_t rows_conditions_filtered = 0; - int64_t block_conditions_filtered_ns = 0; - int64_t block_conditions_filtered_bf_ns = 0; - int64_t block_conditions_filtered_zonemap_ns = 0; - int64_t block_conditions_filtered_zonemap_rp_ns = 0; - int64_t block_conditions_filtered_dict_ns = 0; + int64_t generate_row_ranges_ns = 0; + int64_t generate_row_ranges_by_bf_ns = 0; + int64_t generate_row_ranges_by_zonemap_ns = 0; + int64_t generate_row_ranges_by_dict_ns = 0; int64_t index_load_ns = 0; @@ -372,7 +369,6 @@ struct OlapReaderStatistics { int64_t inverted_index_query_cache_miss = 0; int64_t inverted_index_query_null_bitmap_timer = 0; int64_t inverted_index_query_bitmap_copy_timer = 0; - int64_t inverted_index_query_bitmap_op_timer = 0; int64_t inverted_index_searcher_open_timer = 0; int64_t inverted_index_searcher_search_timer = 0; int64_t inverted_index_searcher_cache_hit = 0; diff --git a/be/src/olap/rowset/rowset_reader_context.h b/be/src/olap/rowset/rowset_reader_context.h index 0d4f5897772ad5..7af1ce7c0472d4 100644 --- a/be/src/olap/rowset/rowset_reader_context.h +++ b/be/src/olap/rowset/rowset_reader_context.h @@ -75,14 +75,12 @@ struct RowsetReaderContext { bool enable_unique_key_merge_on_write = false; const DeleteBitmap* delete_bitmap = nullptr; bool record_rowids = false; - bool is_vertical_compaction = false; bool is_key_column_group = false; const std::set* output_columns = nullptr; RowsetId rowset_id; // slots that cast may be eliminated in storage layer std::map target_cast_type_for_variants; int64_t ttl_seconds = 0; - size_t topn_limit = 0; }; } // namespace doris diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index faad089e09ff72..f43ccf37e78280 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -497,7 +497,7 @@ Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_ra } Status SegmentIterator::_get_row_ranges_by_column_conditions() { - SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_ns); + SCOPED_RAW_TIMER(&_opts.stats->generate_row_ranges_ns); if (_row_bitmap.isEmpty()) { return Status::OK(); } @@ -565,7 +565,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row size_t pre_size = 0; { - SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_bf_ns); + SCOPED_RAW_TIMER(&_opts.stats->generate_row_ranges_by_bf_ns); // first filter data by bloom filter index // bloom filter index only use CondColumn RowRanges bf_row_ranges = RowRanges::create_single(num_rows()); @@ -588,7 +588,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row } { - SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_zonemap_ns); + SCOPED_RAW_TIMER(&_opts.stats->generate_row_ranges_by_zonemap_ns); RowRanges zone_map_row_ranges = RowRanges::create_single(num_rows()); // second filter data by zone map for (const auto& cid : cids) { @@ -652,7 +652,7 @@ Status SegmentIterator::_get_row_ranges_from_conditions(RowRanges* condition_row } { - SCOPED_RAW_TIMER(&_opts.stats->block_conditions_filtered_dict_ns); + SCOPED_RAW_TIMER(&_opts.stats->generate_row_ranges_by_dict_ns); /// Low cardinality optimization is currently not very stable, so to prevent data corruption, /// we are temporarily disabling its use in data compaction. if (_opts.io_ctx.reader_type == ReaderType::READER_QUERY) { @@ -1400,7 +1400,7 @@ Status SegmentIterator::_vec_init_lazy_materialization() { if (!_is_common_expr_column[cid]) { _non_predicate_columns.push_back(cid); } else { - _second_read_column_ids.push_back(cid); + _non_predicate_column_ids.push_back(cid); } } } @@ -1410,13 +1410,13 @@ Status SegmentIterator::_vec_init_lazy_materialization() { if (_lazy_materialization_read) { // insert pred cid to first_read_columns for (auto cid : pred_column_ids) { - _first_read_column_ids.push_back(cid); + _predicate_column_ids.push_back(cid); } } else if (!_is_need_vec_eval && !_is_need_short_eval && !_is_need_expr_eval) { // no pred exists, just read and output column for (int i = 0; i < _schema->num_column_ids(); i++) { auto cid = _schema->column_id(i); - _first_read_column_ids.push_back(cid); + _predicate_column_ids.push_back(cid); } } else { if (_is_need_vec_eval || _is_need_short_eval) { @@ -1428,18 +1428,18 @@ Status SegmentIterator::_vec_init_lazy_materialization() { _short_cir_pred_column_ids.end()); pred_id_set.insert(_vec_pred_column_ids.begin(), _vec_pred_column_ids.end()); - DCHECK(_second_read_column_ids.empty()); - // _second_read_column_ids must be empty. Otherwise _lazy_materialization_read must not false. + DCHECK(_non_predicate_column_ids.empty()); + // _non_predicate_column_ids must be empty. Otherwise _lazy_materialization_read must not false. for (int i = 0; i < _schema->num_column_ids(); i++) { auto cid = _schema->column_id(i); if (pred_id_set.find(cid) != pred_id_set.end()) { - _first_read_column_ids.push_back(cid); + _predicate_column_ids.push_back(cid); } // In the past, if schema columns > pred columns, the _lazy_materialization_read maybe == false, but // we make sure using _lazy_materialization_read= true now, so these logic may never happens. I comment // these lines and we could delete them in the future to make the code more clear. // else if (non_pred_set.find(cid) != non_pred_set.end()) { - // _first_read_column_ids.push_back(cid); + // _predicate_column_ids.push_back(cid); // // when _lazy_materialization_read = false, non-predicate column should also be filtered by sel idx, so we regard it as pred columns // _is_pred_column[cid] = true; // } @@ -1447,7 +1447,7 @@ Status SegmentIterator::_vec_init_lazy_materialization() { } else if (_is_need_expr_eval) { DCHECK(!_is_need_vec_eval && !_is_need_short_eval); for (auto cid : _common_expr_columns) { - _first_read_column_ids.push_back(cid); + _predicate_column_ids.push_back(cid); } } } @@ -1633,7 +1633,7 @@ void SegmentIterator::_output_non_pred_columns(vectorized::Block* block) { * 1. Reads a batch of rowids (up to the specified limit), and checks if they are continuous. * Continuous here means that the rowids form an unbroken sequence (e.g., 1, 2, 3, 4...). * - * 2. For each column that needs to be read (identified by _first_read_column_ids): + * 2. For each column that needs to be read (identified by _predicate_column_ids): * - If the rowids are continuous, the function uses seek_to_ordinal and next_batch * for efficient reading. * - If the rowids are not continuous, the function processes them in smaller batches @@ -1646,13 +1646,13 @@ void SegmentIterator::_output_non_pred_columns(vectorized::Block* block) { */ Status SegmentIterator::_read_columns_by_index(uint32_t nrows_read_limit, uint32_t& nrows_read, bool set_block_rowid) { - SCOPED_RAW_TIMER(&_opts.stats->first_read_ns); + SCOPED_RAW_TIMER(&_opts.stats->predicate_column_read_ns); nrows_read = _range_iter->read_batch_rowids(_block_rowids.data(), nrows_read_limit); bool is_continuous = (nrows_read > 1) && (_block_rowids[nrows_read - 1] - _block_rowids[0] == nrows_read - 1); - for (auto cid : _first_read_column_ids) { + for (auto cid : _predicate_column_ids) { auto& column = _current_return_columns[cid]; if (_no_need_read_key_data(cid, column, nrows_read)) { continue; @@ -1677,9 +1677,9 @@ Status SegmentIterator::_read_columns_by_index(uint32_t nrows_read_limit, uint32 if (is_continuous) { size_t rows_read = nrows_read; - _opts.stats->block_first_read_seek_num += 1; + _opts.stats->predicate_column_read_seek_num += 1; if (_opts.runtime_state && _opts.runtime_state->enable_profile()) { - SCOPED_RAW_TIMER(&_opts.stats->block_first_read_seek_ns); + SCOPED_RAW_TIMER(&_opts.stats->predicate_column_read_seek_ns); RETURN_IF_ERROR(_column_iterators[cid]->seek_to_ordinal(_block_rowids[0])); } else { RETURN_IF_ERROR(_column_iterators[cid]->seek_to_ordinal(_block_rowids[0])); @@ -1701,9 +1701,9 @@ Status SegmentIterator::_read_columns_by_index(uint32_t nrows_read_limit, uint32 if (batch_continuous) { size_t rows_read = current_batch_size; - _opts.stats->block_first_read_seek_num += 1; + _opts.stats->predicate_column_read_seek_num += 1; if (_opts.runtime_state && _opts.runtime_state->enable_profile()) { - SCOPED_RAW_TIMER(&_opts.stats->block_first_read_seek_ns); + SCOPED_RAW_TIMER(&_opts.stats->predicate_column_read_seek_ns); RETURN_IF_ERROR( _column_iterators[cid]->seek_to_ordinal(_block_rowids[processed])); } else { @@ -2066,8 +2066,8 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { RETURN_IF_ERROR(_read_columns_by_index( nrows_read_limit, _current_batch_rows_read, _lazy_materialization_read || _opts.record_rowids || _is_need_expr_eval)); - if (std::find(_first_read_column_ids.begin(), _first_read_column_ids.end(), - _schema->version_col_idx()) != _first_read_column_ids.end()) { + if (std::find(_predicate_column_ids.begin(), _predicate_column_ids.end(), + _schema->version_col_idx()) != _predicate_column_ids.end()) { _replace_version_col(_current_batch_rows_read); } @@ -2092,7 +2092,7 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { if (_non_predicate_columns.empty()) { return Status::InternalError("_non_predicate_columns is empty"); } - RETURN_IF_ERROR(_convert_to_expected_type(_first_read_column_ids)); + RETURN_IF_ERROR(_convert_to_expected_type(_predicate_column_ids)); RETURN_IF_ERROR(_convert_to_expected_type(_non_predicate_columns)); _output_non_pred_columns(block); } else { @@ -2113,27 +2113,28 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { if (selected_size > 0) { // step 3.1: output short circuit and predicate column - // when lazy materialization enables, _first_read_column_ids = distinct(_short_cir_pred_column_ids + _vec_pred_column_ids) + // when lazy materialization enables, _predicate_column_ids = distinct(_short_cir_pred_column_ids + _vec_pred_column_ids) // see _vec_init_lazy_materialization // todo(wb) need to tell input columnids from output columnids - RETURN_IF_ERROR(_output_column_by_sel_idx(block, _first_read_column_ids, + RETURN_IF_ERROR(_output_column_by_sel_idx(block, _predicate_column_ids, _sel_rowid_idx.data(), selected_size)); // step 3.2: read remaining expr column and evaluate it. if (_is_need_expr_eval) { // The predicate column contains the remaining expr column, no need second read. - if (!_second_read_column_ids.empty()) { - SCOPED_RAW_TIMER(&_opts.stats->second_read_ns); + if (!_non_predicate_column_ids.empty()) { + SCOPED_RAW_TIMER(&_opts.stats->non_predicate_read_ns); RETURN_IF_ERROR(_read_columns_by_rowids( - _second_read_column_ids, _block_rowids, _sel_rowid_idx.data(), + _non_predicate_column_ids, _block_rowids, _sel_rowid_idx.data(), selected_size, &_current_return_columns)); - if (std::find(_second_read_column_ids.begin(), - _second_read_column_ids.end(), _schema->version_col_idx()) != - _second_read_column_ids.end()) { + if (std::find(_non_predicate_column_ids.begin(), + _non_predicate_column_ids.end(), + _schema->version_col_idx()) != + _non_predicate_column_ids.end()) { _replace_version_col(selected_size); } - RETURN_IF_ERROR(_convert_to_expected_type(_second_read_column_ids)); - for (auto cid : _second_read_column_ids) { + RETURN_IF_ERROR(_convert_to_expected_type(_non_predicate_column_ids)); + for (auto cid : _non_predicate_column_ids) { auto loc = _schema_block_id_map[cid]; block->replace_by_position(loc, std::move(_current_return_columns[cid])); @@ -2166,17 +2167,17 @@ Status SegmentIterator::_next_batch_internal(vectorized::Block* block) { } } } else if (_is_need_expr_eval) { - RETURN_IF_ERROR(_convert_to_expected_type(_second_read_column_ids)); - for (auto cid : _second_read_column_ids) { + RETURN_IF_ERROR(_convert_to_expected_type(_non_predicate_column_ids)); + for (auto cid : _non_predicate_column_ids) { auto loc = _schema_block_id_map[cid]; block->replace_by_position(loc, std::move(_current_return_columns[cid])); } } } else if (_is_need_expr_eval) { - DCHECK(!_first_read_column_ids.empty()); - RETURN_IF_ERROR(_convert_to_expected_type(_first_read_column_ids)); + DCHECK(!_predicate_column_ids.empty()); + RETURN_IF_ERROR(_convert_to_expected_type(_predicate_column_ids)); // first read all rows are insert block, initialize sel_rowid_idx to all rows. - for (auto cid : _first_read_column_ids) { + for (auto cid : _predicate_column_ids) { auto loc = _schema_block_id_map[cid]; block->replace_by_position(loc, std::move(_current_return_columns[cid])); } diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.h b/be/src/olap/rowset/segment_v2/segment_iterator.h index c2e2139e8ad411..5626d15180c295 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.h +++ b/be/src/olap/rowset/segment_v2/segment_iterator.h @@ -431,8 +431,8 @@ class SegmentIterator : public RowwiseIterator { // first, read predicate columns by various index // second, read non-predicate columns // so we need a field to stand for columns first time to read - std::vector _first_read_column_ids; - std::vector _second_read_column_ids; + std::vector _predicate_column_ids; + std::vector _non_predicate_column_ids; std::vector _columns_to_filter; std::vector _converted_column_ids; std::vector _schema_block_id_map; // map from schema column id to column idx in Block diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index f7585e50422750..83d566fac9a28c 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -63,17 +63,13 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { "SerializeKeyArena", TUnit::BYTES, "MemoryUsage", 1); _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); - _serialize_key_timer = ADD_TIMER(Base::profile(), "SerializeKeyTime"); - _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); _merge_timer = ADD_TIMER(Base::profile(), "MergeTime"); _expr_timer = ADD_TIMER(Base::profile(), "ExprTime"); - _serialize_data_timer = ADD_TIMER(Base::profile(), "SerializeDataTime"); _deserialize_data_timer = ADD_TIMER(Base::profile(), "DeserializeAndMergeTime"); _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); _hash_table_limit_compute_timer = ADD_TIMER(Base::profile(), "DoLimitComputeTime"); _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); - _max_row_size_counter = ADD_COUNTER(Base::profile(), "MaxRowSizeInBytes", TUnit::UNIT); return Status::OK(); } diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 0b78ab15d2ac89..975b04477f203f 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -102,11 +102,8 @@ class AggSinkLocalState : public PipelineXSinkLocalState { RuntimeProfile::Counter* _hash_table_input_counter = nullptr; RuntimeProfile::Counter* _build_timer = nullptr; RuntimeProfile::Counter* _expr_timer = nullptr; - RuntimeProfile::Counter* _serialize_key_timer = nullptr; RuntimeProfile::Counter* _merge_timer = nullptr; - RuntimeProfile::Counter* _serialize_data_timer = nullptr; RuntimeProfile::Counter* _deserialize_data_timer = nullptr; - RuntimeProfile::Counter* _max_row_size_counter = nullptr; RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; RuntimeProfile::Counter* _hash_table_size_counter = nullptr; RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index 9cc35923d08bf0..a406bdc329ef50 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -29,20 +29,18 @@ namespace doris::pipeline { AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), _get_results_timer(nullptr), - _serialize_result_timer(nullptr), _hash_table_iterate_timer(nullptr), _insert_keys_to_column_timer(nullptr), - _serialize_data_timer(nullptr) {} + _insert_values_to_column_timer(nullptr) {} Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _get_results_timer = ADD_TIMER(profile(), "GetResultsTime"); - _serialize_result_timer = ADD_TIMER(profile(), "SerializeResultTime"); _hash_table_iterate_timer = ADD_TIMER(profile(), "HashTableIterateTime"); _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); - _serialize_data_timer = ADD_TIMER(profile(), "SerializeDataTime"); + _insert_values_to_column_timer = ADD_TIMER(profile(), "InsertValuesToColumnTime"); _merge_timer = ADD_TIMER(Base::profile(), "MergeTime"); _deserialize_data_timer = ADD_TIMER(Base::profile(), "DeserializeAndMergeTime"); @@ -57,7 +55,7 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { std::placeholders::_1, std::placeholders::_2, std::placeholders::_3); } else { - _executor.get_result = std::bind(&AggLocalState::_serialize_without_key, this, + _executor.get_result = std::bind(&AggLocalState::_get_results_without_key, this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3); } @@ -68,8 +66,8 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { std::placeholders::_2, std::placeholders::_3); } else { _executor.get_result = std::bind( - &AggLocalState::_serialize_with_serialized_key_result, this, - std::placeholders::_1, std::placeholders::_2, std::placeholders::_3); + &AggLocalState::_get_results_with_serialized_key, this, std::placeholders::_1, + std::placeholders::_2, std::placeholders::_3); } } @@ -93,18 +91,9 @@ Status AggLocalState::_create_agg_status(vectorized::AggregateDataPtr data) { return Status::OK(); } -Status AggLocalState::_destroy_agg_status(vectorized::AggregateDataPtr data) { - auto& shared_state = *Base::_shared_state; - for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { - shared_state.aggregate_evaluators[i]->function()->destroy( - data + shared_state.offsets_of_aggregate_states[i]); - } - return Status::OK(); -} - -Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, - vectorized::Block* block, bool* eos) { - SCOPED_TIMER(_serialize_result_timer); +Status AggLocalState::_get_results_with_serialized_key(RuntimeState* state, + vectorized::Block* block, bool* eos) { + SCOPED_TIMER(_get_results_timer); auto& shared_state = *_shared_state; int key_size = _shared_state->probe_expr_ctxs.size(); int agg_size = _shared_state->aggregate_evaluators.size(); @@ -124,7 +113,6 @@ Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, } } - SCOPED_TIMER(_get_results_timer); std::visit( vectorized::Overload { [&](std::monostate& arg) -> void { @@ -180,7 +168,7 @@ Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, } { - SCOPED_TIMER(_serialize_data_timer); + SCOPED_TIMER(_insert_values_to_column_timer); for (size_t i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { value_data_types[i] = shared_state.aggregate_evaluators[i] ->function() @@ -332,13 +320,13 @@ Status AggLocalState::_get_with_serialized_key_result(RuntimeState* state, vecto return Status::OK(); } -Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Block* block, - bool* eos) { +Status AggLocalState::_get_results_without_key(RuntimeState* state, vectorized::Block* block, + bool* eos) { + SCOPED_TIMER(_get_results_timer); auto& shared_state = *_shared_state; // 1. `child(0)->rows_returned() == 0` mean not data from child // in level two aggregation node should return NULL result // level one aggregation node set `eos = true` return directly - SCOPED_TIMER(_serialize_result_timer); if (UNLIKELY(_shared_state->input_num_rows == 0)) { *eos = true; return Status::OK(); @@ -577,17 +565,6 @@ template Status AggSourceOperatorX::merge_with_serialized_key_helper( template Status AggSourceOperatorX::merge_with_serialized_key_helper( RuntimeState* state, vectorized::Block* block); -size_t AggLocalState::_get_hash_table_size() { - return std::visit( - vectorized::Overload {[&](std::monostate& arg) -> size_t { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, - "uninited hash table"); - return 0; - }, - [&](auto& agg_method) { return agg_method.hash_table->size(); }}, - _shared_state->agg_data->method_variant); -} - void AggLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, size_t num_rows) { diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index a3824a381eb49c..4bb03670bc8c59 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -47,13 +47,12 @@ class AggLocalState final : public PipelineXLocalState { friend class AggSourceOperatorX; Status _get_without_key_result(RuntimeState* state, vectorized::Block* block, bool* eos); - Status _serialize_without_key(RuntimeState* state, vectorized::Block* block, bool* eos); + Status _get_results_without_key(RuntimeState* state, vectorized::Block* block, bool* eos); Status _get_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos); - Status _serialize_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, - bool* eos); + Status _get_results_with_serialized_key(RuntimeState* state, vectorized::Block* block, + bool* eos); Status _create_agg_status(vectorized::AggregateDataPtr data); - Status _destroy_agg_status(vectorized::AggregateDataPtr data); void _make_nullable_output_key(vectorized::Block* block) { if (block->rows() != 0) { auto& shared_state = *Base ::_shared_state; @@ -68,16 +67,14 @@ class AggLocalState final : public PipelineXLocalState { vectorized::ColumnRawPtrs& key_columns, size_t num_rows); void _emplace_into_hash_table(vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, size_t num_rows); - size_t _get_hash_table_size(); vectorized::PODArray _places; std::vector _deserialize_buffer; RuntimeProfile::Counter* _get_results_timer = nullptr; - RuntimeProfile::Counter* _serialize_result_timer = nullptr; RuntimeProfile::Counter* _hash_table_iterate_timer = nullptr; RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; - RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _insert_values_to_column_timer = nullptr; RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 11652db0a85798..839a485f2d98c7 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -29,9 +29,10 @@ Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); - _blocks_memory_usage = - _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + _evaluation_timer = ADD_TIMER(profile(), "GetPartitionBoundTime"); + _compute_agg_data_timer = ADD_TIMER(profile(), "ComputeAggDataTime"); + _compute_partition_by_timer = ADD_TIMER(profile(), "ComputePartitionByTime"); + _compute_order_by_timer = ADD_TIMER(profile(), "ComputeOrderByTime"); return Status::OK(); } @@ -288,33 +289,41 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block } } - for (size_t i = 0; i < _agg_functions_size; - ++i) { //insert _agg_input_columns, execute calculate for its - for (size_t j = 0; j < local_state._agg_expr_ctxs[i].size(); ++j) { - RETURN_IF_ERROR(_insert_range_column( - input_block, local_state._agg_expr_ctxs[i][j], - local_state._shared_state->agg_input_columns[i][j].get(), block_rows)); + { + SCOPED_TIMER(local_state._compute_agg_data_timer); + for (size_t i = 0; i < _agg_functions_size; + ++i) { //insert _agg_input_columns, execute calculate for its + for (size_t j = 0; j < local_state._agg_expr_ctxs[i].size(); ++j) { + RETURN_IF_ERROR(_insert_range_column( + input_block, local_state._agg_expr_ctxs[i][j], + local_state._shared_state->agg_input_columns[i][j].get(), block_rows)); + } } } - //record column idx in block - for (size_t i = 0; i < local_state._shared_state->partition_by_eq_expr_ctxs.size(); ++i) { - int result_col_id = -1; - RETURN_IF_ERROR(local_state._shared_state->partition_by_eq_expr_ctxs[i]->execute( - input_block, &result_col_id)); - DCHECK_GE(result_col_id, 0); - local_state._shared_state->partition_by_column_idxs[i] = result_col_id; + { + SCOPED_TIMER(local_state._compute_partition_by_timer); + for (size_t i = 0; i < local_state._shared_state->partition_by_eq_expr_ctxs.size(); ++i) { + int result_col_id = -1; + RETURN_IF_ERROR(local_state._shared_state->partition_by_eq_expr_ctxs[i]->execute( + input_block, &result_col_id)); + DCHECK_GE(result_col_id, 0); + local_state._shared_state->partition_by_column_idxs[i] = result_col_id; + } } - for (size_t i = 0; i < local_state._shared_state->order_by_eq_expr_ctxs.size(); ++i) { - int result_col_id = -1; - RETURN_IF_ERROR(local_state._shared_state->order_by_eq_expr_ctxs[i]->execute( - input_block, &result_col_id)); - DCHECK_GE(result_col_id, 0); - local_state._shared_state->ordey_by_column_idxs[i] = result_col_id; + { + SCOPED_TIMER(local_state._compute_order_by_timer); + for (size_t i = 0; i < local_state._shared_state->order_by_eq_expr_ctxs.size(); ++i) { + int result_col_id = -1; + RETURN_IF_ERROR(local_state._shared_state->order_by_eq_expr_ctxs[i]->execute( + input_block, &result_col_id)); + DCHECK_GE(result_col_id, 0); + local_state._shared_state->ordey_by_column_idxs[i] = result_col_id; + } } - local_state.mem_tracker()->consume(input_block->allocated_bytes()); - local_state._blocks_memory_usage->add(input_block->allocated_bytes()); + COUNTER_UPDATE(local_state._memory_used_counter, input_block->allocated_bytes()); + COUNTER_SET(local_state._peak_memory_usage_counter, local_state._memory_used_counter->value()); //TODO: if need improvement, the is a tips to maintain a free queue, //so the memory could reuse, no need to new/delete again; diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 7ef650c4383056..084998d2c36cdc 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -58,7 +58,9 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState _agg_expr_ctxs; }; diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 866e5b8119e3c3..c365726a60f04b 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -160,8 +160,11 @@ Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _blocks_memory_usage = - profile()->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + profile()->AddHighWaterMarkCounter("MemoryUsageBlocks", TUnit::BYTES, "", 1); + _evaluation_timer = ADD_TIMER(profile(), "GetPartitionBoundTime"); + _execute_timer = ADD_TIMER(profile(), "ExecuteTime"); + _get_next_timer = ADD_TIMER(profile(), "GetNextTime"); + _get_result_timer = ADD_TIMER(profile(), "GetResultsTime"); return Status::OK(); } @@ -232,12 +235,6 @@ Status AnalyticLocalState::open(RuntimeState* state) { std::placeholders::_1); } } - _executor.insert_result = - std::bind(&AnalyticLocalState::_insert_result_info, this, std::placeholders::_1); - _executor.execute = - std::bind(&AnalyticLocalState::_execute_for_win_func, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3, std::placeholders::_4); - _create_agg_status(); return Status::OK(); } @@ -281,6 +278,7 @@ void AnalyticLocalState::_destroy_agg_status() { void AnalyticLocalState::_execute_for_win_func(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end) { + SCOPED_TIMER(_execute_timer); for (size_t i = 0; i < _agg_functions_size; ++i) { std::vector agg_columns; for (int j = 0; j < _shared_state->agg_input_columns[i].size(); ++j) { @@ -299,6 +297,7 @@ void AnalyticLocalState::_execute_for_win_func(int64_t partition_start, int64_t } void AnalyticLocalState::_insert_result_info(int64_t current_block_rows) { + SCOPED_TIMER(_get_result_timer); int64_t current_block_row_pos = _shared_state->input_block_first_row_positions[_output_block_index]; int64_t get_result_start = _shared_state->current_row_position - current_block_row_pos; @@ -343,6 +342,7 @@ void AnalyticLocalState::_insert_result_info(int64_t current_block_rows) { } Status AnalyticLocalState::_get_next_for_rows(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); while (_shared_state->current_row_position < _shared_state->partition_by_end.pos && _window_end_position < current_block_rows) { int64_t range_start, range_end; @@ -366,31 +366,33 @@ Status AnalyticLocalState::_get_next_for_rows(size_t current_block_rows) { // Make sure range_start <= range_end range_start = std::min(range_start, range_end); } - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, range_start, - range_end); - _executor.insert_result(current_block_rows); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + range_start, range_end); + _insert_result_info(current_block_rows); } return Status::OK(); } Status AnalyticLocalState::_get_next_for_partition(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); if (_next_partition) { - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, - _partition_by_start.pos, _shared_state->partition_by_end.pos); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + _partition_by_start.pos, _shared_state->partition_by_end.pos); } - _executor.insert_result(current_block_rows); + _insert_result_info(current_block_rows); return Status::OK(); } Status AnalyticLocalState::_get_next_for_range(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); while (_shared_state->current_row_position < _shared_state->partition_by_end.pos && _window_end_position < current_block_rows) { if (_shared_state->current_row_position >= _order_by_end.pos) { _update_order_by_range(); - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, - _order_by_start.pos, _order_by_end.pos); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + _order_by_start.pos, _order_by_end.pos); } - _executor.insert_result(current_block_rows); + _insert_result_info(current_block_rows); } return Status::OK(); } @@ -536,7 +538,7 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block local_state.init_result_columns(); size_t current_block_rows = local_state._shared_state->input_blocks[local_state._output_block_index].rows(); - static_cast(local_state._executor.get_next(current_block_rows)); + RETURN_IF_ERROR(local_state._executor.get_next(current_block_rows)); if (local_state._window_end_position == current_block_rows) { break; } diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 38323f1b86bce2..fa7a676f9c43e9 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -96,17 +96,15 @@ class AnalyticLocalState final : public PipelineXLocalState std::vector _agg_functions; RuntimeProfile::Counter* _evaluation_timer = nullptr; + RuntimeProfile::Counter* _execute_timer = nullptr; + RuntimeProfile::Counter* _get_next_timer = nullptr; + RuntimeProfile::Counter* _get_result_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr; - using vectorized_execute = std::function; using vectorized_get_next = std::function; - using vectorized_get_result = std::function; struct executor { - vectorized_execute execute; vectorized_get_next get_next; - vectorized_get_result insert_result; }; executor _executor; diff --git a/be/src/pipeline/exec/cache_source_operator.cpp b/be/src/pipeline/exec/cache_source_operator.cpp index 5f8c5befc6a2b9..e98a18b76a3a98 100644 --- a/be/src/pipeline/exec/cache_source_operator.cpp +++ b/be/src/pipeline/exec/cache_source_operator.cpp @@ -65,7 +65,7 @@ Status CacheSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { // 3. lookup the cache and find proper slot order hit_cache = QueryCache::instance()->lookup(_cache_key, _version, &_query_cache_handle); - _runtime_profile->add_info_string("HitCache", hit_cache ? "1" : "0"); + _runtime_profile->add_info_string("HitCache", std::to_string(hit_cache)); if (hit_cache && !cache_param.force_refresh_query_cache) { _hit_cache_results = _query_cache_handle.get_cache_result(); auto hit_cache_slot_orders = _query_cache_handle.get_cache_slot_orders(); @@ -125,13 +125,16 @@ Status CacheSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b if (local_state._hit_cache_results == nullptr) { Defer insert_cache([&] { - if (*eos && local_state._need_insert_cache) { - local_state._runtime_profile->add_info_string("InsertCache", "1"); - local_state._global_cache->insert(local_state._cache_key, local_state._version, - local_state._local_cache_blocks, - local_state._slot_orders, - local_state._current_query_cache_bytes); - local_state._local_cache_blocks.clear(); + if (*eos) { + local_state._runtime_profile->add_info_string( + "InsertCache", std::to_string(local_state._need_insert_cache)); + if (local_state._need_insert_cache) { + local_state._global_cache->insert(local_state._cache_key, local_state._version, + local_state._local_cache_blocks, + local_state._slot_orders, + local_state._current_query_cache_bytes); + local_state._local_cache_blocks.clear(); + } } }); @@ -163,7 +166,6 @@ Status CacheSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b // over the max bytes, pass through the data, no need to do cache local_state._local_cache_blocks.clear(); local_state._need_insert_cache = false; - local_state._runtime_profile->add_info_string("InsertCache", "0"); } else { local_state._local_cache_blocks.emplace_back(std::move(output_block)); } diff --git a/be/src/pipeline/exec/datagen_operator.cpp b/be/src/pipeline/exec/datagen_operator.cpp index ac86db03d19216..f0a76992643179 100644 --- a/be/src/pipeline/exec/datagen_operator.cpp +++ b/be/src/pipeline/exec/datagen_operator.cpp @@ -70,17 +70,25 @@ Status DataGenSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* RETURN_IF_CANCELLED(state); auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - Status res = local_state._table_func->get_next(state, block, eos); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + { + SCOPED_TIMER(local_state._table_function_execution_timer); + RETURN_IF_ERROR(local_state._table_func->get_next(state, block, eos)); + } + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); + } local_state.reached_limit(block, eos); - return res; + return Status::OK(); } Status DataGenLocalState::init(RuntimeState* state, LocalStateInfo& info) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + _table_function_execution_timer = ADD_TIMER(profile(), "TableFunctionExecutionTime"); + _filter_timer = ADD_TIMER(profile(), "FilterTime"); auto& p = _parent->cast(); _table_func = std::make_shared(p._tuple_id, p._tuple_desc); _table_func->set_tuple_desc(p._tuple_desc); diff --git a/be/src/pipeline/exec/datagen_operator.h b/be/src/pipeline/exec/datagen_operator.h index c63ef97bb7a40f..bada5ec4080d08 100644 --- a/be/src/pipeline/exec/datagen_operator.h +++ b/be/src/pipeline/exec/datagen_operator.h @@ -44,6 +44,8 @@ class DataGenLocalState final : public PipelineXLocalState<> { private: friend class DataGenSourceOperatorX; std::shared_ptr _table_func; + RuntimeProfile::Counter* _table_function_execution_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; class DataGenSourceOperatorX final : public OperatorX { diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp index f61602f576b107..7cea16ad633c3c 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp @@ -72,7 +72,6 @@ Status DistinctStreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_init_timer); _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); - _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); diff --git a/be/src/pipeline/exec/es_scan_operator.cpp b/be/src/pipeline/exec/es_scan_operator.cpp index c7e953a7fa3201..3759931f9b153a 100644 --- a/be/src/pipeline/exec/es_scan_operator.cpp +++ b/be/src/pipeline/exec/es_scan_operator.cpp @@ -44,12 +44,10 @@ static std::string get_host_and_port(const std::vector& Status EsScanLocalState::_init_profile() { RETURN_IF_ERROR(Base::_init_profile()); - _es_profile.reset(new RuntimeProfile("EsIterator")); - Base::_scanner_profile->add_child(_es_profile.get(), true, nullptr); - _rows_read_counter = ADD_COUNTER(_es_profile, "RowsRead", TUnit::UNIT); - _read_timer = ADD_TIMER(_es_profile, "TotalRawReadTime(*)"); - _materialize_timer = ADD_TIMER(_es_profile, "MaterializeTupleTime(*)"); + _blocks_read_counter = ADD_COUNTER(_runtime_profile, "BlocksRead", TUnit::UNIT); + _read_timer = ADD_TIMER(_runtime_profile, "TotalRawReadTime(*)"); + _materialize_timer = ADD_TIMER(_runtime_profile, "MaterializeTupleTime(*)"); return Status::OK(); } diff --git a/be/src/pipeline/exec/es_scan_operator.h b/be/src/pipeline/exec/es_scan_operator.h index 4e80150d0ba8c6..2ae562e4fc7f32 100644 --- a/be/src/pipeline/exec/es_scan_operator.h +++ b/be/src/pipeline/exec/es_scan_operator.h @@ -52,13 +52,12 @@ class EsScanLocalState final : public ScanLocalState { Status _init_scanners(std::list* scanners) override; std::vector> _scan_ranges; - std::unique_ptr _es_profile; // FIXME: non-static data member '_rows_read_counter' of 'EsScanLocalState' shadows member inherited from type 'ScanLocalStateBase' #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wshadow-field" #endif - RuntimeProfile::Counter* _rows_read_counter = nullptr; + RuntimeProfile::Counter* _blocks_read_counter = nullptr; #ifdef __clang__ #pragma clang diagnostic pop #endif diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index db5c4c78a3129a..98162fc1caed10 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -35,12 +35,6 @@ #include "vec/exprs/vexpr.h" namespace doris::pipeline { - -Status ExchangeSinkLocalState::serialize_block(vectorized::Block* src, PBlock* dest, - int num_receivers) { - return _parent->cast().serialize_block(*this, src, dest, num_receivers); -} - bool ExchangeSinkLocalState::transfer_large_data_by_brpc() const { return _parent->cast()._transfer_large_data_by_brpc; } @@ -58,14 +52,10 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _local_sent_rows = ADD_COUNTER(_profile, "LocalSentRows", TUnit::UNIT); _serialize_batch_timer = ADD_TIMER(_profile, "SerializeBatchTime"); _compress_timer = ADD_TIMER(_profile, "CompressTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); - _brpc_wait_timer = ADD_TIMER(_profile, "BrpcSendTime.Wait"); _local_send_timer = ADD_TIMER(_profile, "LocalSendTime"); _split_block_hash_compute_timer = ADD_TIMER(_profile, "SplitBlockHashComputeTime"); - _split_block_distribute_by_channel_timer = - ADD_TIMER(_profile, "SplitBlockDistributeByChannelTime"); + _distribute_rows_into_channels_timer = ADD_TIMER(_profile, "DistributeRowsIntoChannelsTime"); _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); - _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); _overall_throughput = _profile->add_derived_counter( "OverallThroughput", TUnit::BYTES_PER_SECOND, std::bind(&RuntimeProfile::units_per_second, _bytes_sent_counter, @@ -120,7 +110,7 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { std::mt19937 g(rd()); shuffle(channels.begin(), channels.end(), g); } - int local_size = 0; + size_t local_size = 0; for (int i = 0; i < channels.size(); ++i) { RETURN_IF_ERROR(channels[i]->open(state)); if (channels[i]->is_local()) { @@ -130,6 +120,8 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { } only_local_exchange = local_size == channels.size(); + _rpc_channels_num = channels.size() - local_size; + PUniqueId id; id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); @@ -367,7 +359,6 @@ void ExchangeSinkOperatorX::_handle_eof_channel(RuntimeState* state, ChannelPtrT Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); - COUNTER_UPDATE(local_state.rows_sent_counter(), (int64_t)block->rows()); SCOPED_TIMER(local_state.exec_time_counter()); local_state._peak_memory_usage_counter->set(local_state._mem_tracker->peak_consumption()); bool all_receiver_eof = true; @@ -407,14 +398,15 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); bool serialized = false; RETURN_IF_ERROR(local_state._serializer.next_serialized_block( - block, block_holder->get_block(), local_state.channels.size(), &serialized, - eos)); + block, block_holder->get_block(), local_state._rpc_channels_num, + &serialized, eos)); if (serialized) { auto cur_block = local_state._serializer.get_block()->to_block(); if (!cur_block.empty()) { + DCHECK(eos || local_state._serializer.is_local()) << debug_string(state, 0); RETURN_IF_ERROR(local_state._serializer.serialize_block( &cur_block, block_holder->get_block(), - local_state.channels.size())); + local_state._rpc_channels_num)); } else { block_holder->reset_block(); } @@ -481,15 +473,21 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block local_state._partitioner->do_partitioning(state, block, _mem_tracker.get())); } if (_part_type == TPartitionType::HASH_PARTITIONED) { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); RETURN_IF_ERROR(channel_add_rows( state, local_state.channels, local_state._partition_count, local_state._partitioner->get_channel_ids().get(), rows, block, eos)); } else { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); RETURN_IF_ERROR(channel_add_rows( state, local_state.channel_shared_ptrs, local_state._partition_count, local_state._partitioner->get_channel_ids().get(), rows, block, eos)); } } else if (_part_type == TPartitionType::TABLET_SINK_SHUFFLE_PARTITIONED) { + int64_t old_channel_mem_usage = 0; + for (const auto& channel : local_state.channels) { + old_channel_mem_usage += channel->mem_usage(); + } // check out of limit RETURN_IF_ERROR(local_state._send_new_partition_batch()); std::shared_ptr convert_block = std::make_shared(); @@ -521,10 +519,21 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block local_state._row_distribution._deal_batched = true; RETURN_IF_ERROR(local_state._send_new_partition_batch()); } - // the convert_block maybe different with block after execute exprs - // when send data we still use block - RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, num_channels, - channel2rows, block, eos)); + { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); + // the convert_block maybe different with block after execute exprs + // when send data we still use block + RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, num_channels, + channel2rows, block, eos)); + } + int64_t new_channel_mem_usage = 0; + for (const auto& channel : local_state.channels) { + new_channel_mem_usage += channel->mem_usage(); + } + COUNTER_UPDATE(local_state.memory_used_counter(), + new_channel_mem_usage - old_channel_mem_usage); + COUNTER_SET(local_state.peak_memory_usage_counter(), + local_state.memory_used_counter()->value()); } else if (_part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) { { SCOPED_TIMER(local_state._split_block_hash_compute_timer); @@ -533,8 +542,12 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block } std::vector> assignments = local_state.scale_writer_partitioning_exchanger->accept(block); - RETURN_IF_ERROR(channel_add_rows_with_idx( - state, local_state.channels, local_state.channels.size(), assignments, block, eos)); + { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); + RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, + local_state.channels.size(), assignments, + block, eos)); + } } else if (_part_type == TPartitionType::TABLE_SINK_RANDOM_PARTITIONED) { // Control the number of channels according to the flow, thereby controlling the number of table sink writers. @@ -588,24 +601,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block return final_st; } -Status ExchangeSinkOperatorX::serialize_block(ExchangeSinkLocalState& state, vectorized::Block* src, - PBlock* dest, int num_receivers) { - { - SCOPED_TIMER(state.serialize_batch_timer()); - dest->Clear(); - size_t uncompressed_bytes = 0; - size_t compressed_bytes = 0; - RETURN_IF_ERROR(src->serialize(_state->be_exec_version(), dest, &uncompressed_bytes, - &compressed_bytes, _compression_type, - _transfer_large_data_by_brpc)); - COUNTER_UPDATE(state.bytes_sent_counter(), compressed_bytes * num_receivers); - COUNTER_UPDATE(state.uncompressed_bytes_counter(), uncompressed_bytes * num_receivers); - COUNTER_UPDATE(state.compress_timer(), src->get_compress_time()); - } - - return Status::OK(); -} - void ExchangeSinkLocalState::register_channels(pipeline::ExchangeSinkBuffer* buffer) { for (auto channel : channels) { ((vectorized::PipChannel*)channel)->register_exchange_buffer(buffer); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 3bd905fdde6fd4..a4f78bdf61c69e 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -78,27 +78,13 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; Dependency* finishdependency() override { return _finish_dependency.get(); } - Status serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers = 1); void register_channels(pipeline::ExchangeSinkBuffer* buffer); - RuntimeProfile::Counter* brpc_wait_timer() { return _brpc_wait_timer; } RuntimeProfile::Counter* blocks_sent_counter() { return _blocks_sent_counter; } - RuntimeProfile::Counter* rows_sent_counter() { return _rows_sent_counter; } RuntimeProfile::Counter* local_send_timer() { return _local_send_timer; } RuntimeProfile::Counter* local_bytes_send_counter() { return _local_bytes_send_counter; } RuntimeProfile::Counter* local_sent_rows() { return _local_sent_rows; } - RuntimeProfile::Counter* brpc_send_timer() { return _brpc_send_timer; } - RuntimeProfile::Counter* serialize_batch_timer() { return _serialize_batch_timer; } - RuntimeProfile::Counter* split_block_distribute_by_channel_timer() { - return _split_block_distribute_by_channel_timer; - } - RuntimeProfile::Counter* bytes_sent_counter() { return _bytes_sent_counter; } - RuntimeProfile::Counter* split_block_hash_compute_timer() { - return _split_block_hash_compute_timer; - } RuntimeProfile::Counter* merge_block_timer() { return _merge_block_timer; } - RuntimeProfile::Counter* compress_timer() { return _compress_timer; } - RuntimeProfile::Counter* uncompressed_bytes_counter() { return _uncompressed_bytes_counter; } [[nodiscard]] bool transfer_large_data_by_brpc() const; bool is_finished() const override { return _reach_limit.load(); } void set_reach_limit() { _reach_limit = true; }; @@ -130,16 +116,13 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { std::unique_ptr _sink_buffer = nullptr; RuntimeProfile::Counter* _serialize_batch_timer = nullptr; RuntimeProfile::Counter* _compress_timer = nullptr; - RuntimeProfile::Counter* _brpc_send_timer = nullptr; - RuntimeProfile::Counter* _brpc_wait_timer = nullptr; RuntimeProfile::Counter* _bytes_sent_counter = nullptr; RuntimeProfile::Counter* _uncompressed_bytes_counter = nullptr; RuntimeProfile::Counter* _local_sent_rows = nullptr; RuntimeProfile::Counter* _local_send_timer = nullptr; RuntimeProfile::Counter* _split_block_hash_compute_timer = nullptr; - RuntimeProfile::Counter* _split_block_distribute_by_channel_timer = nullptr; + RuntimeProfile::Counter* _distribute_rows_into_channels_timer = nullptr; RuntimeProfile::Counter* _blocks_sent_counter = nullptr; - RuntimeProfile::Counter* _rows_sent_counter = nullptr; // Throughput per total time spent in sender RuntimeProfile::Counter* _overall_throughput = nullptr; // Used to counter send bytes under local data exchange @@ -154,6 +137,7 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { int _sender_id; std::shared_ptr _broadcast_pb_mem_limiter; + size_t _rpc_channels_num = 0; vectorized::BlockSerializer _serializer; std::shared_ptr _queue_dependency = nullptr; @@ -218,8 +202,6 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorXcreate_merger( local_state.vsort_exec_exprs.lhs_ordering_expr_ctxs(), _is_asc_order, _nulls_first, state->batch_size(), _limit, _offset)); local_state.is_ready = true; return Status::OK(); } - auto status = local_state.stream_recvr->get_next(block, eos); - RETURN_IF_ERROR(doris::vectorized::VExprContext::filter_block(local_state.conjuncts(), block, - block->columns())); + { + SCOPED_TIMER(local_state.get_data_from_recvr_timer); + RETURN_IF_ERROR(local_state.stream_recvr->get_next(block, eos)); + } + { + SCOPED_TIMER(local_state.filter_timer); + RETURN_IF_ERROR(doris::vectorized::VExprContext::filter_block(local_state.conjuncts(), + block, block->columns())); + } // In vsortrunmerger, it will set eos=true, and block not empty // so that eos==true, could not make sure that block not have valid data if (!*eos || block->rows() > 0) { @@ -176,7 +187,7 @@ Status ExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block COUNTER_SET(local_state.rows_returned_counter(), local_state.num_rows_returned()); COUNTER_UPDATE(local_state.blocks_returned_counter(), 1); } - return status; + return Status::OK(); } Status ExchangeLocalState::close(RuntimeState* state) { diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index c8ef674d269853..f938f5007d1643 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -59,6 +59,9 @@ class ExchangeLocalState final : public PipelineXLocalState<> { std::vector> deps; std::vector metrics; + RuntimeProfile::Counter* get_data_from_recvr_timer = nullptr; + RuntimeProfile::Counter* filter_timer = nullptr; + RuntimeProfile::Counter* create_merger_timer = nullptr; }; class ExchangeSourceOperatorX final : public OperatorX { diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp index 6db49bb7ab1089..8da335f4fa2c0e 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp @@ -66,6 +66,7 @@ Status GroupCommitBlockSinkLocalState::open(RuntimeState* state) { } Status GroupCommitBlockSinkLocalState::_initialize_load_queue() { + SCOPED_TIMER(_init_load_queue_timer); auto& p = _parent->cast(); if (_state->exec_env()->wal_mgr()->is_running()) { RETURN_IF_ERROR(_state->exec_env()->group_commit_mgr()->get_first_block_load_queue( @@ -240,6 +241,17 @@ Status GroupCommitBlockSinkLocalState::_add_blocks(RuntimeState* state, return Status::OK(); } +Status GroupCommitBlockSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _init_load_queue_timer = ADD_TIMER(_profile, "InitLoadQueueTime"); + _valid_and_convert_block_timer = ADD_TIMER(_profile, "ValidAndConvertBlockTime"); + _find_partition_timer = ADD_TIMER(_profile, "FindPartitionTime"); + _append_blocks_timer = ADD_TIMER(_profile, "AppendBlocksTime"); + return Status::OK(); +} + Status GroupCommitBlockSinkOperatorX::init(const TDataSink& t_sink) { RETURN_IF_ERROR(Base::init(t_sink)); DCHECK(t_sink.__isset.olap_table_sink); @@ -321,10 +333,15 @@ Status GroupCommitBlockSinkOperatorX::sink(RuntimeState* state, vectorized::Bloc std::shared_ptr block; bool has_filtered_rows = false; - RETURN_IF_ERROR(local_state._block_convertor->validate_and_convert_block( - state, input_block, block, local_state._output_vexpr_ctxs, rows, has_filtered_rows)); + { + SCOPED_TIMER(local_state._valid_and_convert_block_timer); + RETURN_IF_ERROR(local_state._block_convertor->validate_and_convert_block( + state, input_block, block, local_state._output_vexpr_ctxs, rows, + has_filtered_rows)); + } local_state._has_filtered_rows = false; if (!local_state._vpartition->is_auto_partition()) { + SCOPED_TIMER(local_state._find_partition_timer); //reuse vars for find_partition local_state._partitions.assign(rows, nullptr); local_state._filter_bitmap.Reset(rows); @@ -354,23 +371,26 @@ Status GroupCommitBlockSinkOperatorX::sink(RuntimeState* state, vectorized::Bloc } } } - - if (local_state._block_convertor->num_filtered_rows() > 0 || local_state._has_filtered_rows) { - auto cloneBlock = block->clone_without_columns(); - auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock); - for (int i = 0; i < rows; ++i) { - if (local_state._block_convertor->filter_map()[i]) { - continue; - } - if (local_state._filter_bitmap.Get(i)) { - continue; + { + SCOPED_TIMER(local_state._append_blocks_timer); + if (local_state._block_convertor->num_filtered_rows() > 0 || + local_state._has_filtered_rows) { + auto cloneBlock = block->clone_without_columns(); + auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock); + for (int i = 0; i < rows; ++i) { + if (local_state._block_convertor->filter_map()[i]) { + continue; + } + if (local_state._filter_bitmap.Get(i)) { + continue; + } + res_block.add_row(block.get(), i); } - res_block.add_row(block.get(), i); + block->swap(res_block.to_block()); } - block->swap(res_block.to_block()); + // add block into block queue + RETURN_IF_ERROR(local_state._add_block(state, block)); } - // add block into block queue - RETURN_IF_ERROR(local_state._add_block(state, block)); return wind_up(); } diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.h b/be/src/pipeline/exec/group_commit_block_sink_operator.h index 32ca0613652ae4..e469aee8df595c 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.h +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.h @@ -42,8 +42,8 @@ class GroupCommitBlockSinkLocalState final : public PipelineXSinkLocalState dependencies() const override { @@ -79,6 +79,11 @@ class GroupCommitBlockSinkLocalState final : public PipelineXSinkLocalState _finish_dependency; std::shared_ptr _create_plan_dependency = nullptr; std::shared_ptr _put_block_dependency = nullptr; + + RuntimeProfile::Counter* _init_load_queue_timer = nullptr; + RuntimeProfile::Counter* _valid_and_convert_block_timer = nullptr; + RuntimeProfile::Counter* _find_partition_timer = nullptr; + RuntimeProfile::Counter* _append_blocks_timer = nullptr; }; class GroupCommitBlockSinkOperatorX final diff --git a/be/src/pipeline/exec/group_commit_scan_operator.cpp b/be/src/pipeline/exec/group_commit_scan_operator.cpp index 3e6ad62c5dcb7c..fbe7f3c6f22836 100644 --- a/be/src/pipeline/exec/group_commit_scan_operator.cpp +++ b/be/src/pipeline/exec/group_commit_scan_operator.cpp @@ -31,6 +31,7 @@ GroupCommitOperatorX::GroupCommitOperatorX(ObjectPool* pool, const TPlanNode& tn Status GroupCommitOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); bool find_node = false; while (!find_node && !*eos) { RETURN_IF_ERROR(local_state.load_block_queue->get_block(state, block, &find_node, eos, diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 5be3fcad112db5..7efeb7692d4b71 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -51,19 +51,19 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _shared_state->build_exprs_size = _build_expr_ctxs.size(); _should_build_hash_table = true; + profile()->add_info_string("BroadcastJoin", std::to_string(p._is_broadcast_join)); if (p._is_broadcast_join) { - profile()->add_info_string("BroadcastJoin", "true"); if (state->enable_share_hash_table_for_broadcast_join()) { _should_build_hash_table = info.task_idx == 0; if (_should_build_hash_table) { - profile()->add_info_string("ShareHashTableEnabled", "true"); p._shared_hashtable_controller->set_builder_and_consumers( state->fragment_instance_id(), p.node_id()); } - } else { - profile()->add_info_string("ShareHashTableEnabled", "false"); } } + profile()->add_info_string("BuildShareHashTable", std::to_string(_should_build_hash_table)); + profile()->add_info_string("ShareHashTableEnabled", + std::to_string(state->enable_share_hash_table_for_broadcast_join())); if (!_should_build_hash_table) { _dependency->block(); _finish_dependency->block(); @@ -72,6 +72,7 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _finish_dependency->shared_from_this()); } + _runtime_filter_init_timer = ADD_TIMER(profile(), "RuntimeFilterInitTime"); _build_blocks_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage", 1); _hash_table_memory_usage = @@ -81,13 +82,10 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo // Build phase auto* record_profile = _should_build_hash_table ? profile() : faker_runtime_profile(); - _build_table_timer = ADD_TIMER(profile(), "BuildTableTime"); - _build_side_merge_block_timer = ADD_TIMER(profile(), "BuildSideMergeBlockTime"); + _build_table_timer = ADD_TIMER(profile(), "BuildHashTableTime"); + _build_side_merge_block_timer = ADD_TIMER(profile(), "MergeBuildBlockTime"); _build_table_insert_timer = ADD_TIMER(record_profile, "BuildTableInsertTime"); _build_expr_call_timer = ADD_TIMER(record_profile, "BuildExprCallTime"); - _build_side_compute_hash_timer = ADD_TIMER(record_profile, "BuildSideHashComputingTime"); - - _allocate_resource_timer = ADD_TIMER(profile(), "AllocateResourceTime"); // Hash Table Init _hash_table_init(state); @@ -253,7 +251,6 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, if (UNLIKELY(rows == 0)) { return Status::OK(); } - COUNTER_UPDATE(_build_rows_counter, rows); block.replace_if_overflow(); vectorized::ColumnRawPtrs raw_ptrs(_build_expr_ctxs.size()); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index c56fb9bc9b1cf7..930d3761791d65 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -96,14 +96,12 @@ class HashJoinBuildSinkLocalState final RuntimeProfile::Counter* _build_table_timer = nullptr; RuntimeProfile::Counter* _build_expr_call_timer = nullptr; RuntimeProfile::Counter* _build_table_insert_timer = nullptr; - RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; - RuntimeProfile::Counter* _allocate_resource_timer = nullptr; - RuntimeProfile::Counter* _build_blocks_memory_usage = nullptr; RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; - RuntimeProfile::HighWaterMarkCounter* _build_arena_memory_usage = nullptr; + RuntimeProfile::Counter* _build_arena_memory_usage = nullptr; + RuntimeProfile::Counter* _runtime_filter_init_timer = nullptr; }; class HashJoinBuildSinkOperatorX final diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index f91e1eaa2a1b17..756a151394b41e 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -56,13 +56,11 @@ Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) _probe_arena_memory_usage = profile()->AddHighWaterMarkCounter("ProbeKeyArena", TUnit::BYTES, "MemoryUsage", 1); // Probe phase - _probe_next_timer = ADD_TIMER(profile(), "ProbeFindNextTime"); _probe_expr_call_timer = ADD_TIMER(profile(), "ProbeExprCallTime"); _search_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenSearchHashTableTime"); _build_side_output_timer = ADD_TIMER(profile(), "ProbeWhenBuildSideOutputTime"); _probe_side_output_timer = ADD_TIMER(profile(), "ProbeWhenProbeSideOutputTime"); - _probe_process_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenProcessHashTableTime"); - _process_other_join_conjunct_timer = ADD_TIMER(profile(), "OtherJoinConjunctTime"); + _non_equal_join_conjuncts_timer = ADD_TIMER(profile(), "NonEqualJoinConjunctEvaluationTime"); _init_probe_side_timer = ADD_TIMER(profile(), "InitProbeSideTime"); return Status::OK(); } @@ -230,7 +228,6 @@ HashJoinProbeOperatorX::HashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) const { auto& local_state = get_local_state(state); - SCOPED_TIMER(local_state._probe_timer); if (local_state._shared_state->short_circuit_for_probe) { // If we use a short-circuit strategy, should return empty block directly. *eos = true; @@ -325,7 +322,7 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc if constexpr (!std::is_same_v) { using HashTableCtxType = std::decay_t; if constexpr (!std::is_same_v) { - st = process_hashtable_ctx.process_data_in_hashtable( + st = process_hashtable_ctx.finish_probing( arg, mutable_join_block, &temp_block, eos, _is_mark_join); } else { st = Status::InternalError("uninited hash table"); diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index ad7e0d284cb385..66d709e6541ad8 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -117,14 +117,12 @@ class HashJoinProbeLocalState final std::make_unique(); RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; - RuntimeProfile::Counter* _probe_next_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; - RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; RuntimeProfile::Counter* _search_hashtable_timer = nullptr; RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; - RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; + RuntimeProfile::Counter* _non_equal_join_conjuncts_timer = nullptr; }; class HashJoinProbeOperatorX final : public JoinProbeOperatorX { diff --git a/be/src/pipeline/exec/jdbc_table_sink_operator.cpp b/be/src/pipeline/exec/jdbc_table_sink_operator.cpp index dba9f6259ff38d..ba017b4a87aa74 100644 --- a/be/src/pipeline/exec/jdbc_table_sink_operator.cpp +++ b/be/src/pipeline/exec/jdbc_table_sink_operator.cpp @@ -47,6 +47,7 @@ Status JdbcTableSinkOperatorX::open(RuntimeState* state) { Status JdbcTableSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); RETURN_IF_ERROR(local_state.sink(state, block, eos)); return Status::OK(); } diff --git a/be/src/pipeline/exec/join/process_hash_table_probe.h b/be/src/pipeline/exec/join/process_hash_table_probe.h index 965d62192b2fed..2ccc9aec8c7e01 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe.h @@ -87,9 +87,8 @@ struct ProcessHashTableProbe { // Process full outer join/ right join / right semi/anti join to output the join result // in hash table template - Status process_data_in_hashtable(HashTableType& hash_table_ctx, - vectorized::MutableBlock& mutable_block, - vectorized::Block* output_block, bool* eos, bool is_mark_join); + Status finish_probing(HashTableType& hash_table_ctx, vectorized::MutableBlock& mutable_block, + vectorized::Block* output_block, bool* eos, bool is_mark_join); /// For null aware join with other conjuncts, if the probe key of one row on left side is null, /// we should make this row match with all rows in build side. @@ -137,7 +136,7 @@ struct ProcessHashTableProbe { RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; - RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; + RuntimeProfile::Counter* _finish_probe_phase_timer = nullptr; int _right_col_idx; int _right_col_len; diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 653cc8ab4473dd..5de033b63e8aad 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -56,7 +56,7 @@ ProcessHashTableProbe::ProcessHashTableProbe(HashJoinProbeLocalState _init_probe_side_timer(parent->_init_probe_side_timer), _build_side_output_timer(parent->_build_side_output_timer), _probe_side_output_timer(parent->_probe_side_output_timer), - _probe_process_hashtable_timer(parent->_probe_process_hashtable_timer), + _finish_probe_phase_timer(parent->_finish_probe_phase_timer), _right_col_idx((_is_right_semi_anti && !_have_other_join_conjunct) ? 0 : _parent->left_table_data_types().size()), @@ -501,8 +501,8 @@ Status ProcessHashTableProbe::do_other_join_conjuncts(vectorized::Bl return Status::OK(); } - SCOPED_TIMER(_parent->_process_other_join_conjunct_timer); - int orig_columns = output_block->columns(); + SCOPED_TIMER(_parent->_non_equal_join_conjuncts_timer); + size_t orig_columns = output_block->columns(); vectorized::IColumn::Filter other_conjunct_filter(row_count, 1); { bool can_be_filter_all = false; @@ -616,10 +616,11 @@ Status ProcessHashTableProbe::do_other_join_conjuncts(vectorized::Bl template template -Status ProcessHashTableProbe::process_data_in_hashtable( - HashTableType& hash_table_ctx, vectorized::MutableBlock& mutable_block, - vectorized::Block* output_block, bool* eos, bool is_mark_join) { - SCOPED_TIMER(_probe_process_hashtable_timer); +Status ProcessHashTableProbe::finish_probing(HashTableType& hash_table_ctx, + vectorized::MutableBlock& mutable_block, + vectorized::Block* output_block, bool* eos, + bool is_mark_join) { + SCOPED_TIMER(_finish_probe_phase_timer); auto& mcol = mutable_block.mutable_columns(); if (is_mark_join) { std::unique_ptr mark_column = @@ -717,8 +718,7 @@ struct ExtractType { vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ size_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ \ - template Status \ - ProcessHashTableProbe::process_data_in_hashtable::Type>( \ + template Status ProcessHashTableProbe::finish_probing::Type>( \ ExtractType::Type & hash_table_ctx, vectorized::MutableBlock & mutable_block, \ vectorized::Block * output_block, bool* eos, bool is_mark_join); diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp index a6f5f0f650a4bf..1dcd5099283f62 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.cpp +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -33,15 +33,11 @@ Status JoinBuildSinkLocalState::init(RuntimeState* stat PipelineXSinkLocalState::profile()->add_info_string("JoinType", to_string(p._join_op)); - _build_rows_counter = ADD_COUNTER(PipelineXSinkLocalState::profile(), - "BuildRows", TUnit::UNIT); _publish_runtime_filter_timer = ADD_TIMER(PipelineXSinkLocalState::profile(), "PublishRuntimeFilterTime"); - _runtime_filter_compute_timer = ADD_TIMER(PipelineXSinkLocalState::profile(), - "RuntimeFilterComputeTime"); - _runtime_filter_init_timer = - ADD_TIMER(PipelineXSinkLocalState::profile(), "RuntimeFilterInitTime"); + _runtime_filter_compute_timer = + ADD_TIMER(PipelineXSinkLocalState::profile(), "BuildRuntimeFilterTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h index 714e0c34190678..9d79a97397ff77 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.h +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -39,10 +39,8 @@ class JoinBuildSinkLocalState : public PipelineXSinkLocalState template friend class JoinBuildSinkOperatorX; - RuntimeProfile::Counter* _build_rows_counter = nullptr; RuntimeProfile::Counter* _publish_runtime_filter_timer = nullptr; RuntimeProfile::Counter* _runtime_filter_compute_timer = nullptr; - RuntimeProfile::Counter* _runtime_filter_init_timer = nullptr; std::vector> _runtime_filters; }; diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 53dcb2c4cfce0e..cc20fa744313f4 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -29,11 +29,10 @@ Status JoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); - _probe_timer = ADD_TIMER(Base::profile(), "ProbeTime"); _join_filter_timer = ADD_TIMER(Base::profile(), "JoinFilterTimer"); _build_output_block_timer = ADD_TIMER(Base::profile(), "BuildOutputBlock"); _probe_rows_counter = ADD_COUNTER_WITH_LEVEL(Base::profile(), "ProbeRows", TUnit::UNIT, 1); - + _finish_probe_phase_timer = ADD_TIMER(Base::profile(), "FinishProbePhaseTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index 3f68c73d04b161..078806cea4fc5a 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -49,10 +49,10 @@ class JoinProbeLocalState : public PipelineXLocalState { size_t _mark_column_id = -1; - RuntimeProfile::Counter* _probe_timer = nullptr; RuntimeProfile::Counter* _probe_rows_counter = nullptr; RuntimeProfile::Counter* _join_filter_timer = nullptr; RuntimeProfile::Counter* _build_output_block_timer = nullptr; + RuntimeProfile::Counter* _finish_probe_phase_timer = nullptr; std::unique_ptr _child_block = nullptr; bool _child_eos = false; diff --git a/be/src/pipeline/exec/memory_scratch_sink_operator.cpp b/be/src/pipeline/exec/memory_scratch_sink_operator.cpp index 131f3caf42c6db..b9f18c43e1e239 100644 --- a/be/src/pipeline/exec/memory_scratch_sink_operator.cpp +++ b/be/src/pipeline/exec/memory_scratch_sink_operator.cpp @@ -33,6 +33,9 @@ Status MemoryScratchSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _get_arrow_schema_timer = ADD_TIMER(_profile, "GetArrowSchemaTime"); + _convert_block_to_arrow_batch_timer = ADD_TIMER(_profile, "ConvertBlockToArrowBatchTime"); + _evaluation_timer = ADD_TIMER(_profile, "EvaluationTime"); // create queue state->exec_env()->result_queue_mgr()->create_queue(state->fragment_instance_id(), &_queue); @@ -92,13 +95,22 @@ Status MemoryScratchSinkOperatorX::sink(RuntimeState* state, vectorized::Block* // Exec vectorized expr here to speed up, block.rows() == 0 means expr exec // failed, just return the error status vectorized::Block block; - RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( - local_state._output_vexpr_ctxs, *input_block, &block)); + { + SCOPED_TIMER(local_state._evaluation_timer); + RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( + local_state._output_vexpr_ctxs, *input_block, &block)); + } std::shared_ptr block_arrow_schema; - // After expr executed, use recaculated schema as final schema - RETURN_IF_ERROR(convert_block_arrow_schema(block, &block_arrow_schema)); - RETURN_IF_ERROR(convert_to_arrow_batch(block, block_arrow_schema, arrow::default_memory_pool(), - &result, _timezone_obj)); + { + SCOPED_TIMER(local_state._get_arrow_schema_timer); + // After expr executed, use recaculated schema as final schema + RETURN_IF_ERROR(get_arrow_schema(block, &block_arrow_schema)); + } + { + SCOPED_TIMER(local_state._convert_block_to_arrow_batch_timer); + RETURN_IF_ERROR(convert_to_arrow_batch( + block, block_arrow_schema, arrow::default_memory_pool(), &result, _timezone_obj)); + } local_state._queue->blocking_put(result); if (local_state._queue->size() > config::max_memory_sink_batch_count) { local_state._queue_dependency->block(); diff --git a/be/src/pipeline/exec/memory_scratch_sink_operator.h b/be/src/pipeline/exec/memory_scratch_sink_operator.h index c2cd78c7cd5aee..75372500d9bffc 100644 --- a/be/src/pipeline/exec/memory_scratch_sink_operator.h +++ b/be/src/pipeline/exec/memory_scratch_sink_operator.h @@ -45,6 +45,9 @@ class MemoryScratchSinkLocalState final : public PipelineXSinkLocalState _queue_dependency = nullptr; + RuntimeProfile::Counter* _get_arrow_schema_timer = nullptr; + RuntimeProfile::Counter* _convert_block_to_arrow_batch_timer = nullptr; + RuntimeProfile::Counter* _evaluation_timer = nullptr; }; class MemoryScratchSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index 1028bca7ce2ca4..304e8e96f0c79c 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -40,6 +40,9 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState auto& p = _parent->cast(); _shared_state->multi_cast_data_streamer->set_dep_by_sender_idx(p._consumer_id, _dependency); _wait_for_rf_timer = ADD_TIMER(_runtime_profile, "WaitForRuntimeFilter"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); + _get_data_timer = ADD_TIMER(_runtime_profile, "GetDataTime"); + _materialize_data_timer = ADD_TIMER(_runtime_profile, "MaterializeDataTime"); // init profile for runtime filter RuntimeFilterConsumer::_init_profile(profile()); init_runtime_filter_dependency(_filter_dependencies, p.operator_id(), p.node_id(), @@ -86,15 +89,19 @@ Status MultiCastDataStreamerSourceOperatorX::get_block(RuntimeState* state, if (!local_state._output_expr_contexts.empty()) { output_block = &tmp_block; } - RETURN_IF_ERROR(local_state._shared_state->multi_cast_data_streamer->pull(_consumer_id, - output_block, eos)); - + { + SCOPED_TIMER(local_state._get_data_timer); + RETURN_IF_ERROR(local_state._shared_state->multi_cast_data_streamer->pull( + _consumer_id, output_block, eos)); + } if (!local_state._conjuncts.empty()) { + SCOPED_TIMER(local_state._filter_timer); RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, output_block->columns())); } if (!local_state._output_expr_contexts.empty() && output_block->rows() > 0) { + SCOPED_TIMER(local_state._materialize_data_timer); RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( local_state._output_expr_contexts, *output_block, block, true)); vectorized::materialize_block_inplace(*block); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index 76472f3ce85e83..b37c4f7e3a8e05 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -67,6 +67,9 @@ class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState> _filter_dependencies; RuntimeProfile::Counter* _wait_for_rf_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _get_data_timer = nullptr; + RuntimeProfile::Counter* _materialize_data_timer = nullptr; }; class MultiCastDataStreamerSourceOperatorX final diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp index 793a37c7396a61..6c1644178228c3 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp @@ -139,7 +139,6 @@ Status NestedLoopJoinBuildSinkOperatorX::sink(doris::RuntimeState* state, vector } if (eos) { - COUNTER_UPDATE(local_state._build_rows_counter, local_state._build_rows); RuntimeFilterBuild rf_ctx(&local_state); RETURN_IF_ERROR(rf_ctx(state)); diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp index 9546ed8df56671..51b3aed079b3b9 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -42,6 +42,10 @@ Status NestedLoopJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _loop_join_timer = ADD_TIMER(profile(), "LoopGenerateJoin"); + _output_temp_blocks_timer = ADD_TIMER(profile(), "OutputTempBlocksTime"); + _update_visited_flags_timer = ADD_TIMER(profile(), "UpdateVisitedFlagsTime"); + _join_conjuncts_evaluation_timer = ADD_TIMER(profile(), "JoinConjunctsEvaluationTime"); + _filtered_by_join_conjuncts_timer = ADD_TIMER(profile(), "FilteredByJoinConjunctsTime"); return Status::OK(); } @@ -164,23 +168,26 @@ Status NestedLoopJoinProbeLocalState::generate_join_block_data(RuntimeState* sta _process_left_child_block(_join_block, now_process_build_block); } - if constexpr (set_probe_side_flag) { - RETURN_IF_ERROR( - (_do_filtering_and_update_visited_flags( - &_join_block, !p._is_left_semi_anti))); - _update_additional_flags(&_join_block); - // If this join operation is left outer join or full outer join, when - // `_left_side_process_count`, means all rows from build - // side have been joined with _left_side_process_count, we should output current - // probe row with null from build side. - if (_left_side_process_count) { - _finalize_current_phase( - _join_block, state->batch_size()); + { + SCOPED_TIMER(_finish_probe_phase_timer); + if constexpr (set_probe_side_flag) { + RETURN_IF_ERROR( + (_do_filtering_and_update_visited_flags( + &_join_block, !p._is_left_semi_anti))); + _update_additional_flags(&_join_block); + // If this join operation is left outer join or full outer join, when + // `_left_side_process_count`, means all rows from build + // side have been joined with _left_side_process_count, we should output current + // probe row with null from build side. + if (_left_side_process_count) { + _finalize_current_phase( + _join_block, state->batch_size()); + } + } else if (_left_side_process_count && p._is_mark_join && + _shared_state->build_blocks.empty()) { + _append_left_data_with_null(_join_block); } - } else if (_left_side_process_count && p._is_mark_join && - _shared_state->build_blocks.empty()) { - _append_left_data_with_null(_join_block); } } @@ -373,6 +380,7 @@ void NestedLoopJoinProbeLocalState::_append_left_data_with_null(vectorized::Bloc void NestedLoopJoinProbeLocalState::_process_left_child_block( vectorized::Block& block, const vectorized::Block& now_process_build_block) const { + SCOPED_TIMER(_output_temp_blocks_timer); auto& p = _parent->cast(); auto dst_columns = block.mutate_columns(); const int max_added_rows = now_process_build_block.rows(); @@ -480,6 +488,7 @@ Status NestedLoopJoinProbeOperatorX::push(doris::RuntimeState* state, vectorized set_build_side_flag, set_probe_side_flag>( state, join_op_variants); }; + SCOPED_TIMER(local_state._loop_join_timer); RETURN_IF_ERROR( std::visit(func, local_state._shared_state->join_op_variants, vectorized::make_bool_variant(_match_all_build || _is_right_semi_anti), diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index 982498c7e2e517..0d1d6510ce61a8 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -63,42 +63,47 @@ class NestedLoopJoinProbeLocalState final void _do_filtering_and_update_visited_flags_impl(vectorized::Block* block, int column_to_keep, int build_block_idx, int processed_blocks_num, bool materialize, Filter& filter) { - if constexpr (SetBuildSideFlag) { - for (size_t i = 0; i < processed_blocks_num; i++) { - auto& build_side_flag = - assert_cast( - _shared_state->build_side_visited_flags[build_block_idx].get()) - ->get_data(); - auto* __restrict build_side_flag_data = build_side_flag.data(); - auto cur_sz = build_side_flag.size(); - const size_t offset = _build_offset_stack.top(); - _build_offset_stack.pop(); - for (size_t j = 0; j < cur_sz; j++) { - build_side_flag_data[j] |= filter[offset + j]; + { + SCOPED_TIMER(_update_visited_flags_timer); + if constexpr (SetBuildSideFlag) { + for (size_t i = 0; i < processed_blocks_num; i++) { + auto& build_side_flag = + assert_cast( + _shared_state->build_side_visited_flags[build_block_idx].get()) + ->get_data(); + auto* __restrict build_side_flag_data = build_side_flag.data(); + auto cur_sz = build_side_flag.size(); + const size_t offset = _build_offset_stack.top(); + _build_offset_stack.pop(); + for (size_t j = 0; j < cur_sz; j++) { + build_side_flag_data[j] |= filter[offset + j]; + } + build_block_idx = build_block_idx == 0 ? _shared_state->build_blocks.size() - 1 + : build_block_idx - 1; } - build_block_idx = build_block_idx == 0 ? _shared_state->build_blocks.size() - 1 - : build_block_idx - 1; } - } - if constexpr (SetProbeSideFlag) { - int end = filter.size(); - for (int i = _left_block_pos == _child_block->rows() ? _left_block_pos - 1 - : _left_block_pos; - i >= _left_block_start_pos; i--) { - int offset = 0; - if (!_probe_offset_stack.empty()) { - offset = _probe_offset_stack.top(); - _probe_offset_stack.pop(); - } - if (!_cur_probe_row_visited_flags[i]) { - _cur_probe_row_visited_flags[i] = - simd::contain_byte(filter.data() + offset, end - offset, 1) ? 1 - : 0; + if constexpr (SetProbeSideFlag) { + int end = filter.size(); + for (int i = _left_block_pos == _child_block->rows() ? _left_block_pos - 1 + : _left_block_pos; + i >= _left_block_start_pos; i--) { + int offset = 0; + if (!_probe_offset_stack.empty()) { + offset = _probe_offset_stack.top(); + _probe_offset_stack.pop(); + } + if (!_cur_probe_row_visited_flags[i]) { + _cur_probe_row_visited_flags[i] = + simd::contain_byte(filter.data() + offset, end - offset, 1) + ? 1 + : 0; + } + end = offset; } - end = offset; } } if (materialize) { + SCOPED_TIMER(_filtered_by_join_conjuncts_timer); vectorized::Block::filter_block_internal(block, filter, column_to_keep); } else { CLEAR_BLOCK @@ -119,8 +124,11 @@ class NestedLoopJoinProbeLocalState final if (LIKELY(!_join_conjuncts.empty() && block->rows() > 0)) { vectorized::IColumn::Filter filter(block->rows(), 1); bool can_filter_all = false; - RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts( - _join_conjuncts, nullptr, IgnoreNull, block, &filter, &can_filter_all)); + { + SCOPED_TIMER(_join_conjuncts_evaluation_timer); + RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts( + _join_conjuncts, nullptr, IgnoreNull, block, &filter, &can_filter_all)); + } if (can_filter_all) { CLEAR_BLOCK @@ -179,6 +187,10 @@ class NestedLoopJoinProbeLocalState final vectorized::VExprContextSPtrs _join_conjuncts; RuntimeProfile::Counter* _loop_join_timer = nullptr; + RuntimeProfile::Counter* _output_temp_blocks_timer = nullptr; + RuntimeProfile::Counter* _update_visited_flags_timer = nullptr; + RuntimeProfile::Counter* _join_conjuncts_evaluation_timer = nullptr; + RuntimeProfile::Counter* _filtered_by_join_conjuncts_timer = nullptr; }; class NestedLoopJoinProbeOperatorX final diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 75e6153af1c724..b27402ac27e7a8 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -43,6 +43,9 @@ namespace doris::pipeline { Status OlapScanLocalState::_init_profile() { RETURN_IF_ERROR(ScanLocalState::_init_profile()); + // Rows read from storage. + // Include the rows read from doris page cache. + _scan_rows = ADD_COUNTER(_runtime_profile, "ScanRows", TUnit::UNIT); // 1. init segment profile _segment_profile.reset(new RuntimeProfile("SegmentIterator")); _scanner_profile->add_child(_segment_profile.get(), true, nullptr); @@ -58,22 +61,20 @@ Status OlapScanLocalState::_init_profile() { _block_load_counter = ADD_COUNTER(_segment_profile, "BlocksLoad", TUnit::UNIT); _block_fetch_timer = ADD_TIMER(_scanner_profile, "BlockFetchTime"); _delete_bitmap_get_agg_timer = ADD_TIMER(_scanner_profile, "DeleteBitmapGetAggTime"); - _sync_rowset_timer = ADD_TIMER(_scanner_profile, "SyncRowsetTime"); - _block_convert_timer = ADD_TIMER(_scanner_profile, "BlockConvertTime"); + if (config::is_cloud_mode()) { + _sync_rowset_timer = ADD_TIMER(_scanner_profile, "SyncRowsetTime"); + } _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); - _block_conditions_filtered_timer = ADD_TIMER(_segment_profile, "BlockConditionsFilteredTime"); - _block_conditions_filtered_bf_timer = - ADD_TIMER(_segment_profile, "BlockConditionsFilteredBloomFilterTime"); + _segment_generate_row_range_timer = ADD_TIMER(_segment_profile, "GenerateRowRangeTime"); + _segment_generate_row_range_by_bf_timer = + ADD_TIMER(_segment_profile, "GenerateRowRangeByBloomFilterIndexTime"); _collect_iterator_merge_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorMergeTime"); - _collect_iterator_normal_next_timer = ADD_TIMER(_segment_profile, "CollectIteratorNormalTime"); - _block_conditions_filtered_zonemap_timer = - ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapTime"); - _block_conditions_filtered_zonemap_rp_timer = - ADD_TIMER(_segment_profile, "BlockConditionsFilteredZonemapRuntimePredicateTime"); - _block_conditions_filtered_dict_timer = - ADD_TIMER(_segment_profile, "BlockConditionsFilteredDictTime"); + _segment_generate_row_range_by_zonemap_timer = + ADD_TIMER(_segment_profile, "GenerateRowRangeByZoneMapIndexTime"); + _segment_generate_row_range_by_dict_timer = + ADD_TIMER(_segment_profile, "GenerateRowRangeByDictTime"); _rows_vec_cond_filtered_counter = ADD_COUNTER(_segment_profile, "RowsVectorPredFiltered", TUnit::UNIT); @@ -86,10 +87,11 @@ Status OlapScanLocalState::_init_profile() { _vec_cond_timer = ADD_TIMER(_segment_profile, "VectorPredEvalTime"); _short_cond_timer = ADD_TIMER(_segment_profile, "ShortPredEvalTime"); _expr_filter_timer = ADD_TIMER(_segment_profile, "ExprFilterEvalTime"); - _first_read_timer = ADD_TIMER(_segment_profile, "FirstReadTime"); - _second_read_timer = ADD_TIMER(_segment_profile, "SecondReadTime"); - _first_read_seek_timer = ADD_TIMER(_segment_profile, "FirstReadSeekTime"); - _first_read_seek_counter = ADD_COUNTER(_segment_profile, "FirstReadSeekCount", TUnit::UNIT); + _predicate_column_read_timer = ADD_TIMER(_segment_profile, "PredicateColumnReadTime"); + _non_predicate_column_read_timer = ADD_TIMER(_segment_profile, "NonPredicateColumnReadTime"); + _predicate_column_read_seek_timer = ADD_TIMER(_segment_profile, "PredicateColumnReadSeekTime"); + _predicate_column_read_seek_counter = + ADD_COUNTER(_segment_profile, "PredicateColumnReadSeekCount", TUnit::UNIT); _lazy_read_timer = ADD_TIMER(_segment_profile, "LazyReadTime"); _lazy_read_seek_timer = ADD_TIMER(_segment_profile, "LazyReadSeekTime"); @@ -99,7 +101,7 @@ Status OlapScanLocalState::_init_profile() { _stats_filtered_counter = ADD_COUNTER(_segment_profile, "RowsStatsFiltered", TUnit::UNIT); _stats_rp_filtered_counter = - ADD_COUNTER(_segment_profile, "RowsZonemapRuntimePredicateFiltered", TUnit::UNIT); + ADD_COUNTER(_segment_profile, "RowsZoneMapRuntimePredicateFiltered", TUnit::UNIT); _bf_filtered_counter = ADD_COUNTER(_segment_profile, "RowsBloomFilterFiltered", TUnit::UNIT); _dict_filtered_counter = ADD_COUNTER(_segment_profile, "RowsDictFiltered", TUnit::UNIT); _del_filtered_counter = ADD_COUNTER(_scanner_profile, "RowsDelFiltered", TUnit::UNIT); @@ -130,8 +132,6 @@ Status OlapScanLocalState::_init_profile() { ADD_TIMER(_segment_profile, "InvertedIndexQueryNullBitmapTime"); _inverted_index_query_bitmap_copy_timer = ADD_TIMER(_segment_profile, "InvertedIndexQueryBitmapCopyTime"); - _inverted_index_query_bitmap_op_timer = - ADD_TIMER(_segment_profile, "InvertedIndexQueryBitmapOpTime"); _inverted_index_searcher_open_timer = ADD_TIMER(_segment_profile, "InvertedIndexSearcherOpenTime"); _inverted_index_searcher_search_timer = @@ -143,7 +143,7 @@ Status OlapScanLocalState::_init_profile() { _inverted_index_downgrade_count_counter = ADD_COUNTER(_segment_profile, "InvertedIndexDowngradeCount", TUnit::UNIT); - _output_index_result_column_timer = ADD_TIMER(_segment_profile, "OutputIndexResultColumnTimer"); + _output_index_result_column_timer = ADD_TIMER(_segment_profile, "OutputIndexResultColumnTime"); _filtered_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentFiltered", TUnit::UNIT); _total_segment_counter = ADD_COUNTER(_segment_profile, "NumSegmentTotal", TUnit::UNIT); _tablet_counter = ADD_COUNTER(_runtime_profile, "TabletNum", TUnit::UNIT); @@ -281,8 +281,9 @@ Status OlapScanLocalState::_init_scanners(std::list* s scan_range->version.data() + scan_range->version.size(), version); tablets.emplace_back(std::move(tablet), version); } - int64_t duration_ns = 0; + if (config::is_cloud_mode()) { + int64_t duration_ns = 0; SCOPED_RAW_TIMER(&duration_ns); std::vector> tasks; tasks.reserve(_scan_ranges.size()); @@ -292,8 +293,8 @@ Status OlapScanLocalState::_init_scanners(std::list* s }); } RETURN_IF_ERROR(cloud::bthread_fork_join(tasks, 10)); + _sync_rowset_timer->update(duration_ns); } - _sync_rowset_timer->update(duration_ns); if (enable_parallel_scan && !p._should_run_serial && !has_cpu_limit && p._push_down_agg_type == TPushAggOp::NONE && @@ -334,25 +335,6 @@ Status OlapScanLocalState::_init_scanners(std::list* s int scanners_per_tablet = std::max(1, 64 / (int)_scan_ranges.size()); - auto build_new_scanner = [&](BaseTabletSPtr tablet, int64_t version, - const std::vector& key_ranges) { - COUNTER_UPDATE(_key_range_counter, key_ranges.size()); - auto scanner = vectorized::NewOlapScanner::create_shared( - this, vectorized::NewOlapScanner::Params { - state(), - _scanner_profile.get(), - key_ranges, - std::move(tablet), - version, - {}, - p._limit, - p._olap_scan_node.is_preaggregation, - }); - RETURN_IF_ERROR(scanner->prepare(state(), _conjuncts)); - scanners->push_back(std::move(scanner)); - return Status::OK(); - }; - for (auto& scan_range : _scan_ranges) { auto tablet = DORIS_TRY(ExecEnv::get_tablet(scan_range->tablet_id)); int64_t version = 0; @@ -378,7 +360,21 @@ Status OlapScanLocalState::_init_scanners(std::list* s ++j, ++i) { scanner_ranges.push_back((*ranges)[i].get()); } - RETURN_IF_ERROR(build_new_scanner(tablet, version, scanner_ranges)); + + COUNTER_UPDATE(_key_range_counter, scanner_ranges.size()); + auto scanner = vectorized::NewOlapScanner::create_shared( + this, vectorized::NewOlapScanner::Params { + state(), + _scanner_profile.get(), + scanner_ranges, + std::move(tablet), + version, + {}, + p._limit, + p._olap_scan_node.is_preaggregation, + }); + RETURN_IF_ERROR(scanner->prepare(state(), _conjuncts)); + scanners->push_back(std::move(scanner)); } } diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index bd3e408f0d1158..69b4f3701b32d2 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -97,11 +97,8 @@ class OlapScanLocalState final : public ScanLocalState { std::unique_ptr _segment_profile; - RuntimeProfile::Counter* _num_disks_accessed_counter = nullptr; - RuntimeProfile::Counter* _tablet_counter = nullptr; RuntimeProfile::Counter* _key_range_counter = nullptr; - RuntimeProfile::Counter* _rows_pushed_cond_filtered_counter = nullptr; RuntimeProfile::Counter* _reader_init_timer = nullptr; RuntimeProfile::Counter* _scanner_init_timer = nullptr; RuntimeProfile::Counter* _process_conjunct_timer = nullptr; @@ -139,23 +136,19 @@ class OlapScanLocalState final : public ScanLocalState { RuntimeProfile::Counter* _block_init_timer = nullptr; RuntimeProfile::Counter* _block_init_seek_timer = nullptr; RuntimeProfile::Counter* _block_init_seek_counter = nullptr; - RuntimeProfile::Counter* _block_conditions_filtered_timer = nullptr; - RuntimeProfile::Counter* _block_conditions_filtered_bf_timer = nullptr; + RuntimeProfile::Counter* _segment_generate_row_range_timer = nullptr; + RuntimeProfile::Counter* _segment_generate_row_range_by_bf_timer = nullptr; RuntimeProfile::Counter* _collect_iterator_merge_next_timer = nullptr; - RuntimeProfile::Counter* _collect_iterator_normal_next_timer = nullptr; - RuntimeProfile::Counter* _block_conditions_filtered_zonemap_timer = nullptr; - RuntimeProfile::Counter* _block_conditions_filtered_zonemap_rp_timer = nullptr; - RuntimeProfile::Counter* _block_conditions_filtered_dict_timer = nullptr; - RuntimeProfile::Counter* _first_read_timer = nullptr; - RuntimeProfile::Counter* _second_read_timer = nullptr; - RuntimeProfile::Counter* _first_read_seek_timer = nullptr; - RuntimeProfile::Counter* _first_read_seek_counter = nullptr; + RuntimeProfile::Counter* _segment_generate_row_range_by_zonemap_timer = nullptr; + RuntimeProfile::Counter* _segment_generate_row_range_by_dict_timer = nullptr; + RuntimeProfile::Counter* _predicate_column_read_timer = nullptr; + RuntimeProfile::Counter* _non_predicate_column_read_timer = nullptr; + RuntimeProfile::Counter* _predicate_column_read_seek_timer = nullptr; + RuntimeProfile::Counter* _predicate_column_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; - // total pages read // used by segment v2 RuntimeProfile::Counter* _total_pages_num_counter = nullptr; @@ -175,7 +168,6 @@ class OlapScanLocalState final : public ScanLocalState { RuntimeProfile::Counter* _inverted_index_query_cache_miss_counter = nullptr; RuntimeProfile::Counter* _inverted_index_query_timer = nullptr; RuntimeProfile::Counter* _inverted_index_query_bitmap_copy_timer = nullptr; - RuntimeProfile::Counter* _inverted_index_query_bitmap_op_timer = nullptr; RuntimeProfile::Counter* _inverted_index_searcher_open_timer = nullptr; RuntimeProfile::Counter* _inverted_index_searcher_search_timer = nullptr; RuntimeProfile::Counter* _inverted_index_searcher_cache_hit_counter = nullptr; diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index 38cee083e2e28c..5da7faeabd364f 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -349,6 +349,10 @@ class PipelineXSinkLocalStateBase { RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } RuntimeProfile::Counter* exec_time_counter() { return _exec_timer; } + RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } + RuntimeProfile::HighWaterMarkCounter* peak_memory_usage_counter() { + return _peak_memory_usage_counter; + } virtual std::vector dependencies() const { return {nullptr}; } // override in exchange sink , AsyncWriterSink diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index d355d99c2e352f..cd707ccc49f8c2 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -46,6 +46,16 @@ Status RepeatLocalState::open(RuntimeState* state) { return Status::OK(); } +Status RepeatLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _evaluate_input_timer = ADD_TIMER(profile(), "EvaluateInputDataTime"); + _get_repeat_data_timer = ADD_TIMER(profile(), "GetRepeatDataTime"); + _filter_timer = ADD_TIMER(profile(), "FilterTime"); + return Status::OK(); +} + Status RepeatOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(tnode.repeat_node.exprs, _expr_ctxs)); @@ -166,23 +176,24 @@ Status RepeatLocalState::add_grouping_id_column(std::size_t rows, std::size_t& c Status RepeatOperatorX::push(RuntimeState* state, vectorized::Block* input_block, bool eos) const { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state._evaluate_input_timer); local_state._child_eos = eos; - auto& _intermediate_block = local_state._intermediate_block; - auto& _expr_ctxs = local_state._expr_ctxs; - DCHECK(!_intermediate_block || _intermediate_block->rows() == 0); + auto& intermediate_block = local_state._intermediate_block; + auto& expr_ctxs = local_state._expr_ctxs; + DCHECK(!intermediate_block || intermediate_block->rows() == 0); if (input_block->rows() > 0) { - _intermediate_block = vectorized::Block::create_unique(); + intermediate_block = vectorized::Block::create_unique(); - for (auto& expr : _expr_ctxs) { + for (auto& expr : expr_ctxs) { int result_column_id = -1; RETURN_IF_ERROR(expr->execute(input_block, &result_column_id)); DCHECK(result_column_id != -1); input_block->get_by_position(result_column_id).column = input_block->get_by_position(result_column_id) .column->convert_to_full_column_if_const(); - _intermediate_block->insert(input_block->get_by_position(result_column_id)); + intermediate_block->insert(input_block->get_by_position(result_column_id)); } - DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns()); + DCHECK_EQ(expr_ctxs.size(), intermediate_block->columns()); } return Status::OK(); @@ -202,33 +213,39 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp } DCHECK(output_block->rows() == 0); - if (_intermediate_block && _intermediate_block->rows() > 0) { - RETURN_IF_ERROR(local_state.get_repeated_block(_intermediate_block.get(), _repeat_id_idx, - output_block)); + { + SCOPED_TIMER(local_state._get_repeat_data_timer); + if (_intermediate_block && _intermediate_block->rows() > 0) { + RETURN_IF_ERROR(local_state.get_repeated_block(_intermediate_block.get(), + _repeat_id_idx, output_block)); - _repeat_id_idx++; + _repeat_id_idx++; - int size = _repeat_id_list.size(); - if (_repeat_id_idx >= size) { - _intermediate_block->clear(); + int size = _repeat_id_list.size(); + if (_repeat_id_idx >= size) { + _intermediate_block->clear(); + _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); + _repeat_id_idx = 0; + } + } else if (local_state._expr_ctxs.empty()) { + auto m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block( + output_block, _output_slots); + auto rows = _child_block.rows(); + auto& columns = m_block.mutable_columns(); + + for (int repeat_id_idx = 0; repeat_id_idx < _repeat_id_list.size(); repeat_id_idx++) { + std::size_t cur_col = 0; + RETURN_IF_ERROR( + local_state.add_grouping_id_column(rows, cur_col, columns, repeat_id_idx)); + } _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); - _repeat_id_idx = 0; } - } else if (local_state._expr_ctxs.empty()) { - auto m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, - _output_slots); - auto rows = _child_block.rows(); - auto& columns = m_block.mutable_columns(); - - for (int repeat_id_idx = 0; repeat_id_idx < _repeat_id_list.size(); repeat_id_idx++) { - std::size_t cur_col = 0; - RETURN_IF_ERROR( - local_state.add_grouping_id_column(rows, cur_col, columns, repeat_id_idx)); - } - _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); } - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, - output_block->columns())); + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, + output_block->columns())); + } *eos = _child_eos && _child_block.rows() == 0; local_state.reached_limit(output_block, eos); COUNTER_SET(local_state._rows_returned_counter, local_state._num_rows_returned); diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 22398df372ae65..31f88f37231aaa 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -36,6 +36,7 @@ class RepeatLocalState final : public PipelineXLocalState { using Base = PipelineXLocalState; RepeatLocalState(RuntimeState* state, OperatorXBase* parent); + Status init(RuntimeState* state, LocalStateInfo& info) override; Status open(RuntimeState* state) override; Status get_repeated_block(vectorized::Block* child_block, int repeat_id_idx, @@ -53,6 +54,10 @@ class RepeatLocalState final : public PipelineXLocalState { int _repeat_id_idx; std::unique_ptr _intermediate_block; vectorized::VExprContextSPtrs _expr_ctxs; + + RuntimeProfile::Counter* _evaluate_input_timer = nullptr; + RuntimeProfile::Counter* _get_repeat_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; class RepeatOperatorX final : public StatefulOperatorX { diff --git a/be/src/pipeline/exec/result_file_sink_operator.cpp b/be/src/pipeline/exec/result_file_sink_operator.cpp index 4d842db5d2346a..72fc1505573410 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.cpp +++ b/be/src/pipeline/exec/result_file_sink_operator.cpp @@ -84,12 +84,6 @@ Status ResultFileSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& i SCOPED_TIMER(_init_timer); _sender_id = info.sender_id; - _brpc_wait_timer = ADD_TIMER(_profile, "BrpcSendTime.Wait"); - _local_send_timer = ADD_TIMER(_profile, "LocalSendTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); - _split_block_distribute_by_channel_timer = - ADD_TIMER(_profile, "SplitBlockDistributeByChannelTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); auto& p = _parent->cast(); CHECK(p._file_opts.get() != nullptr); if (p._is_top_sink) { diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index 86b6035c134ba9..0e6906709f10a8 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -45,14 +45,6 @@ class ResultFileSinkLocalState final [[nodiscard]] int sender_id() const { return _sender_id; } - RuntimeProfile::Counter* brpc_wait_timer() { return _brpc_wait_timer; } - RuntimeProfile::Counter* local_send_timer() { return _local_send_timer; } - RuntimeProfile::Counter* brpc_send_timer() { return _brpc_send_timer; } - RuntimeProfile::Counter* merge_block_timer() { return _merge_block_timer; } - RuntimeProfile::Counter* split_block_distribute_by_channel_timer() { - return _split_block_distribute_by_channel_timer; - } - private: friend class ResultFileSinkOperatorX; @@ -66,12 +58,6 @@ class ResultFileSinkLocalState final bool _only_local_exchange = false; std::unique_ptr> _serializer; std::shared_ptr _block_holder; - RuntimeProfile::Counter* _brpc_wait_timer = nullptr; - RuntimeProfile::Counter* _local_send_timer = nullptr; - RuntimeProfile::Counter* _brpc_send_timer = nullptr; - RuntimeProfile::Counter* _merge_block_timer = nullptr; - RuntimeProfile::Counter* _split_block_distribute_by_channel_timer = nullptr; - int _sender_id; }; diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index 0608beaf522290..ccd74c6d572e5c 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -39,13 +39,12 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _fetch_row_id_timer = ADD_TIMER(profile(), "FetchRowIdTime"); + _write_data_timer = ADD_TIMER(profile(), "WriteDataTime"); static const std::string timer_name = "WaitForDependencyTime"; _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_profile, timer_name, 1); auto fragment_instance_id = state->fragment_instance_id(); - _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); - _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); - if (state->query_options().enable_parallel_result_sink) { _sender = _parent->cast()._sender; } else { @@ -143,12 +142,15 @@ Status ResultSinkOperatorX::open(RuntimeState* state) { Status ResultSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - COUNTER_UPDATE(local_state.rows_sent_counter(), (int64_t)block->rows()); - COUNTER_UPDATE(local_state.blocks_sent_counter(), 1); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); if (_fetch_option.use_two_phase_fetch && block->rows() > 0) { + SCOPED_TIMER(local_state._fetch_row_id_timer); RETURN_IF_ERROR(_second_phase_fetch_data(state, block)); } - RETURN_IF_ERROR(local_state._writer->write(state, *block)); + { + SCOPED_TIMER(local_state._write_data_timer); + RETURN_IF_ERROR(local_state._writer->write(state, *block)); + } if (_fetch_option.use_two_phase_fetch) { // Block structure may be changed by calling _second_phase_fetch_data(). // So we should clear block in case of unmatched columns diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index 3c503096ecb51e..339c167825643b 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -128,8 +128,6 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState _sender = nullptr; std::shared_ptr _writer = nullptr; - RuntimeProfile::Counter* _blocks_sent_counter = nullptr; - RuntimeProfile::Counter* _rows_sent_counter = nullptr; + + RuntimeProfile::Counter* _fetch_row_id_timer = nullptr; + RuntimeProfile::Counter* _write_data_timer = nullptr; }; class ResultSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index cf723cd8732b01..9a83d9c7838c21 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -1052,13 +1052,10 @@ Status ScanLocalState::_init_profile() { ADD_COUNTER(_scanner_profile, "NewlyCreateFreeBlocksNum", TUnit::UNIT); _scale_up_scanners_counter = ADD_COUNTER(_scanner_profile, "NumScaleUpScanners", TUnit::UNIT); // time of transfer thread to wait for block from scan thread - _scanner_wait_batch_timer = ADD_TIMER(_scanner_profile, "ScannerBatchWaitTime"); _scanner_sched_counter = ADD_COUNTER(_scanner_profile, "ScannerSchedCount", TUnit::UNIT); - _scanner_ctx_sched_time = ADD_TIMER(_scanner_profile, "ScannerCtxSchedTime"); _scan_timer = ADD_TIMER(_scanner_profile, "ScannerGetBlockTime"); _scan_cpu_timer = ADD_TIMER(_scanner_profile, "ScannerCpuTime"); - _convert_block_timer = ADD_TIMER(_scanner_profile, "ScannerConvertBlockTime"); _filter_timer = ADD_TIMER(_scanner_profile, "ScannerFilterTime"); // time of scan thread to wait for worker thread of the thread pool diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index 56234d9ea4ef96..7c774a5aaa0dbc 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -102,8 +102,6 @@ class ScanLocalStateBase : public PipelineXLocalState<>, public RuntimeFilterCon std::shared_ptr _scanner_profile; RuntimeProfile::Counter* _scanner_sched_counter = nullptr; - RuntimeProfile::Counter* _scanner_ctx_sched_time = nullptr; - RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; RuntimeProfile::Counter* _scanner_wait_worker_timer = nullptr; // Num of newly created free blocks when running query RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; @@ -114,8 +112,6 @@ class ScanLocalStateBase : public PipelineXLocalState<>, public RuntimeFilterCon // time of get block from scanner RuntimeProfile::Counter* _scan_timer = nullptr; RuntimeProfile::Counter* _scan_cpu_timer = nullptr; - // time of convert input block to output block from scanner - RuntimeProfile::Counter* _convert_block_timer = nullptr; // time of filter output block from scanner RuntimeProfile::Counter* _filter_timer = nullptr; RuntimeProfile::Counter* _memory_usage_counter = nullptr; diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 3ee79629e1a788..33ba7d73100a04 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -71,12 +71,16 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized auto probe_rows = in_block->rows(); if (probe_rows > 0) { - RETURN_IF_ERROR(_extract_probe_column(local_state, *in_block, local_state._probe_columns, - _cur_child_id)); + { + SCOPED_TIMER(local_state._extract_probe_data_timer); + RETURN_IF_ERROR(_extract_probe_column(local_state, *in_block, + local_state._probe_columns, _cur_child_id)); + } RETURN_IF_ERROR(std::visit( [&](auto&& arg) -> Status { using HashTableCtxType = std::decay_t; if constexpr (!std::is_same_v) { + SCOPED_TIMER(local_state._probe_timer); vectorized::HashTableProbe process_hashtable_ctx(&local_state, probe_rows); return process_hashtable_ctx.mark_data_in_hashtable(arg); @@ -99,6 +103,9 @@ Status SetProbeSinkLocalState::init(RuntimeState* state, LocalSink RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + + _probe_timer = ADD_TIMER(Base::profile(), "ProbeTime"); + _extract_probe_data_timer = ADD_TIMER(Base::profile(), "ExtractProbeDataTime"); Parent& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; _dependency->block(); diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index f320c8e89cdcaf..368ea812cdfe01 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -60,6 +60,9 @@ class SetProbeSinkLocalState final : public PipelineXSinkLocalState diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 01c26e4d005c65..f8287ce5fa7863 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -39,8 +39,10 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo auto& valid_element_in_hash_tbl = local_state._shared_state->valid_element_in_hash_tbl; if (in_block->rows() != 0) { - RETURN_IF_ERROR(local_state._mutable_block.merge(*in_block)); - + { + SCOPED_TIMER(local_state._merge_block_timer); + RETURN_IF_ERROR(local_state._mutable_block.merge(*in_block)); + } if (local_state._mutable_block.rows() > std::numeric_limits::max()) { return Status::NotSupported("set operator do not support build table rows over:" + std::to_string(std::numeric_limits::max())); @@ -48,6 +50,7 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo } if (eos || local_state._mutable_block.allocated_bytes() >= BUILD_BLOCK_MAX_SIZE) { + SCOPED_TIMER(local_state._build_timer); build_block = local_state._mutable_block.to_block(); RETURN_IF_ERROR(_process_build_block(local_state, build_block, state)); local_state._mutable_block.clear(); @@ -151,6 +154,7 @@ Status SetSinkLocalState::init(RuntimeState* state, LocalSinkState RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _merge_block_timer = ADD_TIMER(_profile, "MergeBlocksTime"); _build_timer = ADD_TIMER(_profile, "BuildTime"); auto& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index 0e76867b31f989..8f917b2dc79e58 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -48,14 +48,14 @@ class SetSinkLocalState final : public PipelineXSinkLocalState { private: friend class SetSinkOperatorX; - template - friend struct vectorized::HashTableBuild; - RuntimeProfile::Counter* _build_timer; // time to build hash table vectorized::MutableBlock _mutable_block; // every child has its result expr list vectorized::VExprContextSPtrs _child_exprs; vectorized::Arena _arena; + + RuntimeProfile::Counter* _merge_block_timer = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; }; template diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index c6a80f8d06c94f..5a4500d34d8cdc 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -29,6 +29,8 @@ Status SetSourceLocalState::init(RuntimeState* state, LocalStateIn RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _get_data_timer = ADD_TIMER(_runtime_profile, "GetDataTime"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); _shared_state->probe_finished_children_dependency.resize( _parent->cast>()._child_quantity, nullptr); return Status::OK(); @@ -69,21 +71,26 @@ Status SetSourceOperatorX::get_block(RuntimeState* state, vectoriz auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); _create_mutable_cols(local_state, block); - auto st = std::visit( - [&](auto&& arg) -> Status { - using HashTableCtxType = std::decay_t; - if constexpr (!std::is_same_v) { - return _get_data_in_hashtable(local_state, arg, block, - state->batch_size(), eos); - } else { - LOG(FATAL) << "FATAL: uninited hash table"; - __builtin_unreachable(); - } - }, - *local_state._shared_state->hash_table_variants); - RETURN_IF_ERROR(st); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + { + SCOPED_TIMER(local_state._get_data_timer); + RETURN_IF_ERROR(std::visit( + [&](auto&& arg) -> Status { + using HashTableCtxType = std::decay_t; + if constexpr (!std::is_same_v) { + return _get_data_in_hashtable(local_state, arg, block, + state->batch_size(), eos); + } else { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + } + }, + *local_state._shared_state->hash_table_variants)); + } + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); + } local_state.reached_limit(block, eos); return Status::OK(); } diff --git a/be/src/pipeline/exec/set_source_operator.h b/be/src/pipeline/exec/set_source_operator.h index 5157a2f9c979fe..53fbe4e5398432 100644 --- a/be/src/pipeline/exec/set_source_operator.h +++ b/be/src/pipeline/exec/set_source_operator.h @@ -46,6 +46,9 @@ class SetSourceLocalState final : public PipelineXLocalState { std::vector _mutable_cols; //record build column type vectorized::DataTypes _left_table_data_types; + + RuntimeProfile::Counter* _get_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; template diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index e20b21a0bf26c2..6f67262ef1f3ed 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -32,6 +32,8 @@ Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { SCOPED_TIMER(_init_timer); _sort_blocks_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SortBlocks", TUnit::BYTES, "MemoryUsage", 1); + _append_blocks_timer = ADD_TIMER(profile(), "AppendBlockTime"); + _update_runtime_predicate_timer = ADD_TIMER(profile(), "UpdateRuntimePredicateTime"); return Status::OK(); } @@ -120,11 +122,15 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (in_block->rows() > 0) { COUNTER_UPDATE(local_state._sort_blocks_memory_usage, (int64_t)in_block->bytes()); - RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); + { + SCOPED_TIMER(local_state._append_blocks_timer); + RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); + } local_state._mem_tracker->set_consumption(local_state._shared_state->sorter->data_size()); RETURN_IF_CANCELLED(state); if (state->get_query_ctx()->has_runtime_predicate(_node_id)) { + SCOPED_TIMER(local_state._update_runtime_predicate_timer); auto& predicate = state->get_query_ctx()->get_runtime_predicate(_node_id); if (predicate.enable()) { vectorized::Field new_top = local_state._shared_state->sorter->get_top_value(); diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index a5a24e371635fe..6bf87164e71026 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -46,6 +46,8 @@ class SortSinkLocalState : public PipelineXSinkLocalState { // topn top value vectorized::Field old_top {vectorized::Field::Types::Null}; + RuntimeProfile::Counter* _append_blocks_timer = nullptr; + RuntimeProfile::Counter* _update_runtime_predicate_timer = nullptr; }; class SortSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index dfbe42c637ea56..780bd194ac8b6b 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -93,25 +93,18 @@ Status StreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { "SerializeKeyArena", TUnit::BYTES, "MemoryUsage", 1); _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); - _build_table_convert_timer = ADD_TIMER(Base::profile(), "BuildConvertToPartitionedTime"); - _serialize_key_timer = ADD_TIMER(Base::profile(), "SerializeKeyTime"); - _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); _merge_timer = ADD_TIMER(Base::profile(), "MergeTime"); _expr_timer = ADD_TIMER(Base::profile(), "ExprTime"); - _serialize_data_timer = ADD_TIMER(Base::profile(), "SerializeDataTime"); + _insert_values_to_column_timer = ADD_TIMER(Base::profile(), "InsertValuesToColumnTime"); _deserialize_data_timer = ADD_TIMER(Base::profile(), "DeserializeAndMergeTime"); _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); - _max_row_size_counter = ADD_COUNTER(Base::profile(), "MaxRowSizeInBytes", TUnit::UNIT); _hash_table_size_counter = ADD_COUNTER(profile(), "HashTableSize", TUnit::UNIT); - _queue_byte_size_counter = ADD_COUNTER(profile(), "MaxSizeInBlockQueue", TUnit::BYTES); - _queue_size_counter = ADD_COUNTER(profile(), "MaxSizeOfBlockQueue", TUnit::UNIT); _streaming_agg_timer = ADD_TIMER(profile(), "StreamingAggTime"); _build_timer = ADD_TIMER(profile(), "BuildTime"); _expr_timer = ADD_TIMER(Base::profile(), "ExprTime"); _get_results_timer = ADD_TIMER(profile(), "GetResultsTime"); - _serialize_result_timer = ADD_TIMER(profile(), "SerializeResultTime"); _hash_table_iterate_timer = ADD_TIMER(profile(), "HashTableIterateTime"); _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); @@ -683,7 +676,7 @@ Status StreamingAggLocalState::_pre_agg_with_serialized_key(doris::vectorized::B } for (int i = 0; i != _aggregate_evaluators.size(); ++i) { - SCOPED_TIMER(_serialize_data_timer); + SCOPED_TIMER(_insert_values_to_column_timer); RETURN_IF_ERROR( _aggregate_evaluators[i]->streaming_agg_serialize_to_column( in_block, value_columns[i], rows, @@ -852,12 +845,12 @@ Status StreamingAggLocalState::_get_with_serialized_key_result(RuntimeState* sta return Status::OK(); } -Status StreamingAggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Block* block, - bool* eos) { +Status StreamingAggLocalState::_get_results_without_key(RuntimeState* state, + vectorized::Block* block, bool* eos) { // 1. `child(0)->rows_returned() == 0` mean not data from child // in level two aggregation node should return NULL result // level one aggregation node set `eos = true` return directly - SCOPED_TIMER(_serialize_result_timer); + SCOPED_TIMER(_get_results_timer); if (UNLIKELY(_input_num_rows == 0)) { *eos = true; return Status::OK(); @@ -896,10 +889,10 @@ Status StreamingAggLocalState::_serialize_without_key(RuntimeState* state, vecto return Status::OK(); } -Status StreamingAggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, - vectorized::Block* block, - bool* eos) { - SCOPED_TIMER(_serialize_result_timer); +Status StreamingAggLocalState::_get_results_with_serialized_key(RuntimeState* state, + vectorized::Block* block, + bool* eos) { + SCOPED_TIMER(_get_results_timer); auto& p = _parent->cast(); int key_size = _probe_expr_ctxs.size(); int agg_size = _aggregate_evaluators.size(); @@ -918,7 +911,6 @@ Status StreamingAggLocalState::_serialize_with_serialized_key_result(RuntimeStat } } - SCOPED_TIMER(_get_results_timer); std::visit( vectorized::Overload { [&](std::monostate& arg) -> void { @@ -974,7 +966,7 @@ Status StreamingAggLocalState::_serialize_with_serialized_key_result(RuntimeStat } { - SCOPED_TIMER(_serialize_data_timer); + SCOPED_TIMER(_insert_values_to_column_timer); for (size_t i = 0; i < _aggregate_evaluators.size(); ++i) { value_data_types[i] = _aggregate_evaluators[i]->function()->get_serialized_type(); diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.h b/be/src/pipeline/exec/streaming_aggregation_operator.h index c37fa5cbd881ca..59d5491d10c12f 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_operator.h @@ -65,11 +65,11 @@ class StreamingAggLocalState final : public PipelineXLocalState void _update_memusage_with_serialized_key(); Status _init_hash_method(const vectorized::VExprContextSPtrs& probe_exprs); Status _get_without_key_result(RuntimeState* state, vectorized::Block* block, bool* eos); - Status _serialize_without_key(RuntimeState* state, vectorized::Block* block, bool* eos); + Status _get_results_without_key(RuntimeState* state, vectorized::Block* block, bool* eos); Status _get_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos); - Status _serialize_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, - bool* eos); + Status _get_results_with_serialized_key(RuntimeState* state, vectorized::Block* block, + bool* eos); template Status _merge_with_serialized_key_helper(vectorized::Block* block); @@ -83,25 +83,19 @@ class StreamingAggLocalState final : public PipelineXLocalState Status _create_agg_status(vectorized::AggregateDataPtr data); size_t _get_hash_table_size(); - RuntimeProfile::Counter* _queue_byte_size_counter = nullptr; - RuntimeProfile::Counter* _queue_size_counter = nullptr; RuntimeProfile::Counter* _streaming_agg_timer = nullptr; RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; RuntimeProfile::Counter* _hash_table_input_counter = nullptr; RuntimeProfile::Counter* _build_timer = nullptr; RuntimeProfile::Counter* _expr_timer = nullptr; - RuntimeProfile::Counter* _build_table_convert_timer = nullptr; - RuntimeProfile::Counter* _serialize_key_timer = nullptr; RuntimeProfile::Counter* _merge_timer = nullptr; - RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _insert_values_to_column_timer = nullptr; RuntimeProfile::Counter* _deserialize_data_timer = nullptr; - RuntimeProfile::Counter* _max_row_size_counter = nullptr; RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; RuntimeProfile::Counter* _hash_table_size_counter = nullptr; RuntimeProfile::Counter* _get_results_timer = nullptr; - RuntimeProfile::Counter* _serialize_result_timer = nullptr; RuntimeProfile::Counter* _hash_table_iterate_timer = nullptr; RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; @@ -136,13 +130,13 @@ class StreamingAggLocalState final : public PipelineXLocalState if constexpr (NeedFinalize) { return local_state->_get_without_key_result(state, block, eos); } else { - return local_state->_serialize_without_key(state, block, eos); + return local_state->_get_results_without_key(state, block, eos); } } else { if constexpr (NeedFinalize) { return local_state->_get_with_serialized_key_result(state, block, eos); } else { - return local_state->_serialize_with_serialized_key_result(state, block, eos); + return local_state->_get_results_with_serialized_key(state, block, eos); } } } diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 38e69f7cb0e897..c1621470f435b4 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -32,6 +32,18 @@ namespace doris::pipeline { TableFunctionLocalState::TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState<>(state, parent), _child_block(vectorized::Block::create_unique()) {} +Status TableFunctionLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _init_function_timer = ADD_TIMER(_runtime_profile, "InitTableFunctionTime"); + _process_rows_timer = ADD_TIMER(_runtime_profile, "ProcessRowsTime"); + _copy_data_timer = ADD_TIMER(_runtime_profile, "CopyDataTime"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); + _repeat_data_timer = ADD_TIMER(_runtime_profile, "RepeatDataTime"); + return Status::OK(); +} + Status TableFunctionLocalState::open(RuntimeState* state) { SCOPED_TIMER(PipelineXLocalState<>::exec_time_counter()); SCOPED_TIMER(PipelineXLocalState<>::_open_timer); @@ -59,6 +71,7 @@ void TableFunctionLocalState::_copy_output_slots( if (!_current_row_insert_times) { return; } + SCOPED_TIMER(_copy_data_timer); auto& p = _parent->cast(); for (auto index : p._output_slot_indexs) { auto src_column = _child_block->get_by_position(index).column; @@ -197,15 +210,18 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, columns[index]->insert_many_defaults(row_size - columns[index]->size()); } - // 3. eval conjuncts - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, - output_block->columns())); + { + SCOPED_TIMER(_filter_timer); // 3. eval conjuncts + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, + output_block->columns())); + } *eos = _child_eos && _cur_child_offset == -1; return Status::OK(); } void TableFunctionLocalState::process_next_child_row() { + SCOPED_TIMER(_process_rows_timer); _cur_child_offset++; if (_cur_child_offset >= _child_block->rows()) { diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 75b1608fad7112..81160acb7f7611 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -37,6 +37,7 @@ class TableFunctionLocalState final : public PipelineXLocalState<> { TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent); ~TableFunctionLocalState() override = default; + Status init(RuntimeState* state, LocalStateInfo& infos) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state) override { for (auto* fn : _fns) { @@ -67,6 +68,12 @@ class TableFunctionLocalState final : public PipelineXLocalState<> { std::unique_ptr _child_block; int _current_row_insert_times = 0; bool _child_eos = false; + + RuntimeProfile::Counter* _init_function_timer = nullptr; + RuntimeProfile::Counter* _process_rows_timer = nullptr; + RuntimeProfile::Counter* _copy_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _repeat_data_timer = nullptr; }; class TableFunctionOperatorX final : public StatefulOperatorX { @@ -93,6 +100,7 @@ class TableFunctionOperatorX final : public StatefulOperatorXprocess_init(input_block, state)); } local_state.process_next_child_row(); diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 288fc131037fab..8467eeb1d5467a 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -32,6 +32,7 @@ Status UnionSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _expr_timer = ADD_TIMER(_profile, "ExprTime"); auto& p = _parent->cast(); _shared_state->data_queue.set_sink_dependency(_dependency, p._cur_child_id); return Status::OK(); diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index f939950143ae92..aa94ed9a73038f 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -55,6 +55,7 @@ class UnionSinkLocalState final : public PipelineXSinkLocalState { @@ -136,6 +137,7 @@ class UnionSinkOperatorX final : public DataSinkOperatorX { Status materialize_block(RuntimeState* state, vectorized::Block* src_block, int child_idx, vectorized::Block* res_block) { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state._expr_timer); const auto& child_exprs = local_state._child_expr; vectorized::ColumnsWithTypeAndName colunms; for (size_t i = 0; i < child_exprs.size(); ++i) { diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index 728057667e8505..084765e5aaa9c0 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -163,8 +163,7 @@ Status convert_to_arrow_field(SlotDescriptor* desc, std::shared_ptr* result) { +Status get_arrow_schema(const vectorized::Block& block, std::shared_ptr* result) { std::vector> fields; for (const auto& type_and_name : block) { std::shared_ptr arrow_type; @@ -177,20 +176,6 @@ Status convert_block_arrow_schema(const vectorized::Block& block, return Status::OK(); } -Status convert_to_arrow_schema(const RowDescriptor& row_desc, - std::shared_ptr* result) { - std::vector> fields; - for (auto tuple_desc : row_desc.tuple_descriptors()) { - for (auto desc : tuple_desc->slots()) { - std::shared_ptr field; - RETURN_IF_ERROR(convert_to_arrow_field(desc, &field)); - fields.push_back(field); - } - } - *result = arrow::schema(std::move(fields)); - return Status::OK(); -} - Status convert_expr_ctxs_arrow_schema(const vectorized::VExprContextSPtrs& output_vexpr_ctxs, std::shared_ptr* result) { std::vector> fields; diff --git a/be/src/util/arrow/row_batch.h b/be/src/util/arrow/row_batch.h index ddffc3324d3451..5dd76ff66d7ff8 100644 --- a/be/src/util/arrow/row_batch.h +++ b/be/src/util/arrow/row_batch.h @@ -43,12 +43,7 @@ class RowDescriptor; Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr* result); -// Convert Doris RowDescriptor to Arrow Schema. -Status convert_to_arrow_schema(const RowDescriptor& row_desc, - std::shared_ptr* result); - -Status convert_block_arrow_schema(const vectorized::Block& block, - std::shared_ptr* result); +Status get_arrow_schema(const vectorized::Block& block, std::shared_ptr* result); Status convert_expr_ctxs_arrow_schema(const vectorized::VExprContextSPtrs& output_vexpr_ctxs, std::shared_ptr* result); diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h index cd1f8922e1b459..f67fe14fa42600 100644 --- a/be/src/vec/aggregate_functions/aggregate_function.h +++ b/be/src/vec/aggregate_functions/aggregate_function.h @@ -115,21 +115,21 @@ class IAggregateFunction { * Additional parameter arena should be used instead of standard memory allocator if the addition requires memory allocation. */ virtual void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const = 0; + Arena*) const = 0; virtual void add_many(AggregateDataPtr __restrict place, const IColumn** columns, - std::vector& rows, Arena* arena) const {} + std::vector& rows, Arena*) const {} /// Merges state (on which place points to) with other state of current aggregation function. virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const = 0; + Arena*) const = 0; virtual void merge_vec(const AggregateDataPtr* places, size_t offset, ConstAggregateDataPtr rhs, - Arena* arena, const size_t num_rows) const = 0; + Arena*, const size_t num_rows) const = 0; // same as merge_vec, but only call "merge" function when place is not nullptr virtual void merge_vec_selected(const AggregateDataPtr* places, size_t offset, - ConstAggregateDataPtr rhs, Arena* arena, + ConstAggregateDataPtr rhs, Arena*, const size_t num_rows) const = 0; /// Serializes state (to transmit it over the network, for example). @@ -146,21 +146,21 @@ class IAggregateFunction { /// Deserializes state. This function is called only for empty (just created) states. virtual void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, - Arena* arena) const = 0; + Arena*) const = 0; - virtual void deserialize_vec(AggregateDataPtr places, const ColumnString* column, Arena* arena, + virtual void deserialize_vec(AggregateDataPtr places, const ColumnString* column, Arena*, size_t num_rows) const = 0; virtual void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const = 0; + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const = 0; virtual void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const = 0; + Arena*, const size_t num_rows) const = 0; - virtual void deserialize_from_column(AggregateDataPtr places, const IColumn& column, - Arena* arena, size_t num_rows) const = 0; + virtual void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, + size_t num_rows) const = 0; /// Deserializes state and merge it with current aggregation function. virtual void deserialize_and_merge(AggregateDataPtr __restrict place, @@ -169,10 +169,10 @@ class IAggregateFunction { virtual void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, - size_t end, Arena* arena) const = 0; + size_t end, Arena*) const = 0; virtual void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, - const IColumn& column, Arena* arena) const = 0; + const IColumn& column, Arena*) const = 0; /// Inserts results into a column. virtual void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const = 0; @@ -185,33 +185,32 @@ class IAggregateFunction { * and do a single call to "add_batch" for devirtualization and inlining. */ virtual void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, - const IColumn** columns, Arena* arena, bool agg_many = false) const = 0; + const IColumn** columns, Arena*, bool agg_many = false) const = 0; // same as add_batch, but only call "add" function when place is not nullptr virtual void add_batch_selected(size_t batch_size, AggregateDataPtr* places, - size_t place_offset, const IColumn** columns, - Arena* arena) const = 0; + size_t place_offset, const IColumn** columns, Arena*) const = 0; /** The same for single place. */ virtual void add_batch_single_place(size_t batch_size, AggregateDataPtr place, - const IColumn** columns, Arena* arena) const = 0; + const IColumn** columns, Arena*) const = 0; // only used at agg reader virtual void add_batch_range(size_t batch_begin, size_t batch_end, AggregateDataPtr place, - const IColumn** columns, Arena* arena, bool has_null = false) = 0; + const IColumn** columns, Arena*, bool has_null = false) = 0; // only used at window function virtual void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const = 0; + Arena*) const = 0; virtual void streaming_agg_serialize(const IColumn** columns, BufferWritable& buf, - const size_t num_rows, Arena* arena) const = 0; + const size_t num_rows, Arena*) const = 0; virtual void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const = 0; + const size_t num_rows, Arena*) const = 0; const DataTypes& get_argument_types() const { return argument_types; } diff --git a/be/src/vec/aggregate_functions/aggregate_function_avg.h b/be/src/vec/aggregate_functions/aggregate_function_avg.h index 8a18a88839b4db..62fbb8078ea949 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_avg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_avg.h @@ -184,7 +184,7 @@ class AggregateFunctionAvg final column.get_data().push_back(this->data(place).template result()); } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto& col = assert_cast(column); DCHECK(col.size() >= num_rows) << "source column's size should greater than num_rows"; @@ -205,7 +205,7 @@ class AggregateFunctionAvg final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto* src_data = assert_cast(*columns[0]).get_data().data(); auto& dst_col = assert_cast(*dst); dst_col.set_item_size(sizeof(Data)); @@ -219,7 +219,7 @@ class AggregateFunctionAvg final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); DCHECK(col.size() >= num_rows) << "source column's size should greater than num_rows"; @@ -233,7 +233,7 @@ class AggregateFunctionAvg final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -245,19 +245,19 @@ class AggregateFunctionAvg final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h index 6c504b91bf4abd..b0619a63e1ffe8 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h @@ -159,7 +159,7 @@ class AggregateFunctionBitmapSerializationHelper : IAggregateFunctionDataHelper(argument_types_) {} void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { if (version >= BITMAP_SERDE) { auto& col = assert_cast(*dst); char place[sizeof(Data)]; @@ -171,11 +171,11 @@ class AggregateFunctionBitmapSerializationHelper assert_cast(this)->destroy(place); }); assert_cast(this)->add(place, columns, - i, arena); + i, nullptr); data[i] = std::move(this->data(place).value); } } else { - BaseHelper::streaming_agg_serialize_to_column(columns, dst, num_rows, arena); + BaseHelper::streaming_agg_serialize_to_column(columns, dst, num_rows, nullptr); } } @@ -194,7 +194,7 @@ class AggregateFunctionBitmapSerializationHelper } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { if (version >= BITMAP_SERDE) { auto& col = assert_cast(column); const size_t num_rows = column.size(); @@ -204,13 +204,13 @@ class AggregateFunctionBitmapSerializationHelper this->data(place).merge(data[i]); } } else { - BaseHelper::deserialize_and_merge_from_column(place, column, arena); + BaseHelper::deserialize_and_merge_from_column(place, column, nullptr); } } void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); if (version >= BITMAP_SERDE) { @@ -220,12 +220,12 @@ class AggregateFunctionBitmapSerializationHelper this->data(place).merge(data[i]); } } else { - BaseHelper::deserialize_and_merge_from_column_range(place, column, begin, end, arena); + BaseHelper::deserialize_and_merge_from_column_range(place, column, begin, end, nullptr); } } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { if (version >= BITMAP_SERDE) { const auto& col = assert_cast(*column); @@ -234,13 +234,13 @@ class AggregateFunctionBitmapSerializationHelper this->data(places[i] + offset).merge(data[i]); } } else { - BaseHelper::deserialize_and_merge_vec(places, offset, rhs, column, arena, num_rows); + BaseHelper::deserialize_and_merge_vec(places, offset, rhs, column, nullptr, num_rows); } } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { if (version >= BITMAP_SERDE) { const auto& col = assert_cast(*column); const auto* data = col.get_data().data(); @@ -250,7 +250,7 @@ class AggregateFunctionBitmapSerializationHelper } } } else { - BaseHelper::deserialize_and_merge_vec_selected(places, offset, rhs, column, arena, + BaseHelper::deserialize_and_merge_vec_selected(places, offset, rhs, column, nullptr, num_rows); } } diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h index 19352e022fa7a2..5747faf1b8e8c1 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap_agg.h @@ -72,7 +72,7 @@ class AggregateFunctionBitmapAgg final DataTypePtr get_return_type() const override { return std::make_shared(); } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { DCHECK_LT(row_num, columns[0]->size()); if constexpr (arg_nullable) { auto& nullable_col = @@ -90,7 +90,7 @@ class AggregateFunctionBitmapAgg final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { if constexpr (arg_nullable) { auto& nullable_column = assert_cast(*columns[0]); const auto& column = @@ -111,7 +111,7 @@ class AggregateFunctionBitmapAgg final void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { + Arena*) const override { this->data(place).merge(this->data(rhs)); } @@ -130,7 +130,7 @@ class AggregateFunctionBitmapAgg final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& col = assert_cast(*dst); char place[sizeof(Data)]; col.resize(num_rows); @@ -138,12 +138,12 @@ class AggregateFunctionBitmapAgg final for (size_t i = 0; i != num_rows; ++i) { this->create(place); DEFER({ this->destroy(place); }); - this->add(place, columns, i, arena); + this->add(place, columns, i, nullptr); data[i] = std::move(this->data(place).value); } } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto& col = assert_cast(column); DCHECK(col.size() >= num_rows) << "source column's size should greater than num_rows"; @@ -165,7 +165,7 @@ class AggregateFunctionBitmapAgg final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); auto* data = col.get_data().data(); @@ -177,7 +177,7 @@ class AggregateFunctionBitmapAgg final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -188,7 +188,7 @@ class AggregateFunctionBitmapAgg final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { const auto& col = assert_cast(*column); const auto* data = col.get_data().data(); @@ -198,8 +198,8 @@ class AggregateFunctionBitmapAgg final } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { const auto& col = assert_cast(*column); const auto* data = col.get_data().data(); for (size_t i = 0; i != num_rows; ++i) { diff --git a/be/src/vec/aggregate_functions/aggregate_function_count.h b/be/src/vec/aggregate_functions/aggregate_function_count.h index 62aa869771c0a5..7b54d074683b04 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count.h @@ -91,7 +91,7 @@ class AggregateFunctionCount final assert_cast(to).get_data().push_back(data(place).count); } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto data = assert_cast(column).get_data().data(); memcpy(places, data, sizeof(Data) * num_rows); @@ -111,7 +111,7 @@ class AggregateFunctionCount final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& dst_col = assert_cast(*dst); DCHECK(dst_col.item_size() == sizeof(Data)) << "size is not equal: " << dst_col.item_size() << " " << sizeof(Data); @@ -124,7 +124,7 @@ class AggregateFunctionCount final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); auto* data = reinterpret_cast(col.get_data().data()); @@ -135,7 +135,7 @@ class AggregateFunctionCount final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -146,19 +146,19 @@ class AggregateFunctionCount final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, @@ -229,7 +229,7 @@ class AggregateFunctionCountNotNullUnary final } } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto data = assert_cast(column).get_data().data(); memcpy(places, data, sizeof(Data) * num_rows); @@ -249,7 +249,7 @@ class AggregateFunctionCountNotNullUnary final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& col = assert_cast(*dst); DCHECK(col.item_size() == sizeof(Data)) << "size is not equal: " << col.item_size() << " " << sizeof(Data); @@ -263,7 +263,7 @@ class AggregateFunctionCountNotNullUnary final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); auto* data = reinterpret_cast(col.get_data().data()); @@ -274,7 +274,7 @@ class AggregateFunctionCountNotNullUnary final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -286,19 +286,19 @@ class AggregateFunctionCountNotNullUnary final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, diff --git a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h index 5d4a3dde3550a1..1f5093de68263e 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h +++ b/be/src/vec/aggregate_functions/aggregate_function_count_by_enum.h @@ -197,7 +197,7 @@ class AggregateFunctionCountByEnum final DataTypePtr get_return_type() const override { return std::make_shared(); } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { for (int i = 0; i < arg_count; i++) { const auto* nullable_column = check_and_get_column(columns[i]); if (nullable_column == nullptr) { @@ -217,7 +217,7 @@ class AggregateFunctionCountByEnum final void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { + Arena*) const override { this->data(place).merge(this->data(rhs)); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_histogram.h b/be/src/vec/aggregate_functions/aggregate_function_histogram.h index 25fc6957321586..1d2c5725ed370f 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_histogram.h +++ b/be/src/vec/aggregate_functions/aggregate_function_histogram.h @@ -192,7 +192,7 @@ class AggregateFunctionHistogram final DataTypePtr get_return_type() const override { return std::make_shared(); } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { if constexpr (has_input_param) { Int32 input_max_num_buckets = assert_cast(columns[1])->get_element(row_num); @@ -220,7 +220,7 @@ class AggregateFunctionHistogram final void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { + Arena*) const override { this->data(place).merge(this->data(rhs)); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h index 1cf6dc7f2a29a9..44835194eb4b88 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h +++ b/be/src/vec/aggregate_functions/aggregate_function_hll_union_agg.h @@ -122,7 +122,7 @@ class AggregateFunctionHLLUnion } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { this->data(place).add(columns[0], row_num); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h index d314cba7a656a9..a2eecb71391736 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h +++ b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h @@ -332,7 +332,7 @@ class AggregateJavaUdaf final } void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, - const IColumn** columns, Arena* /*arena*/, bool /*agg_many*/) const override { + const IColumn** columns, Arena*, bool /*agg_many*/) const override { int64_t places_address = reinterpret_cast(places); Status st = this->data(_exec_place) .add(places_address, false, columns, 0, batch_size, argument_types, @@ -343,7 +343,7 @@ class AggregateJavaUdaf final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* /*arena*/) const override { + Arena*) const override { int64_t places_address = reinterpret_cast(place); Status st = this->data(_exec_place) .add(places_address, true, columns, 0, batch_size, argument_types, 0); @@ -354,7 +354,7 @@ class AggregateJavaUdaf final void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { frame_start = std::max(frame_start, partition_start); frame_end = std::min(frame_end, partition_end); int64_t places_address = reinterpret_cast(place); diff --git a/be/src/vec/aggregate_functions/aggregate_function_map.h b/be/src/vec/aggregate_functions/aggregate_function_map.h index d56cbf21f31136..3ec25cdc706152 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_map.h +++ b/be/src/vec/aggregate_functions/aggregate_function_map.h @@ -203,7 +203,7 @@ class AggregateFunctionMapAgg final } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { if (columns[0]->is_nullable()) { const auto& nullable_col = assert_cast(*columns[0]); @@ -234,7 +234,7 @@ class AggregateFunctionMapAgg final void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { + Arena*) const override { this->data(place).merge(this->data(rhs)); } @@ -248,7 +248,7 @@ class AggregateFunctionMapAgg final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& col = assert_cast(*dst); for (size_t i = 0; i != num_rows; ++i) { Field key, value; @@ -263,7 +263,7 @@ class AggregateFunctionMapAgg final } } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { const auto& col = assert_cast(column); auto* data = &(this->data(places)); @@ -282,7 +282,7 @@ class AggregateFunctionMapAgg final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); for (size_t i = 0; i != num_rows; ++i) { @@ -293,7 +293,7 @@ class AggregateFunctionMapAgg final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); const auto& col = assert_cast(column); @@ -304,7 +304,7 @@ class AggregateFunctionMapAgg final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { const auto& col = assert_cast(*column); for (size_t i = 0; i != num_rows; ++i) { @@ -314,8 +314,8 @@ class AggregateFunctionMapAgg final } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { const auto& col = assert_cast(*column); for (size_t i = 0; i != num_rows; ++i) { if (places[i]) { diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max.h b/be/src/vec/aggregate_functions/aggregate_function_min_max.h index a5423cd72f511a..efc2854ff149c8 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_min_max.h +++ b/be/src/vec/aggregate_functions/aggregate_function_min_max.h @@ -104,7 +104,7 @@ struct SingleValueDataFixed { } } - void read(BufferReadable& buf, Arena* arena) { + void read(BufferReadable& buf, Arena*) { read_binary(has_value, buf); if (has()) { read_binary(value, buf); @@ -123,53 +123,53 @@ struct SingleValueDataFixed { value = to.value; } - bool change_if_less(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_less(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast&, TypeCheckOnRelease::DISABLE>(column) .get_data()[row_num] < value) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_less(const Self& to, Arena* arena) { + bool change_if_less(const Self& to, Arena*) { if (to.has() && (!has() || to.value < value)) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - bool change_if_greater(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_greater(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast&, TypeCheckOnRelease::DISABLE>(column) .get_data()[row_num] > value) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_greater(const Self& to, Arena* arena) { + bool change_if_greater(const Self& to, Arena*) { if (to.has() && (!has() || to.value > value)) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - void change_first_time(const IColumn& column, size_t row_num, Arena* arena) { + void change_first_time(const IColumn& column, size_t row_num, Arena*) { if (UNLIKELY(!has())) { - change(column, row_num, arena); + change(column, row_num, nullptr); } } - void change_first_time(const Self& to, Arena* arena) { + void change_first_time(const Self& to, Arena*) { if (UNLIKELY(!has() && to.has())) { - change(to, arena); + change(to, nullptr); } } }; @@ -226,7 +226,7 @@ struct SingleValueDataDecimal { } } - void read(BufferReadable& buf, Arena* arena) { + void read(BufferReadable& buf, Arena*) { read_binary(has_value, buf); if (has()) { read_binary(value, buf); @@ -245,53 +245,53 @@ struct SingleValueDataDecimal { value = to.value; } - bool change_if_less(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_less(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast&, TypeCheckOnRelease::DISABLE>(column) .get_data()[row_num] < value) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_less(const Self& to, Arena* arena) { + bool change_if_less(const Self& to, Arena*) { if (to.has() && (!has() || to.value < value)) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - bool change_if_greater(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_greater(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast&, TypeCheckOnRelease::DISABLE>(column) .get_data()[row_num] > value) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_greater(const Self& to, Arena* arena) { + bool change_if_greater(const Self& to, Arena*) { if (to.has() && (!has() || to.value > value)) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - void change_first_time(const IColumn& column, size_t row_num, Arena* arena) { + void change_first_time(const IColumn& column, size_t row_num, Arena*) { if (UNLIKELY(!has())) { - change(column, row_num, arena); + change(column, row_num, nullptr); } } - void change_first_time(const Self& to, Arena* arena) { + void change_first_time(const Self& to, Arena*) { if (UNLIKELY(!has() && to.has())) { - change(to, arena); + change(to, nullptr); } } }; @@ -349,7 +349,7 @@ struct SingleValueDataString { } } - void read(BufferReadable& buf, Arena* arena) { + void read(BufferReadable& buf, Arena*) { Int32 rhs_size; read_binary(rhs_size, buf); @@ -380,7 +380,7 @@ struct SingleValueDataString { StringRef get_string_ref() const { return StringRef(get_data(), size); } /// Assuming to.has() - void change_impl(StringRef value, Arena* arena) { + void change_impl(StringRef value, Arena*) { Int32 value_size = value.size; if (value_size <= MAX_SMALL_STRING_SIZE) { @@ -402,64 +402,64 @@ struct SingleValueDataString { } } - void change(const IColumn& column, size_t row_num, Arena* arena) { + void change(const IColumn& column, size_t row_num, Arena*) { change_impl( assert_cast(column).get_data_at( row_num), - arena); + nullptr); } - void change(const Self& to, Arena* arena) { change_impl(to.get_string_ref(), arena); } + void change(const Self& to, Arena*) { change_impl(to.get_string_ref(), nullptr); } - bool change_if_less(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_less(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast(column).get_data_at( row_num) < get_string_ref()) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_greater(const IColumn& column, size_t row_num, Arena* arena) { + bool change_if_greater(const IColumn& column, size_t row_num, Arena*) { if (!has() || assert_cast(column).get_data_at( row_num) > get_string_ref()) { - change(column, row_num, arena); + change(column, row_num, nullptr); return true; } else { return false; } } - bool change_if_less(const Self& to, Arena* arena) { + bool change_if_less(const Self& to, Arena*) { if (to.has() && (!has() || to.get_string_ref() < get_string_ref())) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - bool change_if_greater(const Self& to, Arena* arena) { + bool change_if_greater(const Self& to, Arena*) { if (to.has() && (!has() || to.get_string_ref() > get_string_ref())) { - change(to, arena); + change(to, nullptr); return true; } else { return false; } } - void change_first_time(const IColumn& column, size_t row_num, Arena* arena) { + void change_first_time(const IColumn& column, size_t row_num, Arena*) { if (UNLIKELY(!has())) { - change(column, row_num, arena); + change(column, row_num, nullptr); } } - void change_first_time(const Self& to, Arena* arena) { + void change_first_time(const Self& to, Arena*) { if (UNLIKELY(!has() && to.has())) { - change(to, arena); + change(to, nullptr); } } }; @@ -472,15 +472,15 @@ struct AggregateFunctionMaxData : public Data { AggregateFunctionMaxData() { reset(); } - void change_if_better(const IColumn& column, size_t row_num, Arena* arena) { + void change_if_better(const IColumn& column, size_t row_num, Arena*) { if constexpr (Data::IsFixedLength) { this->change_if(column, row_num, false); } else { - this->change_if_greater(column, row_num, arena); + this->change_if_greater(column, row_num, nullptr); } } - void change_if_better(const Self& to, Arena* arena) { this->change_if_greater(to, arena); } + void change_if_better(const Self& to, Arena*) { this->change_if_greater(to, nullptr); } void reset() { if constexpr (Data::IsFixedLength) { @@ -500,14 +500,14 @@ struct AggregateFunctionMinData : Data { AggregateFunctionMinData() { reset(); } - void change_if_better(const IColumn& column, size_t row_num, Arena* arena) { + void change_if_better(const IColumn& column, size_t row_num, Arena*) { if constexpr (Data::IsFixedLength) { this->change_if(column, row_num, true); } else { - this->change_if_less(column, row_num, arena); + this->change_if_less(column, row_num, nullptr); } } - void change_if_better(const Self& to, Arena* arena) { this->change_if_less(to, arena); } + void change_if_better(const Self& to, Arena*) { this->change_if_less(to, nullptr); } void reset() { if constexpr (Data::IsFixedLength) { @@ -525,10 +525,10 @@ struct AggregateFunctionAnyData : Data { using Data::IsFixedLength; constexpr static bool IS_ANY = true; - void change_if_better(const IColumn& column, size_t row_num, Arena* arena) { - this->change_first_time(column, row_num, arena); + void change_if_better(const IColumn& column, size_t row_num, Arena*) { + this->change_first_time(column, row_num, nullptr); } - void change_if_better(const Self& to, Arena* arena) { this->change_first_time(to, arena); } + void change_if_better(const Self& to, Arena*) { this->change_first_time(to, nullptr); } static const char* name() { return "any"; } }; @@ -560,25 +560,25 @@ class AggregateFunctionsSingleValue final DataTypePtr get_return_type() const override { return type; } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { - this->data(place).change_if_better(*columns[0], row_num, arena); + Arena*) const override { + this->data(place).change_if_better(*columns[0], row_num, nullptr); } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { if constexpr (Data::IS_ANY) { DCHECK_GT(batch_size, 0); - this->data(place).change_if_better(*columns[0], 0, arena); + this->data(place).change_if_better(*columns[0], 0, nullptr); } else { - Base::add_batch_single_place(batch_size, place, columns, arena); + Base::add_batch_single_place(batch_size, place, columns, nullptr); } } void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { - this->data(place).change_if_better(this->data(rhs), arena); + Arena*) const override { + this->data(place).change_if_better(this->data(rhs), nullptr); } void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { @@ -586,15 +586,15 @@ class AggregateFunctionsSingleValue final } void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, - Arena* arena) const override { - this->data(place).read(buf, arena); + Arena*) const override { + this->data(place).read(buf, nullptr); } void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { this->data(place).insert_result_into(to); } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { if constexpr (Data::IsFixedLength) { const auto& col = assert_cast(column); @@ -604,7 +604,7 @@ class AggregateFunctionsSingleValue final data[i] = column_data[i]; } } else { - Base::deserialize_from_column(places, column, arena, num_rows); + Base::deserialize_from_column(places, column, nullptr, num_rows); } } @@ -623,63 +623,63 @@ class AggregateFunctionsSingleValue final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { if constexpr (Data::IsFixedLength) { auto& dst_column = assert_cast(*dst); dst_column.resize(num_rows); auto* dst_data = reinterpret_cast(dst_column.get_data().data()); for (size_t i = 0; i != num_rows; ++i) { - dst_data[i].change(*columns[0], i, arena); + dst_data[i].change(*columns[0], i, nullptr); } } else { - Base::streaming_agg_serialize_to_column(columns, dst, num_rows, arena); + Base::streaming_agg_serialize_to_column(columns, dst, num_rows, nullptr); } } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { if constexpr (Data::IsFixedLength) { const auto& col = assert_cast(column); auto* column_data = reinterpret_cast(col.get_data().data()); const size_t num_rows = column.size(); for (size_t i = 0; i != num_rows; ++i) { - this->data(place).change_if_better(column_data[i], arena); + this->data(place).change_if_better(column_data[i], nullptr); } } else { - Base::deserialize_and_merge_from_column(place, column, arena); + Base::deserialize_and_merge_from_column(place, column, nullptr); } } void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { if constexpr (Data::IsFixedLength) { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); auto* data = reinterpret_cast(col.get_data().data()); for (size_t i = begin; i <= end; ++i) { - this->data(place).change_if_better(data[i], arena); + this->data(place).change_if_better(data[i], nullptr); } } else { - Base::deserialize_and_merge_from_column_range(place, column, begin, end, arena); + Base::deserialize_and_merge_from_column_range(place, column, begin, end, nullptr); } } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max_by.h b/be/src/vec/aggregate_functions/aggregate_function_min_max_by.h index 4caded0011a81b..5c73ac9aa67cbe 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_min_max_by.h +++ b/be/src/vec/aggregate_functions/aggregate_function_min_max_by.h @@ -64,7 +64,7 @@ struct BitmapValueData { } } - void read(BufferReadable& buf, Arena* arena) { + void read(BufferReadable& buf, Arena*) { read_binary(has_value, buf); if (has()) { DataTypeBitMap::deserialize_as_stream(value, buf); @@ -101,9 +101,9 @@ struct AggregateFunctionMinMaxByBaseData { key.write(buf); } - void read(BufferReadable& buf, Arena* arena) { - value.read(buf, arena); - key.read(buf, arena); + void read(BufferReadable& buf, Arena*) { + value.read(buf, nullptr); + key.read(buf, nullptr); } }; @@ -111,15 +111,15 @@ template struct AggregateFunctionMaxByData : public AggregateFunctionMinMaxByBaseData { using Self = AggregateFunctionMaxByData; void change_if_better(const IColumn& value_column, const IColumn& key_column, size_t row_num, - Arena* arena) { - if (this->key.change_if_greater(key_column, row_num, arena)) { - this->value.change(value_column, row_num, arena); + Arena*) { + if (this->key.change_if_greater(key_column, row_num, nullptr)) { + this->value.change(value_column, row_num, nullptr); } } - void change_if_better(const Self& to, Arena* arena) { - if (this->key.change_if_greater(to.key, arena)) { - this->value.change(to.value, arena); + void change_if_better(const Self& to, Arena*) { + if (this->key.change_if_greater(to.key, nullptr)) { + this->value.change(to.value, nullptr); } } @@ -130,15 +130,15 @@ template struct AggregateFunctionMinByData : public AggregateFunctionMinMaxByBaseData { using Self = AggregateFunctionMinByData; void change_if_better(const IColumn& value_column, const IColumn& key_column, size_t row_num, - Arena* arena) { - if (this->key.change_if_less(key_column, row_num, arena)) { - this->value.change(value_column, row_num, arena); + Arena*) { + if (this->key.change_if_less(key_column, row_num, nullptr)) { + this->value.change(value_column, row_num, nullptr); } } - void change_if_better(const Self& to, Arena* arena) { - if (this->key.change_if_less(to.key, arena)) { - this->value.change(to.value, arena); + void change_if_better(const Self& to, Arena*) { + if (this->key.change_if_less(to.key, nullptr)) { + this->value.change(to.value, nullptr); } } @@ -169,15 +169,15 @@ class AggregateFunctionsMinMaxBy final DataTypePtr get_return_type() const override { return value_type; } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { - this->data(place).change_if_better(*columns[0], *columns[1], row_num, arena); + Arena*) const override { + this->data(place).change_if_better(*columns[0], *columns[1], row_num, nullptr); } void reset(AggregateDataPtr place) const override { this->data(place).reset(); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { - this->data(place).change_if_better(this->data(rhs), arena); + Arena*) const override { + this->data(place).change_if_better(this->data(rhs), nullptr); } void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& buf) const override { @@ -185,8 +185,8 @@ class AggregateFunctionsMinMaxBy final } void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, - Arena* arena) const override { - this->data(place).read(buf, arena); + Arena*) const override { + this->data(place).read(buf, nullptr); } void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& to) const override { diff --git a/be/src/vec/aggregate_functions/aggregate_function_percentile.h b/be/src/vec/aggregate_functions/aggregate_function_percentile.h index a1e739d8758fa7..0766c59f3de1c3 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_percentile.h +++ b/be/src/vec/aggregate_functions/aggregate_function_percentile.h @@ -433,7 +433,7 @@ class AggregateFunctionPercentile final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { const auto& sources = assert_cast(*columns[0]); const auto& quantile = diff --git a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h index 60ab42b5298e8e..1a6ac288583b3e 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h +++ b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h @@ -223,7 +223,7 @@ class ReaderFunctionData final } void add(AggregateDataPtr place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { this->data(place).add(row_num, columns); } @@ -231,7 +231,7 @@ class ReaderFunctionData final void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "ReaderFunctionData do not support add_range_single_place"); __builtin_unreachable(); diff --git a/be/src/vec/aggregate_functions/aggregate_function_rpc.h b/be/src/vec/aggregate_functions/aggregate_function_rpc.h index c92e96aaf9d935..f055d2c8c103a0 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_rpc.h +++ b/be/src/vec/aggregate_functions/aggregate_function_rpc.h @@ -364,7 +364,7 @@ class AggregateRpcUdaf final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { static_cast(this->data(place).add(columns, 0, batch_size, argument_types)); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_sort.h b/be/src/vec/aggregate_functions/aggregate_function_sort.h index 145a07d5446b5c..981580f8e6ac38 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sort.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sort.h @@ -142,12 +142,12 @@ class AggregateFunctionSort } void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { this->data(place).add(columns, _arguments.size(), row_num); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, - Arena* arena) const override { + Arena*) const override { this->data(place).merge(this->data(rhs)); } @@ -156,7 +156,7 @@ class AggregateFunctionSort } void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, - Arena* arena) const override { + Arena*) const override { this->data(place).deserialize(buf); } diff --git a/be/src/vec/aggregate_functions/aggregate_function_sum.h b/be/src/vec/aggregate_functions/aggregate_function_sum.h index 846104915b1e69..13fb3864bd1aaf 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_sum.h +++ b/be/src/vec/aggregate_functions/aggregate_function_sum.h @@ -126,7 +126,7 @@ class AggregateFunctionSum final column.get_data().push_back(this->data(place).get()); } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto& col = assert_cast(column); auto* data = col.get_data().data(); @@ -147,7 +147,7 @@ class AggregateFunctionSum final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& col = assert_cast(*dst); auto& src = assert_cast(*columns[0]); DCHECK(col.item_size() == sizeof(Data)) @@ -162,7 +162,7 @@ class AggregateFunctionSum final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); auto* data = reinterpret_cast(col.get_data().data()); @@ -173,7 +173,7 @@ class AggregateFunctionSum final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { DCHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -184,19 +184,19 @@ class AggregateFunctionSum final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h index e97923a08e6a2d..a3bdad635057fd 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h @@ -141,7 +141,7 @@ class AggregateFunctionUniq final } void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, - const IColumn** columns, Arena* arena, bool /*agg_many*/) const override { + const IColumn** columns, Arena*, bool /*agg_many*/) const override { std::vector keys_container; const KeyType* keys = get_keys(keys_container, *columns[0], batch_size); @@ -175,7 +175,7 @@ class AggregateFunctionUniq final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { std::vector keys_container; const KeyType* keys = get_keys(keys_container, *columns[0], batch_size); auto& set = this->data(place).set; @@ -197,7 +197,7 @@ class AggregateFunctionUniq final } void deserialize_and_merge(AggregateDataPtr __restrict place, AggregateDataPtr __restrict rhs, - BufferReadable& buf, Arena* arena) const override { + BufferReadable& buf, Arena*) const override { auto& set = this->data(place).set; UInt64 size; read_var_uint(size, buf); @@ -212,7 +212,7 @@ class AggregateFunctionUniq final } void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf, - Arena* arena) const override { + Arena*) const override { auto& set = this->data(place).set; UInt64 size; read_var_uint(size, buf); diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.h b/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.h index 4c3fa67e1626ae..90d137c62384f6 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.h +++ b/be/src/vec/aggregate_functions/aggregate_function_uniq_distribute_key.h @@ -112,7 +112,7 @@ class AggregateFunctionUniqDistributeKey final } void add_batch(size_t batch_size, AggregateDataPtr* places, size_t place_offset, - const IColumn** columns, Arena* arena, bool /*agg_many*/) const override { + const IColumn** columns, Arena*, bool /*agg_many*/) const override { std::vector keys_container; const KeyType* keys = get_keys(keys_container, *columns[0], batch_size); @@ -133,7 +133,7 @@ class AggregateFunctionUniqDistributeKey final } void add_batch_single_place(size_t batch_size, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { std::vector keys_container; const KeyType* keys = get_keys(keys_container, *columns[0], batch_size); auto& set = this->data(place).set; @@ -164,7 +164,7 @@ class AggregateFunctionUniqDistributeKey final assert_cast(to).get_data().push_back(this->data(place).count); } - void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena* arena, + void deserialize_from_column(AggregateDataPtr places, const IColumn& column, Arena*, size_t num_rows) const override { auto data = reinterpret_cast( assert_cast(column).get_data().data()); @@ -188,7 +188,7 @@ class AggregateFunctionUniqDistributeKey final } void streaming_agg_serialize_to_column(const IColumn** columns, MutableColumnPtr& dst, - const size_t num_rows, Arena* arena) const override { + const size_t num_rows, Arena*) const override { auto& dst_col = assert_cast(*dst); CHECK(dst_col.item_size() == sizeof(UInt64)) << "size is not equal: " << dst_col.item_size() << " " << sizeof(UInt64); @@ -200,7 +200,7 @@ class AggregateFunctionUniqDistributeKey final } void deserialize_and_merge_from_column(AggregateDataPtr __restrict place, const IColumn& column, - Arena* arena) const override { + Arena*) const override { auto& col = assert_cast(column); const size_t num_rows = column.size(); auto* data = reinterpret_cast(col.get_data().data()); @@ -211,7 +211,7 @@ class AggregateFunctionUniqDistributeKey final void deserialize_and_merge_from_column_range(AggregateDataPtr __restrict place, const IColumn& column, size_t begin, size_t end, - Arena* arena) const override { + Arena*) const override { CHECK(end <= column.size() && begin <= end) << ", begin:" << begin << ", end:" << end << ", column.size():" << column.size(); auto& col = assert_cast(column); @@ -222,19 +222,19 @@ class AggregateFunctionUniqDistributeKey final } void deserialize_and_merge_vec(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, Arena* arena, + AggregateDataPtr rhs, const IColumn* column, Arena*, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec(places, offset, rhs, arena, num_rows); + this->merge_vec(places, offset, rhs, nullptr, num_rows); } void deserialize_and_merge_vec_selected(const AggregateDataPtr* places, size_t offset, - AggregateDataPtr rhs, const IColumn* column, - Arena* arena, const size_t num_rows) const override { - this->deserialize_from_column(rhs, *column, arena, num_rows); + AggregateDataPtr rhs, const IColumn* column, Arena*, + const size_t num_rows) const override { + this->deserialize_from_column(rhs, *column, nullptr, num_rows); DEFER({ this->destroy_vec(rhs, num_rows); }); - this->merge_vec_selected(places, offset, rhs, arena, num_rows); + this->merge_vec_selected(places, offset, rhs, nullptr, num_rows); } void serialize_without_key_to_column(ConstAggregateDataPtr __restrict place, diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h index 517871e2fb642d..cb038fe31168b9 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_window.h +++ b/be/src/vec/aggregate_functions/aggregate_function_window.h @@ -66,7 +66,7 @@ class WindowFunctionRowNumber final void add_range_single_place(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end, AggregateDataPtr place, const IColumn** columns, - Arena* arena) const override { + Arena*) const override { ++data(place).count; } @@ -104,7 +104,7 @@ class WindowFunctionRank final : public IAggregateFunctionDataHelperdata(place).add_range_single_place(partition_start, partition_end, frame_start, frame_end, columns); } @@ -554,7 +554,7 @@ class WindowFunctionData final } void add(AggregateDataPtr place, const IColumn** columns, ssize_t row_num, - Arena* arena) const override { + Arena*) const override { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "WindowFunctionLeadLagData do not support add"); __builtin_unreachable(); diff --git a/be/src/vec/exec/scan/new_es_scanner.cpp b/be/src/vec/exec/scan/new_es_scanner.cpp index d59aebd98c7341..fae83854be0910 100644 --- a/be/src/vec/exec/scan/new_es_scanner.cpp +++ b/be/src/vec/exec/scan/new_es_scanner.cpp @@ -169,8 +169,7 @@ Status NewEsScanner::_get_block_impl(RuntimeState* state, Block* block, bool* eo } Status NewEsScanner::_get_next(std::vector& columns) { - auto read_timer = _local_state->cast()._read_timer; - SCOPED_TIMER(read_timer); + SCOPED_TIMER(_local_state->cast()._read_timer); if (_line_eof && _batch_eof) { _es_eof = true; return Status::OK(); @@ -185,12 +184,8 @@ Status NewEsScanner::_get_next(std::vector& column } } - auto rows_read_counter = - _local_state->cast()._rows_read_counter; - auto materialize_timer = - _local_state->cast()._materialize_timer; - COUNTER_UPDATE(rows_read_counter, 1); - SCOPED_TIMER(materialize_timer); + COUNTER_UPDATE(_local_state->cast()._blocks_read_counter, 1); + SCOPED_TIMER(_local_state->cast()._materialize_timer); RETURN_IF_ERROR(_es_scroll_parser->fill_columns(_tuple_desc, columns, &_line_eof, _docvalue_context, _state->timezone_obj())); if (!_line_eof) { diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index 60240618655322..f40f30f5b16e67 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -226,8 +226,7 @@ Status NewOlapScanner::init() { Status NewOlapScanner::open(RuntimeState* state) { RETURN_IF_ERROR(VScanner::open(state)); - auto* timer = ((pipeline::OlapScanLocalState*)_local_state)->_reader_init_timer; - SCOPED_TIMER(timer); + SCOPED_TIMER(_local_state->cast()._reader_init_timer); auto res = _tablet_reader->init(_tablet_reader_params); if (!res.ok()) { @@ -543,11 +542,9 @@ void NewOlapScanner::_update_realtime_counters() { const OlapReaderStatistics& stats = _tablet_reader->stats(); COUNTER_UPDATE(local_state->_read_compressed_counter, stats.compressed_bytes_read); COUNTER_UPDATE(local_state->_scan_bytes, stats.compressed_bytes_read); - _scan_bytes += stats.compressed_bytes_read; _tablet_reader->mutable_stats()->compressed_bytes_read = 0; COUNTER_UPDATE(local_state->_scan_rows, stats.raw_rows_read); - _scan_rows += stats.raw_rows_read; // if raw_rows_read is reset, scanNode will scan all table rows which may cause BE crash _tablet_reader->mutable_stats()->raw_rows_read = 0; } @@ -562,97 +559,92 @@ void NewOlapScanner::_collect_profile_before_close() { VScanner::_collect_profile_before_close(); #ifndef INCR_COUNTER -#define INCR_COUNTER(Parent) \ - COUNTER_UPDATE(Parent->_io_timer, stats.io_ns); \ - COUNTER_UPDATE(Parent->_read_compressed_counter, stats.compressed_bytes_read); \ - COUNTER_UPDATE(Parent->_scan_bytes, stats.compressed_bytes_read); \ - _scan_bytes += 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->_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->_delete_bitmap_get_agg_timer, stats.delete_bitmap_get_agg_ns); \ - COUNTER_UPDATE(Parent->_block_convert_timer, stats.block_convert_ns); \ - COUNTER_UPDATE(Parent->_scan_rows, stats.raw_rows_read); \ - _scan_rows += _tablet_reader->mutable_stats()->raw_rows_read; \ - COUNTER_UPDATE(Parent->_vec_cond_timer, stats.vec_cond_ns); \ - COUNTER_UPDATE(Parent->_short_cond_timer, stats.short_cond_ns); \ - COUNTER_UPDATE(Parent->_expr_filter_timer, stats.expr_filter_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->_block_conditions_filtered_timer, stats.block_conditions_filtered_ns); \ - COUNTER_UPDATE(Parent->_block_conditions_filtered_bf_timer, \ - stats.block_conditions_filtered_bf_ns); \ - COUNTER_UPDATE(Parent->_collect_iterator_merge_next_timer, \ - stats.collect_iterator_merge_next_timer); \ - COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_timer, \ - stats.block_conditions_filtered_zonemap_ns); \ - COUNTER_UPDATE(Parent->_block_conditions_filtered_zonemap_rp_timer, \ - stats.block_conditions_filtered_zonemap_rp_ns); \ - COUNTER_UPDATE(Parent->_block_conditions_filtered_dict_timer, \ - stats.block_conditions_filtered_dict_ns); \ - COUNTER_UPDATE(Parent->_first_read_timer, stats.first_read_ns); \ - COUNTER_UPDATE(Parent->_second_read_timer, stats.second_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_filtered_counter, stats.rows_vec_cond_filtered); \ - COUNTER_UPDATE(Parent->_rows_short_circuit_cond_filtered_counter, \ - stats.rows_short_circuit_cond_filtered); \ - COUNTER_UPDATE(Parent->_rows_vec_cond_input_counter, stats.vec_cond_input_rows); \ - COUNTER_UPDATE(Parent->_rows_short_circuit_cond_input_counter, \ - stats.short_circuit_cond_input_rows); \ - for (auto& [id, info] : stats.filter_info) { \ - Parent->add_filter_info(id, info); \ - } \ - COUNTER_UPDATE(Parent->_stats_filtered_counter, stats.rows_stats_filtered); \ - COUNTER_UPDATE(Parent->_stats_rp_filtered_counter, stats.rows_stats_rp_filtered); \ - COUNTER_UPDATE(Parent->_dict_filtered_counter, stats.rows_dict_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->_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->_inverted_index_filter_counter, stats.rows_inverted_index_filtered); \ - COUNTER_UPDATE(Parent->_inverted_index_filter_timer, stats.inverted_index_filter_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_query_cache_hit_counter, \ - stats.inverted_index_query_cache_hit); \ - COUNTER_UPDATE(Parent->_inverted_index_query_cache_miss_counter, \ - stats.inverted_index_query_cache_miss); \ - COUNTER_UPDATE(Parent->_inverted_index_query_timer, stats.inverted_index_query_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_query_null_bitmap_timer, \ - stats.inverted_index_query_null_bitmap_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_query_bitmap_copy_timer, \ - stats.inverted_index_query_bitmap_copy_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_query_bitmap_op_timer, \ - stats.inverted_index_query_bitmap_op_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_searcher_open_timer, \ - stats.inverted_index_searcher_open_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_searcher_search_timer, \ - stats.inverted_index_searcher_search_timer); \ - COUNTER_UPDATE(Parent->_inverted_index_searcher_cache_hit_counter, \ - stats.inverted_index_searcher_cache_hit); \ - COUNTER_UPDATE(Parent->_inverted_index_searcher_cache_miss_counter, \ - stats.inverted_index_searcher_cache_miss); \ - COUNTER_UPDATE(Parent->_inverted_index_downgrade_count_counter, \ - stats.inverted_index_downgrade_count); \ - if (config::enable_file_cache) { \ - io::FileCacheProfileReporter cache_profile(Parent->_segment_profile.get()); \ - cache_profile.update(&stats.file_cache_stats); \ - } \ - COUNTER_UPDATE(Parent->_output_index_result_column_timer, \ - stats.output_index_result_column_timer); \ - COUNTER_UPDATE(Parent->_filtered_segment_counter, stats.filtered_segment_number); \ +#define INCR_COUNTER(Parent) \ + COUNTER_UPDATE(Parent->_io_timer, stats.io_ns); \ + COUNTER_UPDATE(Parent->_read_compressed_counter, stats.compressed_bytes_read); \ + COUNTER_UPDATE(Parent->_scan_bytes, 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->_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->_delete_bitmap_get_agg_timer, stats.delete_bitmap_get_agg_ns); \ + COUNTER_UPDATE(Parent->_scan_rows, stats.raw_rows_read); \ + COUNTER_UPDATE(Parent->_vec_cond_timer, stats.vec_cond_ns); \ + COUNTER_UPDATE(Parent->_short_cond_timer, stats.short_cond_ns); \ + COUNTER_UPDATE(Parent->_expr_filter_timer, stats.expr_filter_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->_segment_generate_row_range_timer, stats.generate_row_ranges_ns); \ + COUNTER_UPDATE(Parent->_segment_generate_row_range_by_bf_timer, \ + stats.generate_row_ranges_by_bf_ns); \ + COUNTER_UPDATE(Parent->_collect_iterator_merge_next_timer, \ + stats.collect_iterator_merge_next_timer); \ + COUNTER_UPDATE(Parent->_segment_generate_row_range_by_zonemap_timer, \ + stats.generate_row_ranges_by_zonemap_ns); \ + COUNTER_UPDATE(Parent->_segment_generate_row_range_by_dict_timer, \ + stats.generate_row_ranges_by_dict_ns); \ + COUNTER_UPDATE(Parent->_predicate_column_read_timer, stats.predicate_column_read_ns); \ + COUNTER_UPDATE(Parent->_non_predicate_column_read_timer, stats.non_predicate_read_ns); \ + COUNTER_UPDATE(Parent->_predicate_column_read_seek_timer, \ + stats.predicate_column_read_seek_ns); \ + COUNTER_UPDATE(Parent->_predicate_column_read_seek_counter, \ + stats.predicate_column_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_filtered_counter, stats.rows_vec_cond_filtered); \ + COUNTER_UPDATE(Parent->_rows_short_circuit_cond_filtered_counter, \ + stats.rows_short_circuit_cond_filtered); \ + COUNTER_UPDATE(Parent->_rows_vec_cond_input_counter, stats.vec_cond_input_rows); \ + COUNTER_UPDATE(Parent->_rows_short_circuit_cond_input_counter, \ + stats.short_circuit_cond_input_rows); \ + for (auto& [id, info] : stats.filter_info) { \ + Parent->add_filter_info(id, info); \ + } \ + COUNTER_UPDATE(Parent->_stats_filtered_counter, stats.rows_stats_filtered); \ + COUNTER_UPDATE(Parent->_stats_rp_filtered_counter, stats.rows_stats_rp_filtered); \ + COUNTER_UPDATE(Parent->_dict_filtered_counter, stats.rows_dict_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->_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->_inverted_index_filter_counter, stats.rows_inverted_index_filtered); \ + COUNTER_UPDATE(Parent->_inverted_index_filter_timer, stats.inverted_index_filter_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_query_cache_hit_counter, \ + stats.inverted_index_query_cache_hit); \ + COUNTER_UPDATE(Parent->_inverted_index_query_cache_miss_counter, \ + stats.inverted_index_query_cache_miss); \ + COUNTER_UPDATE(Parent->_inverted_index_query_timer, stats.inverted_index_query_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_query_null_bitmap_timer, \ + stats.inverted_index_query_null_bitmap_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_query_bitmap_copy_timer, \ + stats.inverted_index_query_bitmap_copy_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_searcher_open_timer, \ + stats.inverted_index_searcher_open_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_searcher_search_timer, \ + stats.inverted_index_searcher_search_timer); \ + COUNTER_UPDATE(Parent->_inverted_index_searcher_cache_hit_counter, \ + stats.inverted_index_searcher_cache_hit); \ + COUNTER_UPDATE(Parent->_inverted_index_searcher_cache_miss_counter, \ + stats.inverted_index_searcher_cache_miss); \ + COUNTER_UPDATE(Parent->_inverted_index_downgrade_count_counter, \ + stats.inverted_index_downgrade_count); \ + if (config::enable_file_cache) { \ + io::FileCacheProfileReporter cache_profile(Parent->_segment_profile.get()); \ + cache_profile.update(&stats.file_cache_stats); \ + } \ + COUNTER_UPDATE(Parent->_output_index_result_column_timer, \ + stats.output_index_result_column_timer); \ + COUNTER_UPDATE(Parent->_filtered_segment_counter, stats.filtered_segment_number); \ COUNTER_UPDATE(Parent->_total_segment_counter, stats.total_segment_number); // Update counters for NewOlapScanner @@ -665,11 +657,12 @@ void NewOlapScanner::_collect_profile_before_close() { #undef INCR_COUNTER #endif // Update metrics - DorisMetrics::instance()->query_scan_bytes->increment(_scan_bytes); - DorisMetrics::instance()->query_scan_rows->increment(_scan_rows); + DorisMetrics::instance()->query_scan_bytes->increment( + local_state->_read_compressed_counter->value()); + DorisMetrics::instance()->query_scan_rows->increment(local_state->_scan_rows->value()); auto& tablet = _tablet_reader_params.tablet; - tablet->query_scan_bytes->increment(_scan_bytes); - tablet->query_scan_rows->increment(_scan_rows); + tablet->query_scan_bytes->increment(local_state->_read_compressed_counter->value()); + tablet->query_scan_rows->increment(local_state->_scan_rows->value()); tablet->query_scan_count->increment(1); if (_query_statistics) { _query_statistics->add_scan_bytes_from_local_storage( diff --git a/be/src/vec/exec/scan/new_olap_scanner.h b/be/src/vec/exec/scan/new_olap_scanner.h index 44c300f446e6ea..fd1246b120ba77 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.h +++ b/be/src/vec/exec/scan/new_olap_scanner.h @@ -101,8 +101,6 @@ class NewOlapScanner : public VScanner { std::unordered_set _tablet_columns_convert_to_null_set; // ========= profiles ========== - int64_t _scan_bytes = 0; - int64_t _scan_rows = 0; bool _profile_updated = false; }; } // namespace vectorized diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index ee1d60d2902424..bea222bd0f35b0 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -80,8 +80,6 @@ Status ScannerContext::init() { _scanner_profile = _local_state->_scanner_profile; _scanner_sched_counter = _local_state->_scanner_sched_counter; _newly_create_free_blocks_num = _local_state->_newly_create_free_blocks_num; - _scanner_wait_batch_timer = _local_state->_scanner_wait_batch_timer; - _scanner_ctx_sched_time = _local_state->_scanner_ctx_sched_time; _scale_up_scanners_counter = _local_state->_scale_up_scanners_counter; _scanner_memory_used_counter = _local_state->_memory_used_counter; @@ -224,10 +222,6 @@ Status ScannerContext::init() { return Status::OK(); } -std::string ScannerContext::parent_name() { - return _local_state->get_name(); -} - vectorized::BlockUPtr ScannerContext::get_free_block(bool force) { vectorized::BlockUPtr block = nullptr; if (_free_blocks.try_dequeue(block)) { @@ -257,18 +251,13 @@ void ScannerContext::return_free_block(vectorized::BlockUPtr block) { } } -bool ScannerContext::empty_in_queue(int id) { - std::lock_guard l(_transfer_lock); - return _blocks_queue.empty(); -} - Status ScannerContext::submit_scan_task(std::shared_ptr scan_task) { _scanner_sched_counter->update(1); _num_scheduled_scanners++; return _scanner_scheduler_global->submit(shared_from_this(), scan_task); } -void ScannerContext::append_block_to_queue(std::shared_ptr scan_task) { +void ScannerContext::push_back_scan_task(std::shared_ptr scan_task) { if (scan_task->status_ok()) { for (const auto& [block, _] : scan_task->cached_blocks) { if (block->rows() > 0) { @@ -287,12 +276,12 @@ void ScannerContext::append_block_to_queue(std::shared_ptr scan_task) if (_last_scale_up_time == 0) { _last_scale_up_time = UnixMillis(); } - if (_blocks_queue.empty() && _last_fetch_time != 0) { + if (_tasks_queue.empty() && _last_fetch_time != 0) { // there's no block in queue before current block, so the consumer is waiting _total_wait_block_time += UnixMillis() - _last_fetch_time; } _num_scheduled_scanners--; - _blocks_queue.emplace_back(scan_task); + _tasks_queue.emplace_back(scan_task); _dependency->set_ready(); } @@ -308,9 +297,9 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo _set_scanner_done(); return _process_status; } - if (!_blocks_queue.empty() && !done()) { + if (!_tasks_queue.empty() && !done()) { _last_fetch_time = UnixMillis(); - auto scan_task = _blocks_queue.front(); + auto scan_task = _tasks_queue.front(); DCHECK(scan_task); // The abnormal status of scanner may come from the execution of the scanner itself, @@ -335,7 +324,7 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo return_free_block(std::move(current_block)); } else { // This scan task do not have any cached blocks. - _blocks_queue.pop_front(); + _tasks_queue.pop_front(); // current scanner is finished, and no more data to read if (scan_task->is_eos()) { _num_finished_scanners++; @@ -374,13 +363,13 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo RETURN_IF_ERROR(_try_to_scale_up()); } - if (_num_finished_scanners == _all_scanners.size() && _blocks_queue.empty()) { + if (_num_finished_scanners == _all_scanners.size() && _tasks_queue.empty()) { _set_scanner_done(); _is_finished = true; } *eos = done(); - if (_blocks_queue.empty()) { + if (_tasks_queue.empty()) { _dependency->block(); } return Status::OK(); @@ -466,11 +455,6 @@ Status ScannerContext::validate_block_schema(Block* block) { return Status::OK(); } -void ScannerContext::set_status_on_error(const Status& status) { - std::lock_guard l(_transfer_lock); - _process_status = status; -} - void ScannerContext::stop_scanners(RuntimeState* state) { std::lock_guard l(_transfer_lock); if (_should_stop) { @@ -483,7 +467,7 @@ void ScannerContext::stop_scanners(RuntimeState* state) { sc->_scanner->try_stop(); } } - _blocks_queue.clear(); + _tasks_queue.clear(); // TODO yiguolei, call mark close to scanners if (state->enable_profile()) { std::stringstream scanner_statistics; @@ -533,11 +517,11 @@ void ScannerContext::stop_scanners(RuntimeState* state) { std::string ScannerContext::debug_string() { return fmt::format( - "id: {}, total scanners: {}, blocks in queue: {}," + "id: {}, total scanners: {}, pending tasks: {}," " _should_stop: {}, _is_finished: {}, free blocks: {}," " limit: {}, _num_running_scanners: {}, _max_thread_num: {}," " _max_bytes_in_queue: {}, query_id: {}", - ctx_id, _all_scanners.size(), _blocks_queue.size(), _should_stop, _is_finished, + ctx_id, _all_scanners.size(), _tasks_queue.size(), _should_stop, _is_finished, _free_blocks.size_approx(), limit, _num_scheduled_scanners, _max_thread_num, _max_bytes_in_queue, print_id(_query_id)); } diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 77f6fc541b8495..8a42bc037caf0f 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -75,7 +75,6 @@ class ScanTask { public: std::weak_ptr scanner; std::list> cached_blocks; - uint64_t last_submit_time; // nanoseconds void set_status(Status _status) { if (_status.is()) { @@ -112,7 +111,7 @@ class ScannerContext : public std::enable_shared_from_this, ~ScannerContext() override { SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_thread_context.query_mem_tracker); - _blocks_queue.clear(); + _tasks_queue.clear(); vectorized::BlockUPtr block; while (_free_blocks.try_dequeue(block)) { // do nothing @@ -143,37 +142,25 @@ class ScannerContext : public std::enable_shared_from_this, // set the `eos` to `ScanTask::eos` if there is no more data in current scanner Status submit_scan_task(std::shared_ptr scan_task); - // append the running scanner and its cached block to `_blocks_queue` - void append_block_to_queue(std::shared_ptr scan_task); - - void set_status_on_error(const Status& status); + // Push back a scan task. + void push_back_scan_task(std::shared_ptr scan_task); // Return true if this ScannerContext need no more process bool done() const { return _is_finished || _should_stop; } - bool is_finished() { return _is_finished.load(); } - bool should_stop() { return _should_stop.load(); } std::string debug_string(); RuntimeState* state() { return _state; } - void incr_ctx_scheduling_time(int64_t num) { _scanner_ctx_sched_time->update(num); } - std::string parent_name(); - - bool empty_in_queue(int id); SimplifiedScanScheduler* get_scan_scheduler() { return _scanner_scheduler; } void stop_scanners(RuntimeState* state); - int32_t get_max_thread_num() const { return _max_thread_num; } - void set_max_thread_num(int32_t num) { _max_thread_num = num; } - int batch_size() const { return _batch_size; } // the unique id of this context std::string ctx_id; TUniqueId _query_id; - int32_t queue_idx = -1; ThreadPoolToken* thread_token = nullptr; bool _should_reset_thread_name = true; @@ -195,7 +182,7 @@ class ScannerContext : public std::enable_shared_from_this, const RowDescriptor* _output_row_descriptor = nullptr; std::mutex _transfer_lock; - std::list> _blocks_queue; + std::list> _tasks_queue; Status _process_status = Status::OK(); std::atomic_bool _should_stop = false; @@ -223,8 +210,6 @@ class ScannerContext : public std::enable_shared_from_this, // This counter refers to scan operator's local state RuntimeProfile::Counter* _scanner_memory_used_counter = nullptr; RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; - RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; - RuntimeProfile::Counter* _scanner_ctx_sched_time = nullptr; RuntimeProfile::Counter* _scale_up_scanners_counter = nullptr; QueryThreadContext _query_thread_context; std::shared_ptr _dependency = nullptr; diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 3ad4e758e79980..385b581d2a5725 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -123,7 +123,6 @@ Status ScannerScheduler::init(ExecEnv* env) { Status ScannerScheduler::submit(std::shared_ptr ctx, std::shared_ptr scan_task) { - scan_task->last_submit_time = GetCurrentTimeNanos(); if (ctx->done()) { return Status::OK(); } @@ -154,7 +153,7 @@ Status ScannerScheduler::submit(std::shared_ptr ctx, if (!status.ok()) { scanner_ref->set_status(status); - ctx->append_block_to_queue(scanner_ref); + ctx->push_back_scan_task(scanner_ref); } }); if (!s.ok()) { @@ -184,7 +183,7 @@ Status ScannerScheduler::submit(std::shared_ptr ctx, if (!status.ok()) { scanner_ref->set_status(status); - ctx->append_block_to_queue(scanner_ref); + ctx->push_back_scan_task(scanner_ref); } }; SimplifiedScanTask simple_scan_task = {work_func, ctx}; @@ -212,8 +211,6 @@ std::unique_ptr ScannerScheduler::new_limited_scan_pool_token( void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, std::shared_ptr scan_task) { - // record the time from scanner submission to actual execution in nanoseconds - ctx->incr_ctx_scheduling_time(GetCurrentTimeNanos() - scan_task->last_submit_time); auto task_lock = ctx->task_exec_ctx(); if (task_lock == nullptr) { return; @@ -343,7 +340,7 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, scanner->mark_to_need_to_close(); } scan_task->set_eos(eos); - ctx->append_block_to_queue(scan_task); + ctx->push_back_scan_task(scan_task); } void ScannerScheduler::_register_metrics() { diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index d767df3d2165c9..7862a116193f49 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -126,8 +126,6 @@ Status VFileScanner::prepare(RuntimeState* state, const VExprContextSPtrs& conju _open_reader_timer = ADD_TIMER(_local_state->scanner_profile(), "FileScannerOpenReaderTime"); _cast_to_input_block_timer = ADD_TIMER(_local_state->scanner_profile(), "FileScannerCastInputBlockTime"); - _fill_path_columns_timer = - ADD_TIMER(_local_state->scanner_profile(), "FileScannerFillPathColumnTime"); _fill_missing_columns_timer = ADD_TIMER(_local_state->scanner_profile(), "FileScannerFillMissingColumnTime"); _pre_filter_timer = ADD_TIMER(_local_state->scanner_profile(), "FileScannerPreFilterTimer"); @@ -137,8 +135,6 @@ Status VFileScanner::prepare(RuntimeState* state, const VExprContextSPtrs& conju _not_found_file_counter = ADD_COUNTER(_local_state->scanner_profile(), "NotFoundFileNum", TUnit::UNIT); _file_counter = ADD_COUNTER(_local_state->scanner_profile(), "FileNumber", TUnit::UNIT); - _has_fully_rf_file_counter = - ADD_COUNTER(_local_state->scanner_profile(), "HasFullyRfFileNumber", TUnit::UNIT); _file_cache_statistics.reset(new io::FileCacheStatistics()); _io_ctx.reset(new io::IOContext()); @@ -219,7 +215,7 @@ Status VFileScanner::_process_late_arrival_conjuncts() { _discard_conjuncts(); } if (_applied_rf_num == _total_rf_num) { - COUNTER_UPDATE(_has_fully_rf_file_counter, 1); + _local_state->scanner_profile()->add_info_string("ApplyAllRuntimeFilters", "True"); } return Status::OK(); } diff --git a/be/src/vec/exec/scan/vfile_scanner.h b/be/src/vec/exec/scan/vfile_scanner.h index 662e797e4aaf86..8cb288ac81bf83 100644 --- a/be/src/vec/exec/scan/vfile_scanner.h +++ b/be/src/vec/exec/scan/vfile_scanner.h @@ -171,14 +171,12 @@ class VFileScanner : public VScanner { RuntimeProfile::Counter* _get_block_timer = nullptr; RuntimeProfile::Counter* _open_reader_timer = nullptr; RuntimeProfile::Counter* _cast_to_input_block_timer = nullptr; - RuntimeProfile::Counter* _fill_path_columns_timer = nullptr; RuntimeProfile::Counter* _fill_missing_columns_timer = nullptr; RuntimeProfile::Counter* _pre_filter_timer = nullptr; RuntimeProfile::Counter* _convert_to_output_block_timer = nullptr; RuntimeProfile::Counter* _empty_file_counter = nullptr; RuntimeProfile::Counter* _not_found_file_counter = nullptr; RuntimeProfile::Counter* _file_counter = nullptr; - RuntimeProfile::Counter* _has_fully_rf_file_counter = nullptr; const std::unordered_map* _col_name_to_slot_id = nullptr; // single slot filter conjuncts @@ -207,7 +205,6 @@ class VFileScanner : public VScanner { Status _truncate_char_or_varchar_columns(Block* block); void _truncate_char_or_varchar_column(Block* block, int idx, int len); Status _generate_fill_columns(); - Status _handle_dynamic_block(Block* block); Status _process_conjuncts_for_dict_filter(); Status _process_late_arrival_conjuncts(); void _get_slot_ids(VExpr* expr, std::vector* slot_ids); diff --git a/be/src/vec/exec/scan/vscanner.cpp b/be/src/vec/exec/scan/vscanner.cpp index ae255f85a7f604..97bf563db1fa58 100644 --- a/be/src/vec/exec/scan/vscanner.cpp +++ b/be/src/vec/exec/scan/vscanner.cpp @@ -113,8 +113,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { // 1. Get input block from scanner { // get block time - auto* timer = _local_state->_scan_timer; - SCOPED_TIMER(timer); + SCOPED_TIMER(_local_state->_scan_timer); RETURN_IF_ERROR(_get_block_impl(state, block, eof)); if (*eof) { DCHECK(block->rows() == 0); @@ -128,8 +127,7 @@ Status VScanner::get_block(RuntimeState* state, Block* block, bool* eof) { // 2. Filter the output block finally. { - auto* timer = _local_state->_filter_timer; - SCOPED_TIMER(timer); + SCOPED_TIMER(_local_state->_filter_timer); RETURN_IF_ERROR(_filter_output_block(block)); } // record rows return (after filter) for _limit check diff --git a/be/src/vec/olap/vertical_block_reader.cpp b/be/src/vec/olap/vertical_block_reader.cpp index 5367729f637e15..369bf04459a8f2 100644 --- a/be/src/vec/olap/vertical_block_reader.cpp +++ b/be/src/vec/olap/vertical_block_reader.cpp @@ -76,7 +76,6 @@ Status VerticalBlockReader::_get_segment_iterators(const ReaderParams& read_para << ", version:" << read_params.version; return res; } - _reader_context.is_vertical_compaction = true; for (const auto& rs_split : read_params.rs_splits) { // segment iterator will be inited here // In vertical compaction, every group will load segment so we should cache diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index 1f24ba122e7e3f..a93033297a0041 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -188,11 +188,11 @@ Status Channel::send_current_block(bool eos, Status exec_status) { template Status Channel::send_local_block(Status exec_status, bool eos) { - SCOPED_TIMER(_parent->local_send_timer()); Block block = _serializer.get_block()->to_block(); _serializer.get_block()->set_mutable_columns(block.clone_empty_columns()); if (_recvr_is_valid()) { if constexpr (!std::is_same_v) { + SCOPED_TIMER(_parent->local_send_timer()); COUNTER_UPDATE(_parent->local_bytes_send_counter(), block.bytes()); COUNTER_UPDATE(_parent->local_sent_rows(), block.rows()); COUNTER_UPDATE(_parent->blocks_sent_counter(), 1); @@ -211,9 +211,9 @@ Status Channel::send_local_block(Status exec_status, bool eos) { template Status Channel::send_local_block(Block* block, bool can_be_moved) { - SCOPED_TIMER(_parent->local_send_timer()); if (_recvr_is_valid()) { if constexpr (!std::is_same_v) { + SCOPED_TIMER(_parent->local_send_timer()); COUNTER_UPDATE(_parent->local_bytes_send_counter(), block->bytes()); COUNTER_UPDATE(_parent->local_sent_rows(), block->rows()); COUNTER_UPDATE(_parent->blocks_sent_counter(), 1); @@ -230,7 +230,6 @@ Status Channel::send_remote_block(PBlock* block, bool eos, Status exec_s if constexpr (!std::is_same_v) { COUNTER_UPDATE(_parent->blocks_sent_counter(), 1); } - SCOPED_TIMER(_parent->brpc_send_timer()); if (_send_remote_block_callback == nullptr) { _send_remote_block_callback = DummyBrpcCallback::create_shared(); @@ -355,6 +354,13 @@ Status Channel::close(RuntimeState* state, Status exec_status) { return st; } +template +int64_t Channel::mem_usage() const { + auto* mutable_block = _serializer.get_block(); + int64_t mem_usage = mutable_block ? mutable_block->allocated_bytes() : 0; + return mem_usage; +} + template void Channel::ch_roll_pb_block() { _ch_cur_pb_block = (_ch_cur_pb_block == &_ch_pb_block1 ? &_ch_pb_block2 : &_ch_pb_block1); @@ -375,14 +381,15 @@ Status BlockSerializer::next_serialized_block(Block* block, PBlock* dest { SCOPED_CONSUME_MEM_TRACKER(_parent->mem_tracker()); + if constexpr (!std::is_same_v) { + SCOPED_TIMER(_parent->merge_block_timer()); + } if (rows) { if (!rows->empty()) { - SCOPED_TIMER(_parent->split_block_distribute_by_channel_timer()); const auto* begin = rows->data(); RETURN_IF_ERROR(_mutable_block->add_rows(block, begin, begin + rows->size())); } } else if (!block->empty()) { - SCOPED_TIMER(_parent->merge_block_timer()); RETURN_IF_ERROR(_mutable_block->merge(*block)); } } diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index b046c2efafcddf..12e72ee9418679 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -89,6 +89,7 @@ class BlockSerializer { void reset_block() { _mutable_block.reset(); } void set_is_local(bool is_local) { _is_local = is_local; } + bool is_local() const { return _is_local; } private: Parent* _parent; @@ -183,6 +184,8 @@ class Channel { void set_receiver_eof(Status st) { _receiver_status = st; } + int64_t mem_usage() const; + protected: bool _recvr_is_valid() { if (_local_recvr && !_local_recvr->is_closed()) { @@ -194,7 +197,6 @@ class Channel { } Status _wait_last_brpc() { - SCOPED_TIMER(_parent->brpc_wait_timer()); if (_send_remote_block_callback == nullptr) { return Status::OK(); } diff --git a/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp b/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp deleted file mode 100644 index 4793ae8128aa9f..00000000000000 --- a/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp +++ /dev/null @@ -1,654 +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 -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include "gtest/gtest_pred_impl.h" -#include "olap/hll.h" -#include "runtime/descriptors.cpp" -#include "runtime/descriptors.h" -#include "util/arrow/block_convertor.h" -#include "util/arrow/row_batch.h" -#include "util/bitmap_value.h" -#include "util/quantile_state.h" -#include "util/string_parser.hpp" -#include "vec/columns/column.h" -#include "vec/columns/column_array.h" -#include "vec/columns/column_complex.h" -#include "vec/columns/column_decimal.h" -#include "vec/columns/column_map.h" -#include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" -#include "vec/columns/column_vector.h" -#include "vec/core/block.h" -#include "vec/core/field.h" -#include "vec/core/types.h" -#include "vec/data_types/data_type.h" -#include "vec/data_types/data_type_array.h" -#include "vec/data_types/data_type_bitmap.h" -#include "vec/data_types/data_type_date.h" -#include "vec/data_types/data_type_date_time.h" -#include "vec/data_types/data_type_decimal.h" -#include "vec/data_types/data_type_hll.h" -#include "vec/data_types/data_type_ipv4.h" -#include "vec/data_types/data_type_ipv6.h" -#include "vec/data_types/data_type_map.h" -#include "vec/data_types/data_type_nullable.h" -#include "vec/data_types/data_type_number.h" -#include "vec/data_types/data_type_quantilestate.h" -#include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_type_struct.h" -#include "vec/data_types/data_type_time_v2.h" -#include "vec/io/io_helper.h" -#include "vec/runtime/vdatetime_value.h" -#include "vec/utils/arrow_column_to_doris_column.h" - -namespace doris::vectorized { - -template -void serialize_and_deserialize_arrow_test() { - vectorized::Block block; - std::vector> cols; - if constexpr (is_scalar) { - cols = { - {"k1", FieldType::OLAP_FIELD_TYPE_INT, 1, TYPE_INT, false}, - {"k7", FieldType::OLAP_FIELD_TYPE_INT, 7, TYPE_INT, true}, - {"k2", FieldType::OLAP_FIELD_TYPE_STRING, 2, TYPE_STRING, false}, - {"k3", FieldType::OLAP_FIELD_TYPE_DECIMAL128I, 3, TYPE_DECIMAL128I, false}, - {"k11", FieldType::OLAP_FIELD_TYPE_DATETIME, 11, TYPE_DATETIME, false}, - {"k4", FieldType::OLAP_FIELD_TYPE_BOOL, 4, TYPE_BOOLEAN, false}, - {"k5", FieldType::OLAP_FIELD_TYPE_DECIMAL32, 5, TYPE_DECIMAL32, false}, - {"k6", FieldType::OLAP_FIELD_TYPE_DECIMAL64, 6, TYPE_DECIMAL64, false}, - {"k12", FieldType::OLAP_FIELD_TYPE_DATETIMEV2, 12, TYPE_DATETIMEV2, false}, - {"k8", FieldType::OLAP_FIELD_TYPE_IPV4, 8, TYPE_IPV4, false}, - {"k9", FieldType::OLAP_FIELD_TYPE_IPV6, 9, TYPE_IPV6, false}, - }; - } else { - cols = {{"a", FieldType::OLAP_FIELD_TYPE_ARRAY, 6, TYPE_ARRAY, true}, - {"m", FieldType::OLAP_FIELD_TYPE_MAP, 8, TYPE_MAP, true}, - {"s", FieldType::OLAP_FIELD_TYPE_STRUCT, 5, TYPE_STRUCT, true}}; - } - - int row_num = 7; - // make desc and generate block - TupleDescriptor tuple_desc(PTupleDescriptor(), true); - for (auto t : cols) { - TSlotDescriptor tslot; - std::string col_name = std::get<0>(t); - tslot.__set_colName(col_name); - TypeDescriptor type_desc(std::get<3>(t)); - bool is_nullable(std::get<4>(t)); - switch (std::get<3>(t)) { - case TYPE_BOOLEAN: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnVector::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i % 2); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_INT: - tslot.__set_slotType(type_desc.to_thrift()); - if (is_nullable) { - { - auto column_vector_int32 = vectorized::ColumnVector::create(); - auto column_nullable_vector = - vectorized::make_nullable(std::move(column_vector_int32)); - auto mutable_nullable_vector = std::move(*column_nullable_vector).mutate(); - for (int i = 0; i < row_num; i++) { - if (i % 2 == 0) { - mutable_nullable_vector->insert_default(); - } else { - mutable_nullable_vector->insert(int32(i)); - } - } - auto data_type = vectorized::make_nullable( - std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name( - mutable_nullable_vector->get_ptr(), data_type, col_name); - block.insert(type_and_name); - } - } else { - auto vec = vectorized::ColumnVector::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_DECIMAL32: - type_desc.precision = 9; - type_desc.scale = 2; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type = - std::make_shared>(type_desc.precision, - type_desc.scale); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - if (i == 0) { - data.push_back(Int32(0)); - continue; - } - Int32 val; - StringParser::ParseResult result = StringParser::PARSE_SUCCESS; - i % 2 == 0 ? val = StringParser::string_to_decimal( - "1234567.56", 11, type_desc.precision, type_desc.scale, - &result) - : val = StringParser::string_to_decimal( - "-1234567.56", 12, type_desc.precision, type_desc.scale, - &result); - EXPECT_TRUE(result == StringParser::PARSE_SUCCESS); - data.push_back(val); - } - - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_DECIMAL64: - type_desc.precision = 18; - type_desc.scale = 6; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type = - std::make_shared>(type_desc.precision, - type_desc.scale); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - if (i == 0) { - data.push_back(Int64(0)); - continue; - } - Int64 val; - StringParser::ParseResult result = StringParser::PARSE_SUCCESS; - std::string decimal_string = - i % 2 == 0 ? "-123456789012.123456" : "123456789012.123456"; - val = StringParser::string_to_decimal( - decimal_string.c_str(), decimal_string.size(), type_desc.precision, - type_desc.scale, &result); - EXPECT_TRUE(result == StringParser::PARSE_SUCCESS); - data.push_back(val); - } - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_DECIMAL128I: - type_desc.precision = 27; - type_desc.scale = 9; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type( - doris::vectorized::create_decimal(27, 9, true)); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - __int128_t value = __int128_t(i * pow(10, 9) + i * pow(10, 8)); - data.push_back(value); - } - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_STRING: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto strcol = vectorized::ColumnString::create(); - for (int i = 0; i < row_num; ++i) { - std::string is = std::to_string(i); - strcol->insert_data(is.c_str(), is.size()); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_HLL: - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr hll_data_type(std::make_shared()); - auto hll_column = hll_data_type->create_column(); - std::vector& container = - ((vectorized::ColumnHLL*)hll_column.get())->get_data(); - for (int i = 0; i < row_num; ++i) { - HyperLogLog hll; - hll.update(i); - container.push_back(hll); - } - vectorized::ColumnWithTypeAndName type_and_name(hll_column->get_ptr(), - hll_data_type, col_name); - - block.insert(type_and_name); - } - break; - case TYPE_DATEV2: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_date_v2 = vectorized::ColumnVector::create(); - auto& date_v2_data = column_vector_date_v2->get_data(); - for (int i = 0; i < row_num; ++i) { - DateV2Value value; - value.from_date((uint32_t)((2022 << 9) | (6 << 5) | 6)); - date_v2_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr date_v2_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_date_v2(column_vector_date_v2->get_ptr(), - date_v2_type, col_name); - block.insert(test_date_v2); - } - break; - case TYPE_DATE: // int64 - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_date = vectorized::ColumnVector::create(); - auto& date_data = column_vector_date->get_data(); - for (int i = 0; i < row_num; ++i) { - VecDateTimeValue value; - value.from_date_int64(20210501); - date_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr date_type(std::make_shared()); - vectorized::ColumnWithTypeAndName test_date(column_vector_date->get_ptr(), - date_type, col_name); - block.insert(test_date); - } - break; - case TYPE_DATETIME: // int64 - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_datetime = vectorized::ColumnVector::create(); - auto& datetime_data = column_vector_datetime->get_data(); - for (int i = 0; i < row_num; ++i) { - VecDateTimeValue value; - value.from_date_int64(20210501080910); - datetime_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr datetime_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_datetime(column_vector_datetime->get_ptr(), - datetime_type, col_name); - block.insert(test_datetime); - } - break; - case TYPE_DATETIMEV2: // uint64 - tslot.__set_slotType(type_desc.to_thrift()); - { - // 2022-01-01 11:11:11.111 - auto column_vector_datetimev2 = - vectorized::ColumnVector::create(); - // auto& datetimev2_data = column_vector_datetimev2->get_data(); - DateV2Value value; - string date_literal = "2022-01-01 11:11:11.111"; - value.from_date_str(date_literal.c_str(), date_literal.size()); - char to[64] = {}; - std::cout << "value: " << value.to_string(to) << std::endl; - for (int i = 0; i < row_num; ++i) { - column_vector_datetimev2->insert(value.to_date_int_val()); - } - vectorized::DataTypePtr datetimev2_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_datetimev2( - column_vector_datetimev2->get_ptr(), datetimev2_type, col_name); - block.insert(test_datetimev2); - } - break; - case TYPE_ARRAY: // array - type_desc.add_sub_type(TYPE_STRING, true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - DataTypePtr au = std::make_shared(s); - Array a1, a2; - a1.push_back(String("sss")); - a1.push_back(Null()); - a1.push_back(String("clever amory")); - a2.push_back(String("hello amory")); - a2.push_back(Null()); - a2.push_back(String("cute amory")); - a2.push_back(String("sf")); - MutableColumnPtr array_column = au->create_column(); - array_column->reserve(2); - array_column->insert(a1); - array_column->insert(a2); - vectorized::ColumnWithTypeAndName type_and_name(array_column->get_ptr(), au, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_MAP: - type_desc.add_sub_type(TYPE_STRING, true); - type_desc.add_sub_type(TYPE_STRING, true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - ; - DataTypePtr d = - std::make_shared(std::make_shared()); - DataTypePtr m = std::make_shared(s, d); - Array k1, k2, v1, v2; - k1.push_back("null"); - k1.push_back("doris"); - k1.push_back("clever amory"); - v1.push_back("ss"); - v1.push_back(Null()); - v1.push_back("NULL"); - k2.push_back("hello amory"); - k2.push_back("NULL"); - k2.push_back("cute amory"); - k2.push_back("doris"); - v2.push_back("s"); - v2.push_back("0"); - v2.push_back("sf"); - v2.push_back(Null()); - Map m1, m2; - m1.push_back(k1); - m1.push_back(v1); - m2.push_back(k2); - m2.push_back(v2); - MutableColumnPtr map_column = m->create_column(); - map_column->reserve(2); - map_column->insert(m1); - map_column->insert(m2); - vectorized::ColumnWithTypeAndName type_and_name(map_column->get_ptr(), m, col_name); - block.insert(type_and_name); - } - break; - case TYPE_STRUCT: - type_desc.add_sub_type(TYPE_STRING, "name", true); - type_desc.add_sub_type(TYPE_LARGEINT, "age", true); - type_desc.add_sub_type(TYPE_BOOLEAN, "is", true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - DataTypePtr d = - std::make_shared(std::make_shared()); - DataTypePtr m = - std::make_shared(std::make_shared()); - DataTypePtr st = - std::make_shared(std::vector {s, d, m}); - Tuple t1, t2; - t1.push_back(String("amory cute")); - t1.push_back(__int128_t(37)); - t1.push_back(true); - t2.push_back("null"); - t2.push_back(__int128_t(26)); - t2.push_back(false); - MutableColumnPtr struct_column = st->create_column(); - struct_column->reserve(2); - struct_column->insert(t1); - struct_column->insert(t2); - vectorized::ColumnWithTypeAndName type_and_name(struct_column->get_ptr(), st, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_IPV4: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnIPv4::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_IPV6: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnIPv6::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - default: - break; - } - - tslot.__set_col_unique_id(std::get<2>(t)); - SlotDescriptor* slot = new SlotDescriptor(tslot); - tuple_desc.add_slot(slot); - } - - RowDescriptor row_desc(&tuple_desc, true); - // arrow schema - std::shared_ptr _arrow_schema; - EXPECT_EQ(convert_to_arrow_schema(row_desc, &_arrow_schema), Status::OK()); - - // serialize - std::shared_ptr result; - std::cout << "block data: " << block.dump_data(0, row_num) << std::endl; - std::cout << "_arrow_schema: " << _arrow_schema->ToString(true) << std::endl; - - cctz::time_zone timezone_obj; - TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, timezone_obj); - static_cast(convert_to_arrow_batch(block, _arrow_schema, arrow::default_memory_pool(), - &result, timezone_obj)); - Block new_block = block.clone_empty(); - EXPECT_TRUE(result != nullptr); - std::cout << "result: " << result->ToString() << std::endl; - // deserialize - for (auto t : cols) { - std::string real_column_name = std::get<0>(t); - auto* array = result->GetColumnByName(real_column_name).get(); - auto& column_with_type_and_name = new_block.get_by_name(real_column_name); - if (std::get<3>(t) == PrimitiveType::TYPE_DATE || - std::get<3>(t) == PrimitiveType::TYPE_DATETIME) { - { - auto strcol = vectorized::ColumnString::create(); - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - real_column_name); - static_cast(arrow_column_to_doris_column( - array, 0, type_and_name.column, type_and_name.type, block.rows(), "UTC")); - { - auto& col = column_with_type_and_name.column.get()->assume_mutable_ref(); - auto& date_data = static_cast&>(col).get_data(); - for (int i = 0; i < strcol->size(); ++i) { - StringRef str = strcol->get_data_at(i); - VecDateTimeValue value; - value.from_date_str(str.data, str.size); - date_data.push_back(*reinterpret_cast(&value)); - } - } - } - continue; - } else if (std::get<3>(t) == PrimitiveType::TYPE_DATEV2) { - auto strcol = vectorized::ColumnString::create(); - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - real_column_name); - static_cast(arrow_column_to_doris_column( - array, 0, type_and_name.column, type_and_name.type, block.rows(), "UTC")); - { - auto& col = column_with_type_and_name.column.get()->assume_mutable_ref(); - auto& date_data = static_cast&>(col).get_data(); - for (int i = 0; i < strcol->size(); ++i) { - StringRef str = strcol->get_data_at(i); - DateV2Value value; - value.from_date_str(str.data, str.size); - date_data.push_back(*reinterpret_cast(&value)); - } - } - continue; - } else if (std::get<3>(t) == PrimitiveType::TYPE_DATETIMEV2) { - // now we only support read doris datetimev2 to arrow - block.erase(real_column_name); - new_block.erase(real_column_name); - continue; - } - static_cast(arrow_column_to_doris_column(array, 0, column_with_type_and_name.column, - column_with_type_and_name.type, block.rows(), - "UTC")); - } - - std::cout << block.dump_data() << std::endl; - std::cout << new_block.dump_data() << std::endl; - EXPECT_EQ(block.dump_data(), new_block.dump_data()); -} - -TEST(DataTypeSerDeArrowTest, DataTypeScalaSerDeTest) { - serialize_and_deserialize_arrow_test(); -} - -TEST(DataTypeSerDeArrowTest, DataTypeCollectionSerDeTest) { - serialize_and_deserialize_arrow_test(); -} - -TEST(DataTypeSerDeArrowTest, DataTypeMapNullKeySerDeTest) { - TupleDescriptor tuple_desc(PTupleDescriptor(), true); - TSlotDescriptor tslot; - std::string col_name = "map_null_key"; - tslot.__set_colName(col_name); - TypeDescriptor type_desc(TYPE_MAP); - type_desc.add_sub_type(TYPE_STRING, true); - type_desc.add_sub_type(TYPE_INT, true); - tslot.__set_slotType(type_desc.to_thrift()); - vectorized::Block block; - { - DataTypePtr s = std::make_shared(std::make_shared()); - ; - DataTypePtr d = std::make_shared(std::make_shared()); - DataTypePtr m = std::make_shared(s, d); - Array k1, k2, v1, v2, k3, v3; - k1.push_back("doris"); - k1.push_back("clever amory"); - v1.push_back(Null()); - v1.push_back(30); - k2.push_back("hello amory"); - k2.push_back("NULL"); - k2.push_back("cute amory"); - k2.push_back("doris"); - v2.push_back(26); - v2.push_back(Null()); - v2.push_back(6); - v2.push_back(7); - k3.push_back("test"); - v3.push_back(11); - Map m1, m2, m3; - m1.push_back(k1); - m1.push_back(v1); - m2.push_back(k2); - m2.push_back(v2); - m3.push_back(k3); - m3.push_back(v3); - MutableColumnPtr map_column = m->create_column(); - map_column->reserve(3); - map_column->insert(m1); - map_column->insert(m2); - map_column->insert(m3); - vectorized::ColumnWithTypeAndName type_and_name(map_column->get_ptr(), m, col_name); - block.insert(type_and_name); - } - - tslot.__set_col_unique_id(1); - SlotDescriptor* slot = new SlotDescriptor(tslot); - tuple_desc.add_slot(slot); - RowDescriptor row_desc(&tuple_desc, true); - // arrow schema - std::shared_ptr _arrow_schema; - EXPECT_EQ(convert_to_arrow_schema(row_desc, &_arrow_schema), Status::OK()); - - // serialize - std::shared_ptr result; - std::cout << "block structure: " << block.dump_structure() << std::endl; - std::cout << "_arrow_schema: " << _arrow_schema->ToString(true) << std::endl; - - cctz::time_zone timezone_obj; - TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, timezone_obj); - static_cast(convert_to_arrow_batch(block, _arrow_schema, arrow::default_memory_pool(), - &result, timezone_obj)); - Block new_block = block.clone_empty(); - EXPECT_TRUE(result != nullptr); - std::cout << "result: " << result->ToString() << std::endl; - // deserialize - auto* array = result->GetColumnByName(col_name).get(); - auto& column_with_type_and_name = new_block.get_by_name(col_name); - static_cast(arrow_column_to_doris_column(array, 0, column_with_type_and_name.column, - column_with_type_and_name.type, block.rows(), - "UTC")); - std::cout << block.dump_data() << std::endl; - std::cout << new_block.dump_data() << std::endl; - // new block row_index 0, 2 which row has key null will be filter - EXPECT_EQ(new_block.dump_one_line(0, 1), "{\"doris\":null, \"clever amory\":30}"); - EXPECT_EQ(new_block.dump_one_line(2, 1), "{\"test\":11}"); - EXPECT_EQ(block.dump_data(1, 1), new_block.dump_data(1, 1)); -} - -} // namespace doris::vectorized