diff --git a/.asf.yaml b/.asf.yaml index 7a7d845e4c9bb0..e3d516b35c19a5 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -63,7 +63,6 @@ github: - COMPILE (DORIS_COMPILE) - Need_2_Approval - Cloud UT (Doris Cloud UT) - - performance (Doris Performance) required_pull_request_reviews: dismiss_stale_reviews: true diff --git a/be/src/agent/workload_group_listener.cpp b/be/src/agent/workload_group_listener.cpp index 7b688b7dcdf6ef..0cd5a3ee1ac748 100644 --- a/be/src/agent/workload_group_listener.cpp +++ b/be/src/agent/workload_group_listener.cpp @@ -17,6 +17,7 @@ #include "agent/workload_group_listener.h" +#include "runtime/exec_env.h" #include "runtime/workload_group/workload_group.h" #include "runtime/workload_group/workload_group_manager.h" #include "util/mem_info.h" diff --git a/be/src/agent/workload_group_listener.h b/be/src/agent/workload_group_listener.h index f596535908d079..9578a36f70d63e 100644 --- a/be/src/agent/workload_group_listener.h +++ b/be/src/agent/workload_group_listener.h @@ -20,10 +20,11 @@ #include #include "agent/topic_listener.h" -#include "runtime/exec_env.h" namespace doris { +class ExecEnv; + class WorkloadGroupListener : public TopicListener { public: ~WorkloadGroupListener() {} diff --git a/be/src/apache-orc b/be/src/apache-orc index db01184f765c03..2f937bdc76406f 160000 --- a/be/src/apache-orc +++ b/be/src/apache-orc @@ -1 +1 @@ -Subproject commit db01184f765c03496e4107bd3ac37c077ac4bc5f +Subproject commit 2f937bdc76406f150b484b6e57629aa8a03d48b6 diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 6b4dbe360da651..1acf8efe32e62b 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -92,6 +92,10 @@ Status CloudCumulativeCompaction::prepare_compact() { // plus 1 to skip the delete version. // NOTICE: after that, the cumulative point may be larger than max version of this tablet, but it doesn't matter. update_cumulative_point(); + if (!config::enable_sleep_between_delete_cumu_compaction) { + st = Status::Error( + "_last_delete_version.first not equal to -1"); + } } return st; } diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index b66a9cfbdb2245..650909a29157cd 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -677,7 +677,8 @@ Status CloudStorageEngine::_submit_cumulative_compaction_task(const CloudTabletS auto st = compaction->prepare_compact(); if (!st.ok()) { long now = duration_cast(system_clock::now().time_since_epoch()).count(); - if (st.is()) { + if (st.is() && + st.msg() != "_last_delete_version.first not equal to -1") { // Backoff strategy if no suitable version tablet->last_cumu_no_suitable_version_ms = now; } diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index c3d00e23c98e5e..95a3e61fb5517a 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -251,7 +251,7 @@ DEFINE_mInt32(download_low_speed_limit_kbps, "50"); // download low speed time(seconds) DEFINE_mInt32(download_low_speed_time, "300"); // whether to download small files in batch -DEFINE_mBool(enable_batch_download, "false"); +DEFINE_mBool(enable_batch_download, "true"); DEFINE_String(sys_log_dir, ""); DEFINE_String(user_function_dir, "${DORIS_HOME}/lib/udf"); @@ -1211,7 +1211,7 @@ DEFINE_Bool(exit_on_exception, "false"); DEFINE_Bool(enable_flush_file_cache_async, "true"); // cgroup -DEFINE_mString(doris_cgroup_cpu_path, ""); +DEFINE_String(doris_cgroup_cpu_path, ""); DEFINE_mBool(enable_be_proc_monitor, "false"); DEFINE_mInt32(be_proc_monitor_interval_ms, "10000"); @@ -1402,6 +1402,7 @@ DEFINE_mBool(enable_delete_bitmap_merge_on_compaction, "false"); // Enable validation to check the correctness of table size. DEFINE_Bool(enable_table_size_correctness_check, "false"); DEFINE_Bool(force_regenerate_rowsetid_on_start_error, "false"); +DEFINE_mBool(enable_sleep_between_delete_cumu_compaction, "false"); // clang-format off #ifdef BE_TEST diff --git a/be/src/common/config.h b/be/src/common/config.h index c0b2e19b49a6be..f8a9c3f7480b33 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1292,7 +1292,7 @@ DECLARE_mInt32(tablet_schema_cache_capacity); DECLARE_mBool(exit_on_exception); // cgroup -DECLARE_mString(doris_cgroup_cpu_path); +DECLARE_String(doris_cgroup_cpu_path); DECLARE_mBool(enable_be_proc_monitor); DECLARE_mInt32(be_proc_monitor_interval_ms); DECLARE_Int32(workload_group_metrics_interval_ms); @@ -1487,6 +1487,8 @@ DECLARE_Bool(force_regenerate_rowsetid_on_start_error); DECLARE_mBool(enable_delete_bitmap_merge_on_compaction); // Enable validation to check the correctness of table size. DECLARE_Bool(enable_table_size_correctness_check); +// Enable sleep 5s between delete cumulative compaction. +DECLARE_mBool(enable_sleep_between_delete_cumu_compaction); #ifdef BE_TEST // test s3 diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index 73035ecf3957eb..12bf1749a5694d 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -437,6 +437,8 @@ void Daemon::calculate_metrics_thread() { // update lst map DorisMetrics::instance()->system_metrics()->get_network_traffic( &lst_net_send_bytes, &lst_net_receive_bytes); + + DorisMetrics::instance()->system_metrics()->update_be_avail_cpu_num(); } update_rowsets_and_segments_num_metrics(); } diff --git a/be/src/common/status.h b/be/src/common/status.h index 344f82a81b8e25..d059f289402cea 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -570,7 +570,7 @@ class [[nodiscard]] Status { // and another thread is call to_string method, it may core, because the _err_msg is an unique ptr and // it is deconstructed during copy method. // And also we could not use lock, because we need get status frequently to check if it is cancelled. -// The defaule value is ok. +// The default value is ok. class AtomicStatus { public: AtomicStatus() : error_st_(Status::OK()) {} diff --git a/be/src/common/version_internal.cpp b/be/src/common/version_internal.cpp index 1190242b6aa687..55402fab209400 100644 --- a/be/src/common/version_internal.cpp +++ b/be/src/common/version_internal.cpp @@ -34,6 +34,9 @@ int doris_build_version_minor() { int doris_build_version_patch() { return DORIS_BUILD_VERSION_PATCH; } +int doris_build_version_hotfix() { + return DORIS_BUILD_VERSION_HOTFIX; +} const char* doris_build_version_rc_version() { return DORIS_BUILD_VERSION_RC_VERSION; } @@ -56,4 +59,4 @@ const char* doris_build_info() { } // namespace version -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/common/version_internal.h b/be/src/common/version_internal.h index 8852d26dba9531..f4deaa15aff545 100644 --- a/be/src/common/version_internal.h +++ b/be/src/common/version_internal.h @@ -24,6 +24,7 @@ extern const char* doris_build_version_prefix(); extern int doris_build_version_major(); extern int doris_build_version_minor(); extern int doris_build_version_patch(); +extern int doris_build_version_hotfix(); extern const char* doris_build_version_rc_version(); extern const char* doris_build_version(); @@ -34,4 +35,4 @@ extern const char* doris_build_info(); } // namespace version -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/exec/decompressor.cpp b/be/src/exec/decompressor.cpp index 9365bb00288db1..5da2e6acbb9bdf 100644 --- a/be/src/exec/decompressor.cpp +++ b/be/src/exec/decompressor.cpp @@ -492,15 +492,15 @@ Status Lz4BlockDecompressor::decompress(uint8_t* input, size_t input_len, size_t auto* output_ptr = output; while (input_len > 0) { - //if faild , fall back to large block begin - auto* large_block_input_ptr = input_ptr; - auto* large_block_output_ptr = output_ptr; - if (input_len < sizeof(uint32_t)) { - return Status::InvalidArgument(strings::Substitute( - "fail to do hadoop-lz4 decompress, input_len=$0", input_len)); + *more_input_bytes = sizeof(uint32_t) - input_len; + break; } + //if faild, fall back to large block begin + auto* large_block_input_ptr = input_ptr; + auto* large_block_output_ptr = output_ptr; + uint32_t remaining_decompressed_large_block_len = BigEndian::Load32(input_ptr); input_ptr += sizeof(uint32_t); @@ -609,15 +609,15 @@ Status SnappyBlockDecompressor::decompress(uint8_t* input, size_t input_len, auto* output_ptr = output; while (input_len > 0) { - //if faild , fall back to large block begin - auto* large_block_input_ptr = input_ptr; - auto* large_block_output_ptr = output_ptr; - if (input_len < sizeof(uint32_t)) { - return Status::InvalidArgument(strings::Substitute( - "fail to do hadoop-snappy decompress, input_len=$0", input_len)); + *more_input_bytes = sizeof(uint32_t) - input_len; + break; } + //if faild, fall back to large block begin + auto* large_block_input_ptr = input_ptr; + auto* large_block_output_ptr = output_ptr; + uint32_t remaining_decompressed_large_block_len = BigEndian::Load32(input_ptr); input_ptr += sizeof(uint32_t); diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 8f297d7074ff12..d1567a8fa79cb4 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -498,12 +498,12 @@ class RuntimePredicateWrapper { switch (_filter_type) { case RuntimeFilterType::IN_FILTER: { if (!_context->hybrid_set) { - _context->ignored = true; + set_ignored(); return Status::OK(); } _context->hybrid_set->insert(wrapper->_context->hybrid_set.get()); if (_max_in_num >= 0 && _context->hybrid_set->size() >= _max_in_num) { - _context->ignored = true; + set_ignored(); // release in filter _context->hybrid_set.reset(); } @@ -1337,7 +1337,7 @@ void IRuntimeFilter::set_synced_size(uint64_t global_size) { } void IRuntimeFilter::set_ignored() { - _wrapper->_context->ignored = true; + _wrapper->set_ignored(); } bool IRuntimeFilter::get_ignored() { diff --git a/be/src/http/action/download_binlog_action.cpp b/be/src/http/action/download_binlog_action.cpp index 372f840401c4ad..4bb8b8b70dd722 100644 --- a/be/src/http/action/download_binlog_action.cpp +++ b/be/src/http/action/download_binlog_action.cpp @@ -144,8 +144,19 @@ void handle_get_segment_index_file(StorageEngine& engine, HttpRequest* req, const auto& rowset_id = get_http_param(req, kRowsetIdParameter); const auto& segment_index = get_http_param(req, kSegmentIndexParameter); const auto& segment_index_id = req->param(kSegmentIndexIdParameter); - segment_index_file_path = - tablet->get_segment_index_filepath(rowset_id, segment_index, segment_index_id); + auto segment_file_path = tablet->get_segment_filepath(rowset_id, segment_index); + if (tablet->tablet_schema()->get_inverted_index_storage_format() == + InvertedIndexStorageFormatPB::V1) { + // now CCR not support for variant + index v1 + constexpr std::string_view index_suffix = ""; + segment_index_file_path = InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_file_path), + std::stoll(segment_index_id), index_suffix); + } else { + DCHECK(segment_index_id == "-1"); + segment_index_file_path = InvertedIndexDescriptor::get_index_file_path_v2( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_file_path)); + } is_acquire_md5 = !req->param(kAcquireMD5Parameter).empty(); } catch (const std::exception& e) { HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, e.what()); diff --git a/be/src/io/cache/block_file_cache_profile.h b/be/src/io/cache/block_file_cache_profile.h index 19d7f4139f7f15..54118d5094cd52 100644 --- a/be/src/io/cache/block_file_cache_profile.h +++ b/be/src/io/cache/block_file_cache_profile.h @@ -75,6 +75,7 @@ struct FileCacheProfile { struct FileCacheProfileReporter { RuntimeProfile::Counter* num_local_io_total = nullptr; RuntimeProfile::Counter* num_remote_io_total = nullptr; + RuntimeProfile::Counter* num_inverted_index_remote_io_total = nullptr; RuntimeProfile::Counter* local_io_timer = nullptr; RuntimeProfile::Counter* bytes_scanned_from_cache = nullptr; RuntimeProfile::Counter* bytes_scanned_from_remote = nullptr; @@ -90,6 +91,8 @@ struct FileCacheProfileReporter { cache_profile, 1); num_remote_io_total = ADD_CHILD_COUNTER_WITH_LEVEL(profile, "NumRemoteIOTotal", TUnit::UNIT, cache_profile, 1); + num_inverted_index_remote_io_total = ADD_CHILD_COUNTER_WITH_LEVEL( + profile, "NumInvertedIndexRemoteIOTotal", TUnit::UNIT, cache_profile, 1); local_io_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile, "LocalIOUseTimer", cache_profile, 1); remote_io_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile, "RemoteIOUseTimer", cache_profile, 1); write_cache_io_timer = @@ -107,6 +110,8 @@ struct FileCacheProfileReporter { void update(const FileCacheStatistics* statistics) const { COUNTER_UPDATE(num_local_io_total, statistics->num_local_io_total); COUNTER_UPDATE(num_remote_io_total, statistics->num_remote_io_total); + COUNTER_UPDATE(num_inverted_index_remote_io_total, + statistics->num_inverted_index_remote_io_total); COUNTER_UPDATE(local_io_timer, statistics->local_io_timer); COUNTER_UPDATE(remote_io_timer, statistics->remote_io_timer); COUNTER_UPDATE(write_cache_io_timer, statistics->write_cache_io_timer); diff --git a/be/src/io/cache/cached_remote_file_reader.cpp b/be/src/io/cache/cached_remote_file_reader.cpp index c9a273c5d368a6..f16e0019b6dcc5 100644 --- a/be/src/io/cache/cached_remote_file_reader.cpp +++ b/be/src/io/cache/cached_remote_file_reader.cpp @@ -126,7 +126,7 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* ReadStatistics stats; auto defer_func = [&](int*) { if (io_ctx->file_cache_stats) { - _update_state(stats, io_ctx->file_cache_stats); + _update_state(stats, io_ctx->file_cache_stats, io_ctx->is_inverted_index); io::FileCacheProfile::instance().update(io_ctx->file_cache_stats); } }; @@ -312,7 +312,8 @@ Status CachedRemoteFileReader::read_at_impl(size_t offset, Slice result, size_t* } void CachedRemoteFileReader::_update_state(const ReadStatistics& read_stats, - FileCacheStatistics* statis) const { + FileCacheStatistics* statis, + bool is_inverted_index) const { if (statis == nullptr) { return; } @@ -320,6 +321,9 @@ void CachedRemoteFileReader::_update_state(const ReadStatistics& read_stats, statis->num_local_io_total++; statis->bytes_read_from_local += read_stats.bytes_read; } else { + if (is_inverted_index) { + statis->num_inverted_index_remote_io_total++; + } statis->num_remote_io_total++; statis->bytes_read_from_remote += read_stats.bytes_read; } diff --git a/be/src/io/cache/cached_remote_file_reader.h b/be/src/io/cache/cached_remote_file_reader.h index b3efb83c0803c8..685414cfa3aba6 100644 --- a/be/src/io/cache/cached_remote_file_reader.h +++ b/be/src/io/cache/cached_remote_file_reader.h @@ -76,7 +76,8 @@ class CachedRemoteFileReader final : public FileReader { int64_t local_read_timer = 0; int64_t local_write_timer = 0; }; - void _update_state(const ReadStatistics& stats, FileCacheStatistics* state) const; + void _update_state(const ReadStatistics& stats, FileCacheStatistics* state, + bool is_inverted_index) const; }; } // namespace doris::io diff --git a/be/src/io/fs/err_utils.cpp b/be/src/io/fs/err_utils.cpp index 6552d454824796..e9bed7f5887dc3 100644 --- a/be/src/io/fs/err_utils.cpp +++ b/be/src/io/fs/err_utils.cpp @@ -122,13 +122,13 @@ Status s3fs_error(const Aws::S3::S3Error& err, std::string_view msg) { using namespace Aws::Http; switch (err.GetResponseCode()) { case HttpResponseCode::NOT_FOUND: - return Status::Error("{}: {} {} type={}, request_id={}", msg, - err.GetExceptionName(), err.GetMessage(), + return Status::Error("{}: {} {} code=NOT_FOUND, type={}, request_id={}", + msg, err.GetExceptionName(), err.GetMessage(), err.GetErrorType(), err.GetRequestId()); case HttpResponseCode::FORBIDDEN: - return Status::Error("{}: {} {} type={}, request_id={}", msg, - err.GetExceptionName(), err.GetMessage(), - err.GetErrorType(), err.GetRequestId()); + return Status::Error( + "{}: {} {} code=FORBIDDEN, type={}, request_id={}", msg, err.GetExceptionName(), + err.GetMessage(), err.GetErrorType(), err.GetRequestId()); default: return Status::Error( "{}: {} {} code={} type={}, request_id={}", msg, err.GetExceptionName(), diff --git a/be/src/io/io_common.h b/be/src/io/io_common.h index 80a594473dc376..4acc0538b7ef4f 100644 --- a/be/src/io/io_common.h +++ b/be/src/io/io_common.h @@ -38,6 +38,7 @@ namespace io { struct FileCacheStatistics { int64_t num_local_io_total = 0; int64_t num_remote_io_total = 0; + int64_t num_inverted_index_remote_io_total = 0; int64_t local_io_timer = 0; int64_t bytes_read_from_local = 0; int64_t bytes_read_from_remote = 0; @@ -60,6 +61,7 @@ struct IOContext { int64_t expiration_time = 0; const TUniqueId* query_id = nullptr; // Ref FileCacheStatistics* file_cache_stats = nullptr; // Ref + bool is_inverted_index = false; }; } // namespace io diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 0d811d9348f1f5..aec38699e014a2 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -495,10 +495,35 @@ Status CompactionMixin::execute_compact_impl(int64_t permits) { Status Compaction::do_inverted_index_compaction() { const auto& ctx = _output_rs_writer->context(); if (!config::inverted_index_compaction_enable || _input_row_num <= 0 || - !_stats.rowid_conversion || ctx.columns_to_do_index_compaction.empty()) { + ctx.columns_to_do_index_compaction.empty()) { return Status::OK(); } + auto error_handler = [this](int64_t index_id, int64_t column_uniq_id) { + LOG(WARNING) << "failed to do index compaction" + << ". tablet=" << _tablet->tablet_id() << ". column uniq id=" << column_uniq_id + << ". index_id=" << index_id; + for (auto& rowset : _input_rowsets) { + rowset->set_skip_index_compaction(column_uniq_id); + LOG(INFO) << "mark skipping inverted index compaction next time" + << ". tablet=" << _tablet->tablet_id() << ", rowset=" << rowset->rowset_id() + << ", column uniq id=" << column_uniq_id << ", index_id=" << index_id; + } + }; + + DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_rowid_conversion_null", + { _stats.rowid_conversion = nullptr; }) + if (!_stats.rowid_conversion) { + LOG(WARNING) << "failed to do index compaction, rowid conversion is null" + << ". tablet=" << _tablet->tablet_id() + << ", input row number=" << _input_row_num; + mark_skip_index_compaction(ctx, error_handler); + + return Status::Error( + "failed to do index compaction, rowid conversion is null. tablet={}", + _tablet->tablet_id()); + } + OlapStopWatch inverted_watch; // translation vec @@ -521,8 +546,7 @@ Status Compaction::do_inverted_index_compaction() { auto src_segment_num = src_seg_to_id_map.size(); auto dest_segment_num = dest_segment_num_rows.size(); - DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_dest_segment_num_is_zero", - { dest_segment_num = 0; }) + // when all the input rowsets are deleted, the output rowset will be empty and dest_segment_num will be 0. if (dest_segment_num <= 0) { LOG(INFO) << "skip doing index compaction due to no output segments" << ". tablet=" << _tablet->tablet_id() << ", input row number=" << _input_row_num @@ -600,27 +624,62 @@ Status Compaction::do_inverted_index_compaction() { DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_find_rowset_error", { find_it = rs_id_to_rowset_map.end(); }) if (find_it == rs_id_to_rowset_map.end()) [[unlikely]] { - // DCHECK(false) << _tablet->tablet_id() << ' ' << rowset_id; - return Status::InternalError("cannot find rowset. tablet_id={} rowset_id={}", - _tablet->tablet_id(), rowset_id.to_string()); + LOG(WARNING) << "failed to do index compaction, cannot find rowset. tablet_id=" + << _tablet->tablet_id() << " rowset_id=" << rowset_id.to_string(); + mark_skip_index_compaction(ctx, error_handler); + return Status::Error( + "failed to do index compaction, cannot find rowset. tablet_id={} rowset_id={}", + _tablet->tablet_id(), rowset_id.to_string()); } auto* rowset = find_it->second; auto fs = rowset->rowset_meta()->fs(); DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_get_fs_error", { fs = nullptr; }) if (!fs) { - return Status::InternalError("get fs failed, resource_id={}", - rowset->rowset_meta()->resource_id()); + LOG(WARNING) << "failed to do index compaction, get fs failed. resource_id=" + << rowset->rowset_meta()->resource_id(); + mark_skip_index_compaction(ctx, error_handler); + return Status::Error( + "get fs failed, resource_id={}", rowset->rowset_meta()->resource_id()); } - auto seg_path = DORIS_TRY(rowset->segment_path(seg_id)); + auto seg_path = rowset->segment_path(seg_id); + DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_seg_path_nullptr", { + seg_path = ResultError(Status::Error( + "do_inverted_index_compaction_seg_path_nullptr")); + }) + if (!seg_path.has_value()) { + LOG(WARNING) << "failed to do index compaction, get segment path failed. tablet_id=" + << _tablet->tablet_id() << " rowset_id=" << rowset_id.to_string() + << " seg_id=" << seg_id; + mark_skip_index_compaction(ctx, error_handler); + return Status::Error( + "get segment path failed. tablet_id={} rowset_id={} seg_id={}", + _tablet->tablet_id(), rowset_id.to_string(), seg_id); + } auto inverted_index_file_reader = std::make_unique( - fs, std::string {InvertedIndexDescriptor::get_index_file_path_prefix(seg_path)}, + fs, + std::string {InvertedIndexDescriptor::get_index_file_path_prefix(seg_path.value())}, _cur_tablet_schema->get_inverted_index_storage_format(), rowset->rowset_meta()->inverted_index_file_info(seg_id)); - RETURN_NOT_OK_STATUS_WITH_WARN( - inverted_index_file_reader->init(config::inverted_index_read_buffer_size), - "inverted_index_file_reader init faiqled"); + auto st = inverted_index_file_reader->init(config::inverted_index_read_buffer_size); + DBUG_EXECUTE_IF("Compaction::do_inverted_index_compaction_init_inverted_index_file_reader", + { + st = Status::Error( + "debug point: " + "Compaction::do_inverted_index_compaction_init_inverted_index_" + "file_reader error"); + }) + if (!st.ok()) { + LOG(WARNING) << "failed to do index compaction, init inverted index file reader " + "failed. tablet_id=" + << _tablet->tablet_id() << " rowset_id=" << rowset_id.to_string() + << " seg_id=" << seg_id; + mark_skip_index_compaction(ctx, error_handler); + return Status::Error( + "init inverted index file reader failed. tablet_id={} rowset_id={} seg_id={}", + _tablet->tablet_id(), rowset_id.to_string(), seg_id); + } inverted_index_file_readers[m.second] = std::move(inverted_index_file_reader); } @@ -628,7 +687,20 @@ Status Compaction::do_inverted_index_compaction() { // format: rowsetId_segmentId auto& inverted_index_file_writers = dynamic_cast(_output_rs_writer.get()) ->inverted_index_file_writers(); - DCHECK_EQ(inverted_index_file_writers.size(), dest_segment_num); + DBUG_EXECUTE_IF( + "Compaction::do_inverted_index_compaction_inverted_index_file_writers_size_error", + { inverted_index_file_writers.clear(); }) + if (inverted_index_file_writers.size() != dest_segment_num) { + LOG(WARNING) << "failed to do index compaction, dest segment num not match. tablet_id=" + << _tablet->tablet_id() << " dest_segment_num=" << dest_segment_num + << " inverted_index_file_writers.size()=" + << inverted_index_file_writers.size(); + mark_skip_index_compaction(ctx, error_handler); + return Status::Error( + "dest segment num not match. tablet_id={} dest_segment_num={} " + "inverted_index_file_writers.size()={}", + _tablet->tablet_id(), dest_segment_num, inverted_index_file_writers.size()); + } // use tmp file dir to store index files auto tmp_file_dir = ExecEnv::GetInstance()->get_tmp_file_dirs()->get_tmp_file_dir(); @@ -637,18 +709,6 @@ Status Compaction::do_inverted_index_compaction() { << ". tablet=" << _tablet->tablet_id() << ", source index size=" << src_segment_num << ", destination index size=" << dest_segment_num << "."; - auto error_handler = [this](int64_t index_id, int64_t column_uniq_id) { - LOG(WARNING) << "failed to do index compaction" - << ". tablet=" << _tablet->tablet_id() << ". column uniq id=" << column_uniq_id - << ". index_id=" << index_id; - for (auto& rowset : _input_rowsets) { - rowset->set_skip_index_compaction(column_uniq_id); - LOG(INFO) << "mark skipping inverted index compaction next time" - << ". tablet=" << _tablet->tablet_id() << ", rowset=" << rowset->rowset_id() - << ", column uniq id=" << column_uniq_id << ", index_id=" << index_id; - } - }; - Status status = Status::OK(); for (auto&& column_uniq_id : ctx.columns_to_do_index_compaction) { auto col = _cur_tablet_schema->column_by_uid(column_uniq_id); @@ -658,6 +718,10 @@ Status Compaction::do_inverted_index_compaction() { if (index_meta == nullptr) { status = Status::Error( fmt::format("Can not find index_meta for col {}", col.name())); + LOG(WARNING) << "failed to do index compaction, can not find index_meta for column" + << ". tablet=" << _tablet->tablet_id() + << ", column uniq id=" << column_uniq_id; + error_handler(-1, column_uniq_id); break; } @@ -671,6 +735,11 @@ Status Compaction::do_inverted_index_compaction() { "debug point: Compaction::open_index_file_reader error")); }) if (!res.has_value()) { + LOG(WARNING) << "failed to do index compaction, open inverted index file " + "reader failed" + << ". tablet=" << _tablet->tablet_id() + << ", column uniq id=" << column_uniq_id + << ", src_segment_id=" << src_segment_id; throw Exception(ErrorCode::INVERTED_INDEX_COMPACTION_ERROR, res.error().msg()); } src_idx_dirs[src_segment_id] = std::move(res.value()); @@ -682,6 +751,11 @@ Status Compaction::do_inverted_index_compaction() { "debug point: Compaction::open_inverted_index_file_writer error")); }) if (!res.has_value()) { + LOG(WARNING) << "failed to do index compaction, open inverted index file " + "writer failed" + << ". tablet=" << _tablet->tablet_id() + << ", column uniq id=" << column_uniq_id + << ", dest_segment_id=" << dest_segment_id; throw Exception(ErrorCode::INVERTED_INDEX_COMPACTION_ERROR, res.error().msg()); } // Destination directories in dest_index_dirs do not need to be deconstructed, @@ -714,6 +788,23 @@ Status Compaction::do_inverted_index_compaction() { return Status::OK(); } +void Compaction::mark_skip_index_compaction( + const RowsetWriterContext& context, + const std::function& error_handler) { + for (auto&& column_uniq_id : context.columns_to_do_index_compaction) { + auto col = _cur_tablet_schema->column_by_uid(column_uniq_id); + const auto* index_meta = _cur_tablet_schema->inverted_index(col); + if (index_meta == nullptr) { + LOG(WARNING) << "mark skip index compaction, can not find index_meta for column" + << ". tablet=" << _tablet->tablet_id() + << ", column uniq id=" << column_uniq_id; + error_handler(-1, column_uniq_id); + continue; + } + error_handler(index_meta->index_id(), column_uniq_id); + } +} + void Compaction::construct_index_compaction_columns(RowsetWriterContext& ctx) { for (const auto& index : _cur_tablet_schema->inverted_indexes()) { auto col_unique_ids = index->col_unique_ids(); @@ -789,7 +880,8 @@ void Compaction::construct_index_compaction_columns(RowsetWriterContext& ctx) { // TODO: inverted_index_path auto seg_path = rowset->segment_path(i); DBUG_EXECUTE_IF("Compaction::construct_skip_inverted_index_seg_path_nullptr", { - seg_path = ResultError(Status::Error("error")); + seg_path = ResultError(Status::Error( + "construct_skip_inverted_index_seg_path_nullptr")); }) if (!seg_path) { LOG(WARNING) << seg_path.error(); @@ -800,8 +892,8 @@ void Compaction::construct_index_compaction_columns(RowsetWriterContext& ctx) { try { auto inverted_index_file_reader = std::make_unique( fs, - std::string { - InvertedIndexDescriptor::get_index_file_path_prefix(*seg_path)}, + std::string {InvertedIndexDescriptor::get_index_file_path_prefix( + seg_path.value())}, _cur_tablet_schema->get_inverted_index_storage_format(), rowset->rowset_meta()->inverted_index_file_info(i)); auto st = inverted_index_file_reader->init( diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index ccabf7dadb4733..057f4084b068b3 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -70,6 +70,10 @@ class Compaction { // merge inverted index files Status do_inverted_index_compaction(); + // mark all columns in columns_to_do_index_compaction to skip index compaction next time. + void mark_skip_index_compaction(const RowsetWriterContext& context, + const std::function& error_handler); + void construct_index_compaction_columns(RowsetWriterContext& ctx); virtual Status construct_output_rowset_writer(RowsetWriterContext& ctx) = 0; diff --git a/be/src/olap/cumulative_compaction_time_series_policy.cpp b/be/src/olap/cumulative_compaction_time_series_policy.cpp index 6fa4b8d014313f..64e51c77641311 100644 --- a/be/src/olap/cumulative_compaction_time_series_policy.cpp +++ b/be/src/olap/cumulative_compaction_time_series_policy.cpp @@ -27,11 +27,14 @@ namespace doris { uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score(Tablet* tablet) { uint32_t score = 0; + uint32_t level0_score = 0; bool base_rowset_exist = false; const int64_t point = tablet->cumulative_layer_point(); + int64_t level0_total_size = 0; RowsetMetaSharedPtr first_meta; int64_t first_version = INT64_MAX; + std::list checked_rs_metas; // NOTE: tablet._meta_lock is hold auto& rs_metas = tablet->tablet_meta()->all_rs_metas(); // check the base rowset and collect the rowsets of cumulative part @@ -50,6 +53,12 @@ uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score( } else { // collect the rowsets of cumulative part score += rs_meta->get_compaction_score(); + if (rs_meta->compaction_level() == 0) { + level0_total_size += rs_meta->total_disk_size(); + level0_score += rs_meta->get_compaction_score(); + } else { + checked_rs_metas.push_back(rs_meta); + } } } @@ -64,7 +73,64 @@ uint32_t TimeSeriesCumulativeCompactionPolicy::calc_cumulative_compaction_score( return 0; } - return score; + // Condition 1: the size of input files for compaction meets the requirement of parameter compaction_goal_size + int64_t compaction_goal_size_mbytes = + tablet->tablet_meta()->time_series_compaction_goal_size_mbytes(); + if (level0_total_size >= compaction_goal_size_mbytes * 1024 * 1024) { + return score; + } + + // Condition 2: the number of input files reaches the threshold specified by parameter compaction_file_count_threshold + if (level0_score >= tablet->tablet_meta()->time_series_compaction_file_count_threshold()) { + return score; + } + + // Condition 3: level1 achieve compaction_goal_size + if (tablet->tablet_meta()->time_series_compaction_level_threshold() >= 2) { + checked_rs_metas.sort([](const RowsetMetaSharedPtr& a, const RowsetMetaSharedPtr& b) { + return a->version().first < b->version().first; + }); + int32_t rs_meta_count = 0; + int64_t continuous_size = 0; + for (const auto& rs_meta : checked_rs_metas) { + rs_meta_count++; + continuous_size += rs_meta->total_disk_size(); + if (rs_meta_count >= 2) { + if (continuous_size >= compaction_goal_size_mbytes * 1024 * 1024) { + return score; + } + } + } + } + + int64_t now = UnixMillis(); + int64_t last_cumu = tablet->last_cumu_compaction_success_time(); + if (last_cumu != 0) { + int64_t cumu_interval = now - last_cumu; + + // Condition 4: the time interval between compactions exceeds the value specified by parameter _compaction_time_threshold_second + if (cumu_interval > + (tablet->tablet_meta()->time_series_compaction_time_threshold_seconds() * 1000)) { + return score; + } + } else if (score > 0) { + // If the compaction process has not been successfully executed, + // the condition for triggering compaction based on the last successful compaction time (condition 3) will never be met + tablet->set_last_cumu_compaction_success_time(now); + } + + // Condition 5: If there is a continuous set of empty rowsets, prioritize merging. + std::vector input_rowsets; + std::vector candidate_rowsets = + tablet->pick_candidate_rowsets_to_cumulative_compaction(); + tablet->calc_consecutive_empty_rowsets( + &input_rowsets, candidate_rowsets, + tablet->tablet_meta()->time_series_compaction_empty_rowsets_threshold()); + if (!input_rowsets.empty()) { + return score; + } + + return 0; } void TimeSeriesCumulativeCompactionPolicy::calculate_cumulative_point( diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index 8ca04925bfdd45..6364e8c9e1d866 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -18,6 +18,7 @@ #include "olap/rowset/beta_rowset.h" #include +#include #include #include @@ -556,10 +557,6 @@ Status BetaRowset::add_to_binlog() { } const auto& fs = io::global_local_filesystem(); - - // all segments are in the same directory, so cache binlog_dir without multi times check - std::string binlog_dir; - auto segments_num = num_segments(); VLOG_DEBUG << fmt::format("add rowset to binlog. rowset_id={}, segments_num={}", rowset_id().to_string(), segments_num); @@ -568,17 +565,25 @@ Status BetaRowset::add_to_binlog() { std::vector linked_success_files; Defer remove_linked_files {[&]() { // clear linked files if errors happen if (!status.ok()) { - LOG(WARNING) << "will delete linked success files due to error " << status; + LOG(WARNING) << "will delete linked success files due to error " + << status.to_string_no_stack(); std::vector paths; for (auto& file : linked_success_files) { paths.emplace_back(file); LOG(WARNING) << "will delete linked success file " << file << " due to error"; } static_cast(fs->batch_delete(paths)); - LOG(WARNING) << "done delete linked success files due to error " << status; + LOG(WARNING) << "done delete linked success files due to error " + << status.to_string_no_stack(); } }}; + // The publish_txn might fail even if the add_to_binlog success, so we need to check + // whether a file already exists before linking. + auto errno_is_file_exists = []() { return Errno::no() == EEXIST; }; + + // all segments are in the same directory, so cache binlog_dir without multi times check + std::string binlog_dir; for (int i = 0; i < segments_num; ++i) { auto seg_file = local_segment_path(_tablet_path, rowset_id().to_string(), i); @@ -596,7 +601,7 @@ Status BetaRowset::add_to_binlog() { (std::filesystem::path(binlog_dir) / std::filesystem::path(seg_file).filename()) .string(); VLOG_DEBUG << "link " << seg_file << " to " << binlog_file; - if (!fs->link_file(seg_file, binlog_file).ok()) { + if (!fs->link_file(seg_file, binlog_file).ok() && !errno_is_file_exists()) { status = Status::Error("fail to create hard link. from={}, to={}, errno={}", seg_file, binlog_file, Errno::no()); return status; @@ -613,7 +618,12 @@ Status BetaRowset::add_to_binlog() { std::filesystem::path(index_file).filename()) .string(); VLOG_DEBUG << "link " << index_file << " to " << binlog_index_file; - RETURN_IF_ERROR(fs->link_file(index_file, binlog_index_file)); + if (!fs->link_file(index_file, binlog_index_file).ok() && !errno_is_file_exists()) { + status = Status::Error( + "fail to create hard link. from={}, to={}, errno={}", index_file, + binlog_index_file, Errno::no()); + return status; + } linked_success_files.push_back(binlog_index_file); } } else { @@ -624,7 +634,12 @@ Status BetaRowset::add_to_binlog() { std::filesystem::path(index_file).filename()) .string(); VLOG_DEBUG << "link " << index_file << " to " << binlog_index_file; - RETURN_IF_ERROR(fs->link_file(index_file, binlog_index_file)); + if (!fs->link_file(index_file, binlog_index_file).ok() && !errno_is_file_exists()) { + status = Status::Error( + "fail to create hard link. from={}, to={}, errno={}", index_file, + binlog_index_file, Errno::no()); + return status; + } linked_success_files.push_back(binlog_index_file); } } diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index b96cf4f7e6794e..9d5328de869304 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -871,8 +871,18 @@ Status MapFileColumnIterator::next_batch(size_t* n, vectorized::MutableColumnPtr size_t num_read = *n; auto null_map_ptr = static_cast(*dst).get_null_map_column_ptr(); - bool null_signs_has_null = false; - RETURN_IF_ERROR(_null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + // in not-null to null linked-schemachange mode, + // actually we do not change dat data include meta in footer, + // so may dst from changed meta which is nullable but old data is not nullable, + // if so, we should set null_map to all null by default + if (_null_iterator) { + bool null_signs_has_null = false; + RETURN_IF_ERROR( + _null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + } else { + auto& null_map = assert_cast(*null_map_ptr); + null_map.insert_many_vals(0, num_read); + } DCHECK(num_read == *n); } return Status::OK(); @@ -932,8 +942,18 @@ Status StructFileColumnIterator::next_batch(size_t* n, vectorized::MutableColumn size_t num_read = *n; auto null_map_ptr = static_cast(*dst).get_null_map_column_ptr(); - bool null_signs_has_null = false; - RETURN_IF_ERROR(_null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + // in not-null to null linked-schemachange mode, + // actually we do not change dat data include meta in footer, + // so may dst from changed meta which is nullable but old data is not nullable, + // if so, we should set null_map to all null by default + if (_null_iterator) { + bool null_signs_has_null = false; + RETURN_IF_ERROR( + _null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + } else { + auto& null_map = assert_cast(*null_map_ptr); + null_map.insert_many_vals(0, num_read); + } DCHECK(num_read == *n); } @@ -1086,8 +1106,18 @@ Status ArrayFileColumnIterator::next_batch(size_t* n, vectorized::MutableColumnP auto null_map_ptr = static_cast(*dst).get_null_map_column_ptr(); size_t num_read = *n; - bool null_signs_has_null = false; - RETURN_IF_ERROR(_null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + // in not-null to null linked-schemachange mode, + // actually we do not change dat data include meta in footer, + // so may dst from changed meta which is nullable but old data is not nullable, + // if so, we should set null_map to all null by default + if (_null_iterator) { + bool null_signs_has_null = false; + RETURN_IF_ERROR( + _null_iterator->next_batch(&num_read, null_map_ptr, &null_signs_has_null)); + } else { + auto& null_map = assert_cast(*null_map_ptr); + null_map.insert_many_vals(0, num_read); + } DCHECK(num_read == *n); } diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compaction.cpp b/be/src/olap/rowset/segment_v2/inverted_index_compaction.cpp index f988c46c027c26..dcbdca921ab8e8 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_compaction.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_compaction.cpp @@ -79,7 +79,16 @@ Status compact_column(int64_t index_id, // delete temporary segment_path, only when inverted_index_ram_dir_enable is false if (!config::inverted_index_ram_dir_enable) { - std::ignore = io::global_local_filesystem()->delete_directory(tmp_path.data()); + auto st = io::global_local_filesystem()->delete_directory(tmp_path.data()); + DBUG_EXECUTE_IF("compact_column_delete_tmp_path_error", { + st = Status::Error( + "debug point: compact_column_delete_tmp_path_error in index compaction"); + }) + if (!st.ok()) { + LOG(WARNING) << "compact column failed to delete tmp path: " << tmp_path + << ", error: " << st.to_string(); + return st; + } } return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.cpp index 60006ea84550a2..f1b2b0eaedd4fd 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_compound_reader.cpp @@ -96,12 +96,19 @@ void CSIndexInput::readInternal(uint8_t* b, const int32_t len) { if (start + len > _length) { _CLTHROWA(CL_ERR_IO, "read past EOF"); } - base->setIoContext(_io_ctx); + + if (_io_ctx) { + base->setIoContext(_io_ctx); + } + base->setIndexFile(_is_index_file); base->seek(fileOffset + start); bool read_from_buffer = true; base->readBytes(b, len, read_from_buffer); - base->setIoContext(nullptr); + + if (_io_ctx) { + base->setIoContext(nullptr); + } } CSIndexInput::~CSIndexInput() = default; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_file_reader.cpp index 8d480829a0cd37..813a78f2a3fa86 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_reader.cpp @@ -27,21 +27,27 @@ namespace doris::segment_v2 { -Status InvertedIndexFileReader::init(int32_t read_buffer_size) { +Status InvertedIndexFileReader::init(int32_t read_buffer_size, const io::IOContext* io_ctx) { if (!_inited) { _read_buffer_size = read_buffer_size; if (_storage_format >= InvertedIndexStorageFormatPB::V2) { - auto st = _init_from(read_buffer_size); + auto st = _init_from(read_buffer_size, io_ctx); if (!st.ok()) { return st; } } _inited = true; + } else { + if (_storage_format == InvertedIndexStorageFormatPB::V2) { + if (_stream) { + _stream->setIoContext(io_ctx); + } + } } return Status::OK(); } -Status InvertedIndexFileReader::_init_from(int32_t read_buffer_size) { +Status InvertedIndexFileReader::_init_from(int32_t read_buffer_size, const io::IOContext* io_ctx) { auto index_file_full_path = InvertedIndexDescriptor::get_index_file_path_v2(_index_path_prefix); std::unique_lock lock(_mutex); // Lock for writing @@ -76,6 +82,7 @@ Status InvertedIndexFileReader::_init_from(int32_t read_buffer_size) { err.what()); } _stream = std::unique_ptr(index_input); + _stream->setIoContext(io_ctx); // 3. read file int32_t version = _stream->readInt(); // Read version number diff --git a/be/src/olap/rowset/segment_v2/inverted_index_file_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_file_reader.h index 443d40cfaf0d4f..ed6ee85e7d7bf1 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_file_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_file_reader.h @@ -58,7 +58,8 @@ class InvertedIndexFileReader { _storage_format(storage_format), _idx_file_info(idx_file_info) {} - Status init(int32_t read_buffer_size = config::inverted_index_read_buffer_size); + Status init(int32_t read_buffer_size = config::inverted_index_read_buffer_size, + const io::IOContext* io_ctx = nullptr); Result> open(const TabletIndex* index_meta) const; void debug_file_entries(); std::string get_index_file_cache_key(const TabletIndex* index_meta) const; @@ -70,7 +71,7 @@ class InvertedIndexFileReader { int64_t get_inverted_file_size() const { return _stream == nullptr ? 0 : _stream->length(); } private: - Status _init_from(int32_t read_buffer_size); + Status _init_from(int32_t read_buffer_size, const io::IOContext* io_ctx); Result> _open(int64_t index_id, const std::string& index_suffix) const; diff --git a/be/src/olap/rowset/segment_v2/inverted_index_fs_directory.h b/be/src/olap/rowset/segment_v2/inverted_index_fs_directory.h index dde436054cd35b..41d9fb48356299 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_fs_directory.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_fs_directory.h @@ -180,6 +180,7 @@ class DorisFSDirectory::FSIndexInput : public lucene::store::BufferedIndexInput : BufferedIndexInput(buffer_size) { this->_pos = 0; this->_handle = std::move(handle); + _io_ctx.is_inverted_index = true; } protected: diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index 889fee1fc87ef9..b40f9121125207 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -121,7 +121,8 @@ Status InvertedIndexReader::read_null_bitmap(const io::IOContext* io_ctx, if (!dir) { // TODO: ugly code here, try to refact. - auto st = _inverted_index_file_reader->init(config::inverted_index_read_buffer_size); + auto st = _inverted_index_file_reader->init(config::inverted_index_read_buffer_size, + io_ctx); if (!st.ok()) { LOG(WARNING) << st; return st; @@ -137,7 +138,6 @@ Status InvertedIndexReader::read_null_bitmap(const io::IOContext* io_ctx, InvertedIndexDescriptor::get_temporary_null_bitmap_file_name(); if (dir->fileExists(null_bitmap_file_name)) { null_bitmap_in = dir->openInput(null_bitmap_file_name); - null_bitmap_in->setIoContext(io_ctx); size_t null_bitmap_size = null_bitmap_in->length(); faststring buf; buf.resize(null_bitmap_size); @@ -180,7 +180,8 @@ Status InvertedIndexReader::handle_searcher_cache( SCOPED_RAW_TIMER(&stats->inverted_index_searcher_open_timer); IndexSearcherPtr searcher; - auto st = _inverted_index_file_reader->init(config::inverted_index_read_buffer_size); + auto st = + _inverted_index_file_reader->init(config::inverted_index_read_buffer_size, io_ctx); if (!st.ok()) { LOG(WARNING) << st; return st; @@ -211,6 +212,9 @@ Status InvertedIndexReader::create_index_searcher(lucene::store::Directory* dir, auto searcher_result = DORIS_TRY(index_searcher_builder->get_index_searcher(dir)); *searcher = searcher_result; + // When the meta information has been read, the ioContext needs to be reset to prevent it from being used by other queries. + static_cast(dir)->getDorisIndexInput()->setIoContext(nullptr); + // NOTE: before mem_tracker hook becomes active, we caculate reader memory size by hand. mem_tracker->consume(index_searcher_builder->get_reader_size()); return Status::OK(); diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 5c230c44be183c..911dc094558b1a 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -705,8 +705,10 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet InvertedIndexStorageFormatPB::V1) { for (const auto& index : tablet_schema.inverted_indexes()) { auto index_id = index->index_id(); - auto index_file = ref_tablet->get_segment_index_filepath( - rowset_id, segment_index, index_id); + auto index_file = InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix( + segment_file_path), + index_id, index->get_index_suffix()); auto snapshot_segment_index_file_path = fmt::format("{}/{}_{}_{}.binlog-index", schema_full_path, rowset_id, segment_index, index_id); diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index e00b5b595e20dc..24cda8232f115c 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -463,6 +463,16 @@ Status StorageEngine::_check_file_descriptor_number() { << ", use default configuration instead."; return Status::OK(); } + if (getenv("SKIP_CHECK_ULIMIT") == nullptr) { + LOG(INFO) << "will check 'ulimit' value."; + } else if (std::string(getenv("SKIP_CHECK_ULIMIT")) == "true") { + LOG(INFO) << "the 'ulimit' value check is skipped" + << ", the SKIP_CHECK_ULIMIT env value is " << getenv("SKIP_CHECK_ULIMIT"); + return Status::OK(); + } else { + LOG(INFO) << "the SKIP_CHECK_ULIMIT env value is " << getenv("SKIP_CHECK_ULIMIT") + << ", will check ulimit value."; + } if (l.rlim_cur < config::min_file_descriptor_number) { LOG(ERROR) << "File descriptor number is less than " << config::min_file_descriptor_number << ". Please use (ulimit -n) to set a value equal or greater than " diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index a1a56507ffc67a..c7919b3f8dca24 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -1750,8 +1750,13 @@ Status Tablet::prepare_compaction_and_calculate_permits( } if (!res.ok()) { - tablet->set_last_cumu_compaction_failure_time(UnixMillis()); permits = 0; + // if we meet a delete version, should increase the cumulative point to let base compaction handle the delete version. + // no need to wait 5s. + if (!(res.msg() == "_last_delete_version.first not equal to -1") || + config::enable_sleep_between_delete_cumu_compaction) { + tablet->set_last_cumu_compaction_failure_time(UnixMillis()); + } if (!res.is()) { DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1); return Status::InternalError("prepare cumulative compaction with err: {}", res); @@ -1759,6 +1764,12 @@ Status Tablet::prepare_compaction_and_calculate_permits( // return OK if OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION, so that we don't need to // print too much useless logs. // And because we set permits to 0, so even if we return OK here, nothing will be done. + LOG_INFO( + "cumulative compaction meet delete rowset, increase cumu point without other " + "operation.") + .tag("tablet id:", tablet->tablet_id()) + .tag("after cumulative compaction, cumu point:", + tablet->cumulative_layer_point()); return Status::OK(); } } else if (compaction_type == CompactionType::BASE_COMPACTION) { @@ -2604,30 +2615,6 @@ std::string Tablet::get_segment_filepath(std::string_view rowset_id, int64_t seg return fmt::format("{}/_binlog/{}_{}.dat", _tablet_path, rowset_id, segment_index); } -std::string Tablet::get_segment_index_filepath(std::string_view rowset_id, - std::string_view segment_index, - std::string_view index_id) const { - auto format = _tablet_meta->tablet_schema()->get_inverted_index_storage_format(); - if (format == doris::InvertedIndexStorageFormatPB::V1) { - return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, - index_id); - } else { - return fmt::format("{}/_binlog/{}_{}.idx", _tablet_path, rowset_id, segment_index); - } -} - -std::string Tablet::get_segment_index_filepath(std::string_view rowset_id, int64_t segment_index, - int64_t index_id) const { - auto format = _tablet_meta->tablet_schema()->get_inverted_index_storage_format(); - if (format == doris::InvertedIndexStorageFormatPB::V1) { - return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, - index_id); - } else { - DCHECK(index_id == -1); - return fmt::format("{}/_binlog/{}_{}.idx", _tablet_path, rowset_id, segment_index); - } -} - std::vector Tablet::get_binlog_filepath(std::string_view binlog_version) const { const auto& [rowset_id, num_segments] = get_binlog_info(binlog_version); std::vector binlog_filepath; @@ -2672,10 +2659,25 @@ void Tablet::gc_binlogs(int64_t version) { // add binlog segment files and index files for (int64_t i = 0; i < num_segments; ++i) { - wait_for_deleted_binlog_files.emplace_back(get_segment_filepath(rowset_id, i)); - for (const auto& index : this->tablet_schema()->inverted_indexes()) { - wait_for_deleted_binlog_files.emplace_back( - get_segment_index_filepath(rowset_id, i, index->index_id())); + auto segment_file_path = get_segment_filepath(rowset_id, i); + wait_for_deleted_binlog_files.emplace_back(segment_file_path); + + // index files + if (tablet_schema()->has_inverted_index()) { + if (tablet_schema()->get_inverted_index_storage_format() == + InvertedIndexStorageFormatPB::V1) { + for (const auto& index : tablet_schema()->inverted_indexes()) { + auto index_file = InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix( + segment_file_path), + index->index_id(), index->get_index_suffix()); + wait_for_deleted_binlog_files.emplace_back(index_file); + } + } else { + auto index_file = InvertedIndexDescriptor::get_index_file_path_v2( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_file_path)); + wait_for_deleted_binlog_files.emplace_back(index_file); + } } } }; diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 40b911d6391b9b..d00476f044191c 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -441,11 +441,6 @@ class Tablet final : public BaseTablet { std::string get_segment_filepath(std::string_view rowset_id, std::string_view segment_index) const; std::string get_segment_filepath(std::string_view rowset_id, int64_t segment_index) const; - std::string get_segment_index_filepath(std::string_view rowset_id, - std::string_view segment_index, - std::string_view index_id) const; - std::string get_segment_index_filepath(std::string_view rowset_id, int64_t segment_index, - int64_t index_id) const; bool can_add_binlog(uint64_t total_binlog_size) const; void gc_binlogs(int64_t version); Status ingest_binlog_metas(RowsetBinlogMetasPB* metas_pb); diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index aaaf3e203280cf..2adb23e31a34aa 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -409,8 +409,9 @@ Status EngineStorageMigrationTask::_copy_index_and_data_files( InvertedIndexStorageFormatPB::V1) { for (const auto& index : tablet_schema.inverted_indexes()) { auto index_id = index->index_id(); - auto index_file = - _tablet->get_segment_index_filepath(rowset_id, segment_index, index_id); + auto index_file = InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_file_path), + index_id, index->get_index_suffix()); auto snapshot_segment_index_file_path = fmt::format("{}/{}_{}_{}.binlog-index", full_path, rowset_id, segment_index, index_id); diff --git a/be/src/olap/txn_manager.cpp b/be/src/olap/txn_manager.cpp index d227f53053128b..c54b9c5e8f980f 100644 --- a/be/src/olap/txn_manager.cpp +++ b/be/src/olap/txn_manager.cpp @@ -548,8 +548,9 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, if (!status.ok()) { return Status::Error( "add rowset to binlog failed. when publish txn rowset_id: {}, tablet id: {}, " - "txn id: {}", - rowset->rowset_id().to_string(), tablet_id, transaction_id); + "txn id: {}, status: {}", + rowset->rowset_id().to_string(), tablet_id, transaction_id, + status.to_string_no_stack()); } } diff --git a/be/src/pipeline/common/set_utils.h b/be/src/pipeline/common/set_utils.h index 2caf5b7d0b814c..38a82a501ff534 100644 --- a/be/src/pipeline/common/set_utils.h +++ b/be/src/pipeline/common/set_utils.h @@ -20,13 +20,20 @@ #include #include -#include "pipeline/exec/join/join_op.h" #include "vec/common/hash_table/hash_map_util.h" namespace doris { +struct RowRefWithFlag { + bool visited; + uint32_t row_num = 0; + RowRefWithFlag() = default; + RowRefWithFlag(size_t row_num_count, bool is_visited = false) + : visited(is_visited), row_num(row_num_count) {} +}; + template -using SetData = PHHashMap>; +using SetData = PHHashMap>; template using SetFixedKeyHashTableContext = vectorized::MethodKeysFixed>; @@ -39,9 +46,8 @@ using SetPrimaryTypeHashTableContextNullable = vectorized::MethodSingleNullableC vectorized::MethodOneNumber>>>; using SetSerializedHashTableContext = - vectorized::MethodSerialized>; -using SetMethodOneString = - vectorized::MethodStringNoCache>; + vectorized::MethodSerialized>; +using SetMethodOneString = vectorized::MethodStringNoCache>; using SetHashTableVariants = std::variant { public: @@ -55,4 +56,5 @@ class AssertNumRowsOperatorX final : public StreamingOperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/cache_source_operator.h b/be/src/pipeline/exec/cache_source_operator.h index e764323846b153..146c984d04aa3f 100644 --- a/be/src/pipeline/exec/cache_source_operator.h +++ b/be/src/pipeline/exec/cache_source_operator.h @@ -25,6 +25,7 @@ #include "pipeline/query_cache/query_cache.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace vectorized { @@ -101,4 +102,5 @@ class CacheSourceOperatorX final : public OperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/data_queue.h b/be/src/pipeline/exec/data_queue.h index f5bd84cc278d0a..d97f58c0debdb6 100644 --- a/be/src/pipeline/exec/data_queue.h +++ b/be/src/pipeline/exec/data_queue.h @@ -29,6 +29,7 @@ #include "vec/core/block.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class Dependency; @@ -108,4 +109,5 @@ class DataQueue { SpinLock _source_lock; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/datagen_operator.h b/be/src/pipeline/exec/datagen_operator.h index bada5ec4080d08..ffc2c6f946fb3a 100644 --- a/be/src/pipeline/exec/datagen_operator.h +++ b/be/src/pipeline/exec/datagen_operator.h @@ -24,6 +24,7 @@ #include "pipeline/exec/operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -70,4 +71,5 @@ class DataGenSourceOperatorX final : public OperatorX { std::vector _runtime_filter_descs; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/empty_set_operator.cpp b/be/src/pipeline/exec/empty_set_operator.cpp index 7233e46dfd1e52..2dfe9701558da0 100644 --- a/be/src/pipeline/exec/empty_set_operator.cpp +++ b/be/src/pipeline/exec/empty_set_operator.cpp @@ -22,6 +22,7 @@ #include "pipeline/exec/operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status EmptySetSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { @@ -29,4 +30,5 @@ Status EmptySetSourceOperatorX::get_block(RuntimeState* state, vectorized::Block return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/empty_set_operator.h b/be/src/pipeline/exec/empty_set_operator.h index 6b200bfdbde249..d8e920b256494d 100644 --- a/be/src/pipeline/exec/empty_set_operator.h +++ b/be/src/pipeline/exec/empty_set_operator.h @@ -22,6 +22,7 @@ #include "operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class EmptySetLocalState final : public PipelineXLocalState { public: @@ -43,4 +44,5 @@ class EmptySetSourceOperatorX final : public OperatorX { [[nodiscard]] bool is_source() const override { return true; } }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/es_scan_operator.h b/be/src/pipeline/exec/es_scan_operator.h index 2ae562e4fc7f32..6e64110997e3af 100644 --- a/be/src/pipeline/exec/es_scan_operator.h +++ b/be/src/pipeline/exec/es_scan_operator.h @@ -26,6 +26,7 @@ #include "pipeline/exec/scan_operator.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class NewEsScanner; @@ -86,4 +87,5 @@ class EsScanOperatorX final : public ScanOperatorX { std::vector _column_names; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 65e7698737076e..e3f895444d4168 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -47,6 +47,7 @@ #include "vec/sink/vdata_stream_sender.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { BroadcastPBlockHolder::~BroadcastPBlockHolder() { @@ -576,4 +577,5 @@ void ExchangeSinkBuffer::update_profile(RuntimeProfile* profile) { } } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index b2eb32414feca2..458c7c3f66e3ee 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -40,6 +40,7 @@ #include "util/ref_count_closure.h" namespace doris { +#include "common/compile_check_begin.h" class PTransmitDataParams; class TUniqueId; @@ -318,4 +319,5 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index aa893fc0a26f2e..e7fed76be8fa16 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -112,6 +112,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf } } only_local_exchange = local_size == channels.size(); + _rpc_channels_num = channels.size() - local_size; if (!only_local_exchange) { _sink_buffer = p.get_sink_buffer(state->fragment_instance_id().lo); @@ -206,17 +207,12 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { std::mt19937 g(rd()); shuffle(channels.begin(), channels.end(), g); } - 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()) { - local_size++; _last_local_channel_idx = i; } } - only_local_exchange = local_size == channels.size(); - - _rpc_channels_num = channels.size() - local_size; PUniqueId id; id.set_hi(_state->query_id().hi); @@ -228,7 +224,7 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { _parent->operator_id(), _parent->node_id(), "BroadcastDependency", true); _broadcast_pb_mem_limiter = vectorized::BroadcastPBlockHolderMemLimiter::create_shared(_broadcast_dependency); - } else if (local_size > 0) { + } else if (!only_local_exchange) { size_t dep_id = 0; for (auto& channel : channels) { if (channel->is_local()) { diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index e88389b1d7bb5a..85575beb9f7e47 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -31,6 +31,7 @@ #include "vec/sink/vdata_stream_sender.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; class TDataSink; @@ -263,4 +264,5 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX { std::vector _nulls_first; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/file_scan_operator.h b/be/src/pipeline/exec/file_scan_operator.h index 2777a013d62f61..87c5bcd2e54de5 100644 --- a/be/src/pipeline/exec/file_scan_operator.h +++ b/be/src/pipeline/exec/file_scan_operator.h @@ -29,6 +29,7 @@ #include "vec/exec/scan/split_source_connector.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class VFileScanner; } // namespace vectorized @@ -86,4 +87,5 @@ class FileScanOperatorX final : public ScanOperatorX { const std::string _table_name; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline 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 e469aee8df595c..5eabb280c4315d 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.h +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.h @@ -22,8 +22,9 @@ #include "runtime/group_commit_mgr.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class OlapTableBlockConvertor; -} +} // namespace doris::vectorized namespace doris::pipeline { @@ -125,4 +126,5 @@ class GroupCommitBlockSinkOperatorX final TGroupCommitMode::type _group_commit_mode; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/group_commit_scan_operator.h b/be/src/pipeline/exec/group_commit_scan_operator.h index 46f50f3772440a..d1428899ede6b9 100644 --- a/be/src/pipeline/exec/group_commit_scan_operator.h +++ b/be/src/pipeline/exec/group_commit_scan_operator.h @@ -27,6 +27,7 @@ #include "runtime/group_commit_mgr.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class GroupCommitOperatorX; class GroupCommitLocalState final : public ScanLocalState { @@ -60,4 +61,5 @@ class GroupCommitOperatorX final : public ScanOperatorX { const int64_t _table_id; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index b2a79a941f79e7..19e8493e596a7e 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -135,26 +135,16 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu } }}; - if (!_runtime_filter_slots || _runtime_filters.empty() || state->is_cancelled()) { + if (!_runtime_filter_slots || _runtime_filters.empty() || state->is_cancelled() || !_eos) { return Base::close(state, exec_status); } try { - if (state->get_task()->wake_up_by_downstream()) { - if (_should_build_hash_table) { - // partitial ignore rf to make global rf work - RETURN_IF_ERROR( - _runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); - RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); - } else { - // do not publish filter coz local rf not inited and useless - return Base::close(state, exec_status); - } + if (state->get_task()->wake_up_early()) { + // partitial ignore rf to make global rf work or ignore useless rf + RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); } else if (_should_build_hash_table) { - if (p._shared_hashtable_controller && - !p._shared_hash_table_context->complete_build_stage) { - return Status::InternalError("close before sink meet eos"); - } auto* block = _shared_state->build_block.get(); uint64_t hash_table_size = block ? block->rows() : 0; { @@ -166,26 +156,25 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu SCOPED_TIMER(_runtime_filter_compute_timer); _runtime_filter_slots->insert(block); } - } else if ((p._shared_hashtable_controller && !p._shared_hash_table_context->signaled) || - (p._shared_hash_table_context && - !p._shared_hash_table_context->complete_build_stage)) { - throw Exception(ErrorCode::INTERNAL_ERROR, "build_sink::close meet error state"); - } else { - RETURN_IF_ERROR( - _runtime_filter_slots->copy_from_shared_context(p._shared_hash_table_context)); } SCOPED_TIMER(_publish_runtime_filter_timer); RETURN_IF_ERROR(_runtime_filter_slots->publish(state, !_should_build_hash_table)); } catch (Exception& e) { + bool blocked_by_complete_build_stage = p._shared_hashtable_controller && + !p._shared_hash_table_context->complete_build_stage; + bool blocked_by_shared_hash_table_signal = !_should_build_hash_table && + p._shared_hashtable_controller && + !p._shared_hash_table_context->signaled; + return Status::InternalError( - "rf process meet error: {}, wake_up_by_downstream: {}, should_build_hash_table: " - "{}, _finish_dependency: {}, complete_build_stage: {}, shared_hash_table_signaled: " + "rf process meet error: {}, wake_up_early: {}, should_build_hash_table: " + "{}, _finish_dependency: {}, blocked_by_complete_build_stage: {}, " + "blocked_by_shared_hash_table_signal: " "{}", - e.to_string(), state->get_task()->wake_up_by_downstream(), _should_build_hash_table, - _finish_dependency->debug_string(), - p._shared_hash_table_context && !p._shared_hash_table_context->complete_build_stage, - p._shared_hashtable_controller && !p._shared_hash_table_context->signaled); + e.to_string(), state->get_task()->wake_up_early(), _should_build_hash_table, + _finish_dependency->debug_string(), blocked_by_complete_build_stage, + blocked_by_shared_hash_table_signal); } return Base::close(state, exec_status); } @@ -479,7 +468,6 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); - local_state._eos = eos; if (local_state._should_build_hash_table) { // If eos or have already met a null value using short-circuit strategy, we do not need to pull // data from probe side. @@ -556,6 +544,9 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* return _shared_hash_table_context->status; } + RETURN_IF_ERROR(local_state._runtime_filter_slots->copy_from_shared_context( + _shared_hash_table_context)); + local_state.profile()->add_info_string( "SharedHashTableFrom", print_id( @@ -581,6 +572,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* } if (eos) { + local_state._eos = true; local_state.init_short_circuit_for_probe(); // Since the comparison of null values is meaningless, null aware left anti/semi join should not output null // when the build side is not empty. diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index 7c663b256832ed..0db525f1bf5222 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -17,6 +17,8 @@ #include "hashjoin_probe_operator.h" +#include + #include #include "common/cast_set.h" @@ -240,7 +242,7 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc // If we use a short-circuit strategy, should return block directly by add additional null data. auto block_rows = local_state._probe_block.rows(); if (local_state._probe_eos && block_rows == 0) { - *eos = local_state._probe_eos; + *eos = true; return Status::OK(); } @@ -616,21 +618,34 @@ Status HashJoinProbeOperatorX::open(RuntimeState* state) { size_t idx = 0; for (const auto* slot : slots_to_check) { auto data_type = slot->get_data_type_ptr(); - auto target_data_type = idx < right_col_idx ? _left_table_data_types[idx] - : _right_table_data_types[idx - right_col_idx]; + const auto slot_on_left = idx < right_col_idx; + auto target_data_type = slot_on_left ? _left_table_data_types[idx] + : _right_table_data_types[idx - right_col_idx]; ++idx; if (data_type->equals(*target_data_type)) { continue; } - auto data_type_non_nullable = vectorized::remove_nullable(data_type); - if (data_type_non_nullable->equals(*target_data_type)) { + /// For outer join(left/right/full), the non-nullable columns may be converted to nullable. + const auto accept_nullable_not_match = + _join_op == TJoinOp::FULL_OUTER_JOIN || + (slot_on_left ? _join_op == TJoinOp::RIGHT_OUTER_JOIN + : _join_op == TJoinOp::LEFT_OUTER_JOIN); + + if (accept_nullable_not_match) { + auto data_type_non_nullable = vectorized::remove_nullable(data_type); + if (data_type_non_nullable->equals(*target_data_type)) { + continue; + } + } else if (data_type->equals(*target_data_type)) { continue; } - return Status::InternalError("intermediate slot({}) data type not match: '{}' vs '{}'", - slot->id(), data_type->get_name(), - _left_table_data_types[idx]->get_name()); + return Status::InternalError( + "Join node(id={}, OP={}) intermediate slot({}, #{})'s on {} table data type not " + "match: '{}' vs '{}'", + _node_id, _join_op, slot->col_name(), slot->id(), (slot_on_left ? "left" : "right"), + data_type->get_name(), target_data_type->get_name()); } _build_side_child.reset(); diff --git a/be/src/pipeline/exec/hive_table_sink_operator.h b/be/src/pipeline/exec/hive_table_sink_operator.h index 58e705fd8e46c7..8af3e5bd5e9764 100644 --- a/be/src/pipeline/exec/hive_table_sink_operator.h +++ b/be/src/pipeline/exec/hive_table_sink_operator.h @@ -21,6 +21,7 @@ #include "vec/sink/writer/vhive_table_writer.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class HiveTableSinkOperatorX; @@ -83,4 +84,5 @@ class HiveTableSinkOperatorX final : public DataSinkOperatorX { TOdbcTableType::type _table_type; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/jdbc_table_sink_operator.h b/be/src/pipeline/exec/jdbc_table_sink_operator.h index 3ea702fd0baf0a..a0dae301a5fcad 100644 --- a/be/src/pipeline/exec/jdbc_table_sink_operator.h +++ b/be/src/pipeline/exec/jdbc_table_sink_operator.h @@ -23,6 +23,7 @@ #include "vec/sink/writer/vjdbc_table_writer.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class JdbcTableSinkOperatorX; class JdbcTableSinkLocalState final @@ -59,4 +60,5 @@ class JdbcTableSinkOperatorX final : public DataSinkOperatorX -struct Batch { - static constexpr uint32_t MAX_SIZE = 7; /// Adequate values are 3, 7, 15, 31. - - uint8_t size = 0; /// It's smaller than size_t but keeps align in Arena. - Batch* next = nullptr; - RowRefType row_refs[MAX_SIZE]; - - Batch(Batch* parent) : next(parent) {} - - bool full() const { return size == MAX_SIZE; } - - Batch* insert(RowRefType&& row_ref, vectorized::Arena& pool) { - if (full()) { - auto batch = pool.alloc>(); - *batch = Batch(this); - batch->insert(std::move(row_ref), pool); - return batch; - } - - row_refs[size++] = std::move(row_ref); - return this; - } -}; - -template -class ForwardIterator { -public: - using RowRefType = typename RowRefListType::RowRefType; - ForwardIterator() : root(nullptr), first(false), batch(nullptr), position(0) {} - - ForwardIterator(RowRefListType* begin) - : root(begin), first(true), batch((&root->next)), position(0) {} - - RowRefType& operator*() { - if (first) { - return *root; - } - return batch->operator[](position); - } - - RowRefType* operator->() { return &(**this); } - - void operator++() { - if (first) { - first = false; - return; - } - - if (batch && position < batch->size()) { - ++position; - } - } - - bool ok() const { return first || (batch && position < batch->size()); } - -private: - RowRefListType* root = nullptr; - bool first; - std::vector* batch = nullptr; - size_t position; -}; - -struct RowRefList : RowRef { - using RowRefType = RowRef; - - RowRefList() = default; - RowRefList(size_t row_num_) : RowRef(row_num_) {} - - ForwardIterator begin() { return {this}; } - - /// insert element after current one - void insert(RowRefType&& row_ref, vectorized::Arena& pool) { next.emplace_back(row_ref); } - - void clear() { next.clear(); } - -private: - friend class ForwardIterator; - std::vector next; -}; - -struct RowRefListWithFlag : RowRef { - using RowRefType = RowRef; - - RowRefListWithFlag() = default; - RowRefListWithFlag(size_t row_num_) : RowRef(row_num_) {} - - ForwardIterator begin() { return {this}; } - - /// insert element after current one - void insert(RowRefType&& row_ref, vectorized::Arena& pool) { next.emplace_back(row_ref); } - - void clear() { next.clear(); } - - bool visited = false; - -private: - friend class ForwardIterator; - std::vector next; -}; - -struct RowRefListWithFlags : RowRefWithFlag { - using RowRefType = RowRefWithFlag; - - RowRefListWithFlags() = default; - RowRefListWithFlags(size_t row_num_) : RowRefWithFlag(row_num_) {} - - ForwardIterator begin() { return {this}; } - - /// insert element after current one - void insert(RowRefType&& row_ref, vectorized::Arena& pool) { next.emplace_back(row_ref); } - - void clear() { next.clear(); } - -private: - friend class ForwardIterator; - std::vector next; -}; - -} // namespace doris 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 14e0edd977f57b..91fd82f0644939 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe.h @@ -19,7 +19,6 @@ #include -#include "join_op.h" #include "vec/columns/column.h" #include "vec/columns/columns_number.h" #include "vec/common/arena.h" diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h index 9d79a97397ff77..2a24f6a0492f3b 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.h +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -20,6 +20,7 @@ #include "operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" template class JoinBuildSinkOperatorX; @@ -78,4 +79,5 @@ class JoinBuildSinkOperatorX : public DataSinkOperatorX { const std::vector _runtime_filter_descs; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index 078806cea4fc5a..161fd18fa1dab8 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -20,6 +20,7 @@ #include "operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" template class JoinProbeOperatorX; template @@ -123,4 +124,5 @@ class JoinProbeOperatorX : public StatefulOperatorX { const bool _use_specific_projections; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/memory_scratch_sink_operator.h b/be/src/pipeline/exec/memory_scratch_sink_operator.h index c74659d15b96f2..352826955fca99 100644 --- a/be/src/pipeline/exec/memory_scratch_sink_operator.h +++ b/be/src/pipeline/exec/memory_scratch_sink_operator.h @@ -23,6 +23,7 @@ #include "runtime/result_queue_mgr.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class MemoryScratchSinkOperatorX; class MemoryScratchSinkLocalState final : public PipelineXSinkLocalState { @@ -67,4 +68,5 @@ class MemoryScratchSinkOperatorX final : public DataSinkOperatorX { TUserIdentity _user_identity; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.h b/be/src/pipeline/exec/multi_cast_data_stream_sink.h index 57b5974064b6a2..9d69b3fb5bdc9e 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.h @@ -20,6 +20,7 @@ #include "operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class MultiCastDataStreamSinkOperatorX; class MultiCastDataStreamSinkLocalState final @@ -75,4 +76,5 @@ class MultiCastDataStreamSinkOperatorX final std::atomic _num_dests; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 51a73cf0c2b053..380538d0ac0805 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -20,6 +20,7 @@ #include "vec/sink/vdata_stream_sender.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class Dependency; struct MultiCastBlock { @@ -84,4 +85,5 @@ class MultiCastDataStreamer { std::vector _dependencies; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.h b/be/src/pipeline/exec/nested_loop_join_build_operator.h index 5c41088a7059d4..11bcba2bd8fc3a 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.h @@ -23,6 +23,7 @@ #include "pipeline/exec/join_build_sink_operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class NestedLoopJoinBuildSinkOperatorX; @@ -89,4 +90,5 @@ class NestedLoopJoinBuildSinkOperatorX final RowDescriptor _row_descriptor; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 7b06e216b81bf7..34fa741ff1ec00 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -40,6 +40,7 @@ #include "vec/functions/in.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status OlapScanLocalState::_init_profile() { RETURN_IF_ERROR(ScanLocalState::_init_profile()); @@ -347,13 +348,13 @@ Status OlapScanLocalState::_init_scanners(std::list* s int ranges_per_scanner = std::max(1, (int)ranges->size() / std::min(scanners_per_tablet, size_based_scanners_per_tablet)); - int num_ranges = ranges->size(); - for (int i = 0; i < num_ranges;) { + int64_t num_ranges = ranges->size(); + for (int64_t i = 0; i < num_ranges;) { std::vector scanner_ranges; scanner_ranges.push_back((*ranges)[i].get()); ++i; - for (int j = 1; i < num_ranges && j < ranges_per_scanner && - (*ranges)[i]->end_include == (*ranges)[i - 1]->end_include; + for (int64_t j = 1; i < num_ranges && j < ranges_per_scanner && + (*ranges)[i]->end_include == (*ranges)[i - 1]->end_include; ++j, ++i) { scanner_ranges.push_back((*ranges)[i].get()); } @@ -587,4 +588,5 @@ OlapScanOperatorX::OlapScanOperatorX(ObjectPool* pool, const TPlanNode& tnode, i } } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/olap_scan_operator.h b/be/src/pipeline/exec/olap_scan_operator.h index 9e8624b3a0b255..91980d6a3f172b 100644 --- a/be/src/pipeline/exec/olap_scan_operator.h +++ b/be/src/pipeline/exec/olap_scan_operator.h @@ -26,6 +26,7 @@ #include "pipeline/exec/scan_operator.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { class NewOlapScanner; @@ -198,4 +199,5 @@ class OlapScanOperatorX final : public ScanOperatorX { TQueryCacheParam _cache_param; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index 8a9ffaaf769c31..3453a57a67b9bc 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -21,6 +21,7 @@ #include "vec/sink/writer/vtablet_writer.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class OlapTableSinkOperatorX; @@ -75,4 +76,5 @@ class OlapTableSinkOperatorX final : public DataSinkOperatorX; template class AsyncWriterSink; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index c84c4e7b43f981..a2c8e110cedac3 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -39,6 +39,7 @@ #include "vec/runtime/vdata_stream_recvr.h" namespace doris { +#include "common/compile_check_begin.h" class RowDescriptor; class RuntimeState; class TDataSink; @@ -859,4 +860,5 @@ class AsyncWriterSink : public PipelineXSinkLocalState { std::shared_ptr _finish_dependency; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index 48b8fe9cb765a1..d0c28afe9de5ba 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -24,6 +24,7 @@ #include "vec/common/hash_table/hash.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status PartitionSortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); @@ -66,7 +67,7 @@ PartitionSortSinkOperatorX::PartitionSortSinkOperatorX(ObjectPool* pool, int ope _pool(pool), _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), _limit(tnode.limit), - _partition_exprs_num(tnode.partition_sort_node.partition_exprs.size()), + _partition_exprs_num(cast_set(tnode.partition_sort_node.partition_exprs.size())), _topn_phase(tnode.partition_sort_node.ptopn_phase), _has_global_limit(tnode.partition_sort_node.has_global_limit), _top_n_algorithm(tnode.partition_sort_node.top_n_algorithm), @@ -212,7 +213,7 @@ Status PartitionSortSinkOperatorX::_emplace_into_hash_table( }; SCOPED_TIMER(local_state._emplace_key_timer); - int row = num_rows; + int64_t row = num_rows; for (row = row - 1; row >= 0 && !local_state._is_need_passthrough; --row) { auto& mapped = *agg_method.lazy_emplace(state, row, creator, creator_for_null_key); @@ -274,4 +275,5 @@ bool PartitionSortSinkLocalState::check_whether_need_passthrough() { } // NOLINTEND(readability-simplify-boolean-expr) +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index 6926445f18f2f4..32bbf38202713f 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -24,6 +24,7 @@ #include "vec/common/sort/partition_sorter.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class PartitionSortSinkOperatorX; class PartitionSortSinkLocalState : public PipelineXSinkLocalState { @@ -110,4 +111,5 @@ class PartitionSortSinkOperatorX final : public DataSinkOperatorX { @@ -324,4 +325,5 @@ class PartitionedAggSinkOperatorX : public DataSinkOperatorX(state, _shared_state->shared_from_this(), exception_catch_func)); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h index 7e73241745e029..6fb0ecaba01e20 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h @@ -22,6 +22,7 @@ #include "operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -99,4 +100,5 @@ class PartitionedAggSourceOperatorX : public OperatorX std::unique_ptr _agg_source_operator; }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp index 20b25d54ff9f16..3e7f95374f53d2 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -23,6 +23,7 @@ #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" PartitionedHashJoinProbeLocalState::PartitionedHashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) @@ -866,4 +867,5 @@ Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, vectori return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h index f8fc0780b6fc3f..a19e88d7203e62 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -27,6 +27,7 @@ #include "pipeline/exec/spill_utils.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -213,4 +214,5 @@ class PartitionedHashJoinProbeOperatorX final }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index 878c3870946f1c..852dccae71ca3b 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -23,6 +23,7 @@ #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, doris::pipeline::LocalSinkStateInfo& info) { @@ -246,11 +247,11 @@ Status PartitionedHashJoinSinkLocalState::revoke_memory(RuntimeState* state) { return _revoke_unpartitioned_block(state); } - _spilling_streams_count = _shared_state->partitioned_build_blocks.size(); + _spilling_streams_count = cast_set(_shared_state->partitioned_build_blocks.size()); auto query_id = state->query_id(); - for (size_t i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { + for (int i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { vectorized::SpillStreamSPtr& spilling_stream = _shared_state->spilled_streams[i]; auto& mutable_block = _shared_state->partitioned_build_blocks[i]; @@ -555,4 +556,5 @@ Status PartitionedHashJoinSinkOperatorX::revoke_memory(RuntimeState* state) { return local_state.revoke_memory(state); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index d1fe30e06f2dd2..e16e52dcaf9453 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -28,6 +28,7 @@ #include "vec/runtime/partitioner.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -148,4 +149,5 @@ class PartitionedHashJoinSinkOperatorX }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index 5c94d43f0d1e05..48131e0d96e4c6 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -24,6 +24,7 @@ #include "vec/core/block.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -221,8 +222,7 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp _repeat_id_idx++; - int size = _repeat_id_list.size(); - if (_repeat_id_idx >= size) { + if (_repeat_id_idx >= _repeat_id_list.size()) { _intermediate_block->clear(); _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); _repeat_id_idx = 0; @@ -251,4 +251,5 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 31f88f37231aaa..2c2af32de0b0fb 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -23,6 +23,7 @@ #include "pipeline/exec/operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -92,4 +93,5 @@ class RepeatOperatorX final : public StatefulOperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/result_file_sink_operator.cpp b/be/src/pipeline/exec/result_file_sink_operator.cpp index c65b9dda89d0ec..f806d9533d9e4c 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.cpp +++ b/be/src/pipeline/exec/result_file_sink_operator.cpp @@ -28,6 +28,7 @@ #include "vec/sink/vdata_stream_sender.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" ResultFileSinkLocalState::ResultFileSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) @@ -143,4 +144,5 @@ Status ResultFileSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_ return local_state.sink(state, in_block, eos); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index e9f2b8eeb9c670..c3c5e345f77e1a 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -21,6 +21,7 @@ #include "vec/sink/writer/vfile_result_writer.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class BroadcastPBlockHolder; } // namespace doris::vectorized @@ -88,4 +89,5 @@ class ResultFileSinkOperatorX final : public DataSinkOperatorX _sender = nullptr; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index f8196910021b2c..8aeecbbddc12dc 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -35,6 +35,7 @@ #include "vec/sink/vmysql_result_writer.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); @@ -208,4 +209,5 @@ Status ResultSinkLocalState::close(RuntimeState* state, Status exec_status) { return final_status; } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index 339c167825643b..479343ed6d5ea5 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -25,6 +25,7 @@ #include "runtime/result_writer.h" namespace doris { +#include "common/compile_check_begin.h" class BufferControlBlock; namespace pipeline { @@ -172,4 +173,5 @@ class ResultSinkOperatorX final : public DataSinkOperatorX }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index 4519a3ca283f6f..c6c9cdf405d5a4 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -35,8 +35,9 @@ #include "vec/utils/util.hpp" namespace doris::vectorized { +#include "common/compile_check_begin.h" class ScannerDelegate; -} +} // namespace doris::vectorized namespace doris::pipeline { @@ -436,4 +437,5 @@ class ScanOperatorX : public OperatorX { std::vector topn_filter_source_node_ids; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/schema_scan_operator.cpp b/be/src/pipeline/exec/schema_scan_operator.cpp index ddc2821cac14a1..2e2f80f5e24838 100644 --- a/be/src/pipeline/exec/schema_scan_operator.cpp +++ b/be/src/pipeline/exec/schema_scan_operator.cpp @@ -26,6 +26,7 @@ #include "vec/data_types/data_type_factory.hpp" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -144,7 +145,7 @@ Status SchemaScanOperatorX::open(RuntimeState* state) { return Status::InternalError("Failed to get tuple descriptor."); } - _slot_num = _dest_tuple_desc->slots().size(); + _slot_num = cast_set(_dest_tuple_desc->slots().size()); // get src tuple desc const auto* schema_table = static_cast(_dest_tuple_desc->table_desc()); @@ -269,4 +270,5 @@ Status SchemaScanOperatorX::get_block(RuntimeState* state, vectorized::Block* bl return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/schema_scan_operator.h b/be/src/pipeline/exec/schema_scan_operator.h index c8ddf885e98a0f..2d861002748163 100644 --- a/be/src/pipeline/exec/schema_scan_operator.h +++ b/be/src/pipeline/exec/schema_scan_operator.h @@ -24,6 +24,7 @@ #include "operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -88,4 +89,5 @@ class SchemaScanOperatorX final : public OperatorX { std::unique_ptr _schema_scanner; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/select_operator.h b/be/src/pipeline/exec/select_operator.h index 5370cd9e293c34..584a6f74308903 100644 --- a/be/src/pipeline/exec/select_operator.h +++ b/be/src/pipeline/exec/select_operator.h @@ -22,6 +22,7 @@ #include "operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class SelectOperatorX; class SelectLocalState final : public PipelineXLocalState { @@ -55,4 +56,5 @@ class SelectOperatorX final : public StreamingOperatorX { [[nodiscard]] bool is_source() const override { return false; } }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 4c250d5603b499..db487b0f9e7252 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -25,6 +25,7 @@ #include "vec/common/hash_table/hash_table_set_probe.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace vectorized { @@ -69,7 +70,7 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); - auto probe_rows = in_block->rows(); + uint32_t probe_rows = cast_set(in_block->rows()); if (probe_rows > 0) { { SCOPED_TIMER(local_state._extract_probe_data_timer); @@ -220,8 +221,8 @@ void SetProbeSinkOperatorX::_refresh_hash_table( ? (valid_element_in_hash_tbl < arg.hash_table ->size()) // When intersect, shrink as long as the element decreases - : (valid_element_in_hash_tbl < - arg.hash_table->size() * + : ((double)valid_element_in_hash_tbl < + (double)arg.hash_table->size() * need_shrink_ratio); // When except, element decreases need to within the 'need_shrink_ratio' before shrinking if (is_need_shrink) { @@ -231,7 +232,7 @@ void SetProbeSinkOperatorX::_refresh_hash_table( local_state._shared_state->valid_element_in_hash_tbl); while (iter != iter_end) { auto& mapped = iter->get_second(); - auto it = mapped.begin(); + auto* it = &mapped; if constexpr (is_intersect) { if (it->visited) { @@ -249,7 +250,7 @@ void SetProbeSinkOperatorX::_refresh_hash_table( } else if (is_intersect) { while (iter != iter_end) { auto& mapped = iter->get_second(); - auto it = mapped.begin(); + auto* it = &mapped; it->visited = false; ++iter; } @@ -269,4 +270,5 @@ template class SetProbeSinkLocalState; template class SetProbeSinkOperatorX; template class SetProbeSinkOperatorX; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index 368ea812cdfe01..6b764c1e509951 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -23,6 +23,7 @@ #include "operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace vectorized { @@ -116,4 +117,5 @@ class SetProbeSinkOperatorX final : public DataSinkOperatorX::_get_data_in_hashtable( auto block_size = 0; auto add_result = [&local_state, &block_size, this](auto value) { - auto it = value.begin(); + auto* it = &value; if constexpr (is_intersect) { if (it->visited) { //intersected: have done probe, so visited values it's the result _add_result_columns(local_state, value, block_size); @@ -147,8 +147,8 @@ Status SetSourceOperatorX::_get_data_in_hashtable( *eos = iter == hash_table_ctx.hash_table->end(); if (*eos && hash_table_ctx.hash_table->has_null_key_data()) { - auto value = hash_table_ctx.hash_table->template get_null_key_data(); - if constexpr (std::is_same_v>) { + auto value = hash_table_ctx.hash_table->template get_null_key_data(); + if constexpr (std::is_same_v>) { add_result(value); } } @@ -168,15 +168,13 @@ Status SetSourceOperatorX::_get_data_in_hashtable( template void SetSourceOperatorX::_add_result_columns( - SetSourceLocalState& local_state, RowRefListWithFlags& value, - int& block_size) { + SetSourceLocalState& local_state, RowRefWithFlag& value, int& block_size) { auto& build_col_idx = local_state._shared_state->build_col_idx; auto& build_block = local_state._shared_state->build_block; - auto it = value.begin(); for (auto idx = build_col_idx.begin(); idx != build_col_idx.end(); ++idx) { auto& column = *build_block.get_by_position(idx->second).column; - local_state._mutable_cols[idx->first]->insert_from(column, it->row_num); + local_state._mutable_cols[idx->first]->insert_from(column, value.row_num); } block_size++; } diff --git a/be/src/pipeline/exec/set_source_operator.h b/be/src/pipeline/exec/set_source_operator.h index 976ffde3bf23ea..d881e9277fb7b6 100644 --- a/be/src/pipeline/exec/set_source_operator.h +++ b/be/src/pipeline/exec/set_source_operator.h @@ -83,8 +83,8 @@ class SetSourceOperatorX final : public OperatorX& local_state, - RowRefListWithFlags& value, int& block_size); + void _add_result_columns(SetSourceLocalState& local_state, RowRefWithFlag& value, + int& block_size); const size_t _child_quantity; }; #include "common/compile_check_end.h" diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 072f28723a36ea..6bec42ac62d192 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -25,6 +25,7 @@ #include "vec/common/sort/topn_sorter.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); @@ -176,4 +177,5 @@ void SortSinkOperatorX::reset(RuntimeState* state) { auto& local_state = get_local_state(state); local_state._shared_state->sorter->reset(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index 6bf87164e71026..766c6c0ffc9a59 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -23,6 +23,7 @@ #include "vec/core/field.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" class SortSinkOperatorX; @@ -109,4 +110,5 @@ class SortSinkOperatorX final : public DataSinkOperatorX { const bool _reuse_mem; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index 7f801b79c0b12b..2fb09d7278fda8 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -22,6 +22,7 @@ #include "pipeline/exec/operator.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" SortLocalState::SortLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState(state, parent) {} @@ -79,4 +80,5 @@ Status SortSourceOperatorX::build_merger(RuntimeState* state, return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 20714eb44e5e60..a638b04b368eaa 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -23,6 +23,7 @@ #include "operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -69,4 +70,5 @@ class SortSourceOperatorX final : public OperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/spill_utils.h b/be/src/pipeline/exec/spill_utils.h index 925e7df44e607e..2ba6f22a60b10c 100644 --- a/be/src/pipeline/exec/spill_utils.h +++ b/be/src/pipeline/exec/spill_utils.h @@ -26,6 +26,7 @@ #include "vec/runtime/partitioner.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" using SpillPartitionerType = vectorized::Crc32HashPartitioner; class SpillRunnable : public Runnable { @@ -70,4 +71,5 @@ class SpillRunnable : public Runnable { std::function _func; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index 1c8d2c47bc698a..b6e5788a07c626 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -29,6 +29,7 @@ #include "vec/exprs/vslot_ref.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -228,7 +229,7 @@ Status StreamingAggLocalState::_merge_with_serialized_key_helper(vectorized::Blo } } - int rows = block->rows(); + size_t rows = block->rows(); if (_places.size() < rows) { _places.resize(rows); } @@ -270,7 +271,7 @@ Status StreamingAggLocalState::_merge_with_serialized_key_helper(vectorized::Blo for (int i = 0; i < _aggregate_evaluators.size(); ++i) { if (_aggregate_evaluators[i]->is_merge() || for_spill) { - int col_id = 0; + size_t col_id = 0; if constexpr (for_spill) { col_id = _probe_expr_ctxs.size() + i; } else { @@ -403,7 +404,7 @@ Status StreamingAggLocalState::_execute_with_serialized_key_helper(vectorized::B } } - int rows = block->rows(); + size_t rows = block->rows(); if (_places.size() < rows) { _places.resize(rows); } @@ -542,8 +543,8 @@ bool StreamingAggLocalState::_should_expand_preagg_hash_tables() { const int64_t aggregated_input_rows = input_rows - _cur_num_rows_returned; // TODO chenhao // const int64_t expected_input_rows = estimated_input_cardinality_ - num_rows_returned_; - double current_reduction = - static_cast(aggregated_input_rows) / ht_rows; + double current_reduction = static_cast(aggregated_input_rows) / + static_cast(ht_rows); // TODO: workaround for IMPALA-2490: subplan node rows_returned counter may be // inaccurate, which could lead to a divide by zero below. @@ -615,7 +616,7 @@ Status StreamingAggLocalState::_pre_agg_with_serialized_key(doris::vectorized::B } } - int rows = in_block->rows(); + size_t rows = in_block->rows(); _places.resize(rows); // Stop expanding hash tables if we're not reducing the input sufficiently. As our @@ -739,7 +740,7 @@ Status StreamingAggLocalState::_get_with_serialized_key_result(RuntimeState* sta auto columns_with_schema = vectorized::VectorizedUtils::create_columns_with_type_and_name(p._row_descriptor); - int key_size = _probe_expr_ctxs.size(); + size_t key_size = _probe_expr_ctxs.size(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -750,7 +751,7 @@ Status StreamingAggLocalState::_get_with_serialized_key_result(RuntimeState* sta } } vectorized::MutableColumns value_columns; - for (int i = key_size; i < columns_with_schema.size(); ++i) { + for (size_t i = key_size; i < columns_with_schema.size(); ++i) { if (!mem_reuse) { value_columns.emplace_back(columns_with_schema[i].type->create_column()); } else { @@ -852,7 +853,7 @@ Status StreamingAggLocalState::_get_results_without_key(RuntimeState* state, block->clear(); DCHECK(_agg_data->without_key != nullptr); - int agg_size = _aggregate_evaluators.size(); + const auto agg_size = _aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); std::vector data_types(agg_size); @@ -888,8 +889,8 @@ Status StreamingAggLocalState::_get_results_with_serialized_key(RuntimeState* st bool* eos) { SCOPED_TIMER(_get_results_timer); auto& p = _parent->cast(); - int key_size = _probe_expr_ctxs.size(); - int agg_size = _aggregate_evaluators.size(); + const auto key_size = _probe_expr_ctxs.size(); + const auto agg_size = _aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); vectorized::DataTypes value_data_types(agg_size); @@ -1013,7 +1014,7 @@ Status StreamingAggLocalState::_get_without_key_result(RuntimeState* state, auto& p = _parent->cast(); *block = vectorized::VectorizedUtils::create_empty_columnswithtypename(p._row_descriptor); - int agg_size = _aggregate_evaluators.size(); + const auto agg_size = _aggregate_evaluators.size(); vectorized::MutableColumns columns(agg_size); std::vector data_types(agg_size); @@ -1170,8 +1171,8 @@ Status StreamingAggOperatorX::open(RuntimeState* state) { DCHECK_EQ(_intermediate_tuple_desc->slots().size(), _output_tuple_desc->slots().size()); RETURN_IF_ERROR(vectorized::VExpr::prepare(_probe_expr_ctxs, state, _child->row_desc())); - int j = _probe_expr_ctxs.size(); - for (int i = 0; i < j; ++i) { + size_t j = _probe_expr_ctxs.size(); + for (size_t i = 0; i < j; ++i) { auto nullable_output = _output_tuple_desc->slots()[i]->is_nullable(); auto nullable_input = _probe_expr_ctxs[i]->root()->is_nullable(); if (nullable_output != nullable_input) { @@ -1179,7 +1180,7 @@ Status StreamingAggOperatorX::open(RuntimeState* state) { _make_nullable_keys.emplace_back(i); } } - for (int i = 0; i < _aggregate_evaluators.size(); ++i, ++j) { + for (size_t i = 0; i < _aggregate_evaluators.size(); ++i, ++j) { SlotDescriptor* intermediate_slot_desc = _intermediate_tuple_desc->slots()[j]; SlotDescriptor* output_slot_desc = _output_tuple_desc->slots()[j]; RETURN_IF_ERROR(_aggregate_evaluators[i]->prepare( @@ -1290,4 +1291,5 @@ bool StreamingAggOperatorX::need_more_input_data(RuntimeState* state) const { return local_state._pre_aggregated_block->empty() && !local_state._child_eos; } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.h b/be/src/pipeline/exec/streaming_aggregation_operator.h index b695880ac2857b..bd35cd940f2974 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_operator.h @@ -27,6 +27,7 @@ #include "vec/core/block.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -237,4 +238,5 @@ class StreamingAggOperatorX final : public StatefulOperatorXget_value( columns[p._child_slots.size() + p._fn_num - 1], - state->batch_size() - columns[p._child_slots.size()]->size()); + //// It has already been checked that + // columns[p._child_slots.size()]->size() < state->batch_size(), + // so columns[p._child_slots.size()]->size() will not exceed the range of int. + state->batch_size() - (int)columns[p._child_slots.size()]->size()); _current_row_insert_times += repeat_times; for (int i = 0; i < p._fn_num - 1; i++) { _fns[i]->get_same_many_values(columns[i + p._child_slots.size()], repeat_times); @@ -276,7 +280,7 @@ Status TableFunctionOperatorX::init(const TPlanNode& tnode, RuntimeState* state) fn->set_expr_context(ctx); _fns.push_back(fn); } - _fn_num = _fns.size(); + _fn_num = cast_set(_fns.size()); // Prepare output slot ids RETURN_IF_ERROR(_prepare_output_slot_ids(tnode)); @@ -304,7 +308,7 @@ Status TableFunctionOperatorX::open(doris::RuntimeState* state) { } } - for (size_t i = 0; i < _child_slots.size(); i++) { + for (int i = 0; i < _child_slots.size(); i++) { if (_slot_need_copy(i)) { _output_slot_indexs.push_back(i); } else { @@ -315,4 +319,5 @@ Status TableFunctionOperatorX::open(doris::RuntimeState* state) { return vectorized::VExpr::open(_vfn_ctxs, state); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 81160acb7f7611..9aa26e9ae22b10 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -24,6 +24,7 @@ #include "vec/exprs/table_function/table_function.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; } // namespace doris @@ -154,4 +155,5 @@ class TableFunctionOperatorX final : public StatefulOperatorX _child_slot_sizes; }; +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 8467eeb1d5467a..56491b5258bc55 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -19,6 +19,7 @@ #include +#include "common/cast_set.h" #include "common/compiler_util.h" // IWYU pragma: keep #include "common/status.h" #include "pipeline/exec/data_queue.h" @@ -27,6 +28,7 @@ #include "util/runtime_profile.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status UnionSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); @@ -54,7 +56,8 @@ Status UnionSinkLocalState::open(RuntimeState* state) { UnionSinkOperatorX::UnionSinkOperatorX(int child_id, int sink_id, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : Base(sink_id, tnode.node_id, tnode.node_id), - _first_materialized_child_idx(tnode.union_node.first_materialized_child_idx), + _first_materialized_child_idx( + cast_set(tnode.union_node.first_materialized_child_idx)), _row_descriptor(descs, tnode.row_tuples, tnode.nullable_tuples), _cur_child_id(child_id), _child_size(tnode.num_children) {} @@ -130,4 +133,5 @@ Status UnionSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block return Status::OK(); } +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index aa94ed9a73038f..3a8880622cb108 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -26,6 +26,7 @@ #include "vec/core/block.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -152,4 +153,5 @@ class UnionSinkOperatorX final : public DataSinkOperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index ecaaf22922b657..d13658488e2c9b 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -30,6 +30,7 @@ #include "vec/core/block.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace pipeline { @@ -148,7 +149,7 @@ Status UnionSourceOperatorX::get_next_const(RuntimeState* state, vectorized::Blo vectorized::Block tmp_block; tmp_block.insert({vectorized::ColumnUInt8::create(1), std::make_shared(), ""}); - int const_expr_lists_size = _const_expr_lists[_const_expr_list_idx].size(); + int const_expr_lists_size = cast_set(_const_expr_lists[_const_expr_list_idx].size()); if (_const_expr_list_idx && const_expr_lists_size != _const_expr_lists[0].size()) { return Status::InternalError( "[UnionNode]const expr at {}'s count({}) not matched({} expected)", @@ -183,4 +184,5 @@ Status UnionSourceOperatorX::get_next_const(RuntimeState* state, vectorized::Blo } } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 200e7de8597b91..0ee66c3da7447b 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -24,6 +24,7 @@ #include "operator.h" namespace doris { +#include "common/compile_check_begin.h" class RuntimeState; namespace vectorized { @@ -123,4 +124,5 @@ class UnionSourceOperatorX final : public OperatorX { }; } // namespace pipeline +#include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/pipeline.cpp b/be/src/pipeline/pipeline.cpp index e4678b7dcf3a83..6c39d361e59c77 100644 --- a/be/src/pipeline/pipeline.cpp +++ b/be/src/pipeline/pipeline.cpp @@ -112,7 +112,7 @@ void Pipeline::make_all_runnable() { if (_sink->count_down_destination()) { for (auto* task : _tasks) { if (task) { - task->set_wake_up_by_downstream(); + task->set_wake_up_early(); } } for (auto* task : _tasks) { diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 8ab0f1d151568d..5ae89db55a45ac 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -35,6 +35,7 @@ #include "cloud/config.h" #include "common/cast_set.h" #include "common/config.h" +#include "common/exception.h" #include "common/logging.h" #include "common/status.h" #include "io/fs/stream_load_pipe.h" @@ -498,8 +499,8 @@ Status PipelineFragmentContext::_build_pipeline_tasks(const doris::TPipelineFrag if (pipeline_id_to_task.contains(_pipelines[pip_idx]->id())) { auto* task = pipeline_id_to_task[_pipelines[pip_idx]->id()]; DCHECK(pipeline_id_to_profile[pip_idx]); - RETURN_IF_ERROR(task->prepare(local_params, request.fragment.output_sink, - _query_ctx.get())); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(task->prepare( + local_params, request.fragment.output_sink, _query_ctx.get())); } } { diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index 6814881ac7a300..5ed725010ec364 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -223,9 +224,6 @@ bool PipelineTask::_wait_to_start() { _blocked_dep = _execution_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { static_cast(_blocked_dep)->start_watcher(); - if (_wake_up_by_downstream) { - _eos = true; - } return true; } @@ -233,9 +231,6 @@ bool PipelineTask::_wait_to_start() { _blocked_dep = op_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { _blocked_dep->start_watcher(); - if (_wake_up_by_downstream) { - _eos = true; - } return true; } } @@ -257,9 +252,6 @@ bool PipelineTask::_is_blocked() { _blocked_dep = dep->is_blocked_by(this); if (_blocked_dep != nullptr) { _blocked_dep->start_watcher(); - if (_wake_up_by_downstream) { - _eos = true; - } return true; } } @@ -279,9 +271,6 @@ bool PipelineTask::_is_blocked() { _blocked_dep = op_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { _blocked_dep->start_watcher(); - if (_wake_up_by_downstream) { - _eos = true; - } return true; } } @@ -289,15 +278,15 @@ bool PipelineTask::_is_blocked() { } Status PipelineTask::execute(bool* eos) { - SCOPED_TIMER(_task_profile->total_time_counter()); - SCOPED_TIMER(_exec_timer); - SCOPED_ATTACH_TASK(_state); - _eos = _sink->is_finished(_state) || _eos || _wake_up_by_downstream; - *eos = _eos; if (_eos) { - // If task is waken up by finish dependency, `_eos` is set to true by last execution, and we should return here. + *eos = true; return Status::OK(); } + + SCOPED_TIMER(_task_profile->total_time_counter()); + SCOPED_TIMER(_exec_timer); + SCOPED_ATTACH_TASK(_state); + int64_t time_spent = 0; DBUG_EXECUTE_IF("fault_inject::PipelineXTask::execute", { Status status = Status::Error("fault_inject pipeline_task execute failed"); @@ -320,27 +309,31 @@ Status PipelineTask::execute(bool* eos) { if (_wait_to_start()) { return Status::OK(); } - if (_wake_up_by_downstream) { - _eos = true; - *eos = true; - return Status::OK(); - } + // The status must be runnable if (!_opened && !_fragment_context->is_canceled()) { + if (_wake_up_early) { + *eos = true; + _eos = true; + return Status::OK(); + } RETURN_IF_ERROR(_open()); } + auto set_wake_up_and_dep_ready = [&]() { + if (wake_up_early()) { + return; + } + set_wake_up_early(); + clear_blocking_state(); + }; + _task_profile->add_info_string("TaskState", "Runnable"); _task_profile->add_info_string("BlockedByDependency", ""); while (!_fragment_context->is_canceled()) { if (_is_blocked()) { return Status::OK(); } - if (_wake_up_by_downstream) { - _eos = true; - *eos = true; - return Status::OK(); - } /// When a task is cancelled, /// its blocking state will be cleared and it will transition to a ready state (though it is not truly ready). @@ -361,47 +354,47 @@ Status PipelineTask::execute(bool* eos) { RETURN_IF_ERROR(_sink->revoke_memory(_state)); continue; } - *eos = _eos; DBUG_EXECUTE_IF("fault_inject::PipelineXTask::executing", { Status status = Status::Error("fault_inject pipeline_task executing failed"); return status; }); - // `_dry_run` means sink operator need no more data // `_sink->is_finished(_state)` means sink operator should be finished - if (_dry_run || _sink->is_finished(_state)) { - *eos = true; - _eos = true; - } else { + if (_sink->is_finished(_state)) { + set_wake_up_and_dep_ready(); + } + + // `_dry_run` means sink operator need no more data + *eos = wake_up_early() || _dry_run; + if (!*eos) { SCOPED_TIMER(_get_block_timer); _get_block_counter->update(1); RETURN_IF_ERROR(_root->get_block_after_projects(_state, block, eos)); } + if (*eos) { + RETURN_IF_ERROR(close(Status::OK(), false)); + } + if (_block->rows() != 0 || *eos) { SCOPED_TIMER(_sink_timer); - Status status = Status::OK(); - // Define a lambda function to catch sink exception, because sink will check - // return error status with EOF, it is special, could not return directly. - auto sink_function = [&]() -> Status { - Status internal_st; - internal_st = _sink->sink(_state, block, *eos); - return internal_st; - }; - status = sink_function(); - if (!status.is()) { - RETURN_IF_ERROR(status); + Status status = _sink->sink(_state, block, *eos); + + if (status.is()) { + set_wake_up_and_dep_ready(); + } else if (!status) { + return status; } - *eos = status.is() ? true : *eos; + if (*eos) { // just return, the scheduler will do finish work - _eos = true; _task_profile->add_info_string("TaskState", "Finished"); + _eos = true; return Status::OK(); } } } - static_cast(get_task_queue()->push_back(this)); + RETURN_IF_ERROR(get_task_queue()->push_back(this)); return Status::OK(); } @@ -470,17 +463,14 @@ void PipelineTask::finalize() { _le_state_map.clear(); } -Status PipelineTask::close(Status exec_status) { +Status PipelineTask::close(Status exec_status, bool close_sink) { int64_t close_ns = 0; - Defer defer {[&]() { - if (_task_queue) { - _task_queue->update_statistics(this, close_ns); - } - }}; Status s; { SCOPED_RAW_TIMER(&close_ns); - s = _sink->close(_state, exec_status); + if (close_sink) { + s = _sink->close(_state, exec_status); + } for (auto& op : _operators) { auto tem = op->close(_state); if (!tem.ok() && s.ok()) { @@ -489,10 +479,18 @@ Status PipelineTask::close(Status exec_status) { } } if (_opened) { - _fresh_profile_counter(); - COUNTER_SET(_close_timer, close_ns); + COUNTER_UPDATE(_close_timer, close_ns); COUNTER_UPDATE(_task_profile->total_time_counter(), close_ns); } + + if (close_sink && _opened) { + _task_profile->add_info_string("WakeUpEarly", wake_up_early() ? "true" : "false"); + _fresh_profile_counter(); + } + + if (_task_queue) { + _task_queue->update_statistics(this, close_ns); + } return s; } @@ -508,10 +506,10 @@ std::string PipelineTask::debug_string() { auto elapsed = _fragment_context->elapsed_time() / 1000000000.0; fmt::format_to(debug_string_buffer, "PipelineTask[this = {}, id = {}, open = {}, eos = {}, finish = {}, dry run = " - "{}, elapse time = {}s, _wake_up_by_downstream = {}], block dependency = {}, is " + "{}, elapse time = {}s, _wake_up_early = {}], block dependency = {}, is " "running = {}\noperators: ", (void*)this, _index, _opened, _eos, _finalized, _dry_run, elapsed, - _wake_up_by_downstream.load(), + _wake_up_early.load(), cur_blocked_dep && !_finalized ? cur_blocked_dep->debug_string() : "NULL", is_running()); for (size_t i = 0; i < _operators.size(); i++) { diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 4bb062122c0c08..1a31e5954f479c 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -61,7 +61,7 @@ class PipelineTask { // if the pipeline create a bunch of pipeline task // must be call after all pipeline task is finish to release resource - Status close(Status exec_status); + Status close(Status exec_status, bool close_sink = true); PipelineFragmentContext* fragment_context() { return _fragment_context; } @@ -135,7 +135,7 @@ class PipelineTask { int task_id() const { return _index; }; bool is_finalized() const { return _finalized; } - void set_wake_up_by_downstream() { _wake_up_by_downstream = true; } + void set_wake_up_early() { _wake_up_early = true; } void clear_blocking_state() { _state->get_query_ctx()->get_execution_dependency()->set_always_ready(); @@ -237,7 +237,7 @@ class PipelineTask { PipelineId pipeline_id() const { return _pipeline->id(); } - bool wake_up_by_downstream() const { return _wake_up_by_downstream; } + bool wake_up_early() const { return _wake_up_early; } private: friend class RuntimeFilterDependency; @@ -319,7 +319,7 @@ class PipelineTask { std::atomic _running = false; std::atomic _eos = false; - std::atomic _wake_up_by_downstream = false; + std::atomic _wake_up_early = false; }; } // namespace doris::pipeline diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp index c16db7c67d3420..b4a38173d72222 100644 --- a/be/src/runtime/runtime_filter_mgr.cpp +++ b/be/src/runtime/runtime_filter_mgr.cpp @@ -219,35 +219,14 @@ Status RuntimeFilterMgr::get_merge_addr(TNetworkAddress* addr) { Status RuntimeFilterMergeControllerEntity::_init_with_desc( const TRuntimeFilterDesc* runtime_filter_desc, const TQueryOptions* query_options, - const std::vector* target_info, - const int producer_size) { - std::unique_lock guard(_filter_map_mutex); - std::shared_ptr cnt_val = std::make_shared(); - // runtime_filter_desc and target will be released, - // so we need to copy to cnt_val - cnt_val->producer_size = producer_size; - cnt_val->runtime_filter_desc = *runtime_filter_desc; - cnt_val->pool.reset(new ObjectPool()); - cnt_val->filter = cnt_val->pool->add(new IRuntimeFilter(_state, runtime_filter_desc)); - - auto filter_id = runtime_filter_desc->filter_id; - RETURN_IF_ERROR( - cnt_val->filter->init_with_desc(&cnt_val->runtime_filter_desc, query_options, -1)); - cnt_val->filter->set_ignored(); - _filter_map.emplace(filter_id, cnt_val); - return Status::OK(); -} - -Status RuntimeFilterMergeControllerEntity::_init_with_desc( - const TRuntimeFilterDesc* runtime_filter_desc, const TQueryOptions* query_options, - const std::vector* targetv2_info, + const std::vector&& targetv2_info, const int producer_size) { std::shared_ptr cnt_val = std::make_shared(); // runtime_filter_desc and target will be released, // so we need to copy to cnt_val cnt_val->producer_size = producer_size; cnt_val->runtime_filter_desc = *runtime_filter_desc; - cnt_val->targetv2_info = *targetv2_info; + cnt_val->targetv2_info = targetv2_info; cnt_val->pool.reset(new ObjectPool()); cnt_val->filter = cnt_val->pool->add(new IRuntimeFilter(_state, runtime_filter_desc)); auto filter_id = runtime_filter_desc->filter_id; @@ -268,36 +247,21 @@ Status RuntimeFilterMergeControllerEntity::init(UniqueId query_id, if (runtime_filter_params.__isset.rid_to_runtime_filter) { for (const auto& filterid_to_desc : runtime_filter_params.rid_to_runtime_filter) { int filter_id = filterid_to_desc.first; - const auto& target_iter = runtime_filter_params.rid_to_target_param.find(filter_id); - if (target_iter == runtime_filter_params.rid_to_target_param.end() && - !runtime_filter_params.__isset.rid_to_target_paramv2) { - // This runtime filter has to target info - return Status::InternalError("runtime filter params meet error"); - } else if (target_iter == runtime_filter_params.rid_to_target_param.end()) { - const auto& targetv2_iter = - runtime_filter_params.rid_to_target_paramv2.find(filter_id); - if (targetv2_iter == runtime_filter_params.rid_to_target_paramv2.end()) { - // This runtime filter has to target info - return Status::InternalError("runtime filter params meet error"); - } - const auto& build_iter = - runtime_filter_params.runtime_filter_builder_num.find(filter_id); - if (build_iter == runtime_filter_params.runtime_filter_builder_num.end()) { - // This runtime filter has to builder info - return Status::InternalError("runtime filter params meet error"); - } - - RETURN_IF_ERROR(_init_with_desc(&filterid_to_desc.second, &query_options, - &targetv2_iter->second, build_iter->second)); - } else { - const auto& build_iter = - runtime_filter_params.runtime_filter_builder_num.find(filter_id); - if (build_iter == runtime_filter_params.runtime_filter_builder_num.end()) { - return Status::InternalError("runtime filter params meet error"); - } - RETURN_IF_ERROR(_init_with_desc(&filterid_to_desc.second, &query_options, - &target_iter->second, build_iter->second)); + const auto& targetv2_iter = runtime_filter_params.rid_to_target_paramv2.find(filter_id); + const auto& build_iter = + runtime_filter_params.runtime_filter_builder_num.find(filter_id); + if (build_iter == runtime_filter_params.runtime_filter_builder_num.end()) { + // This runtime filter has no builder info + return Status::InternalError( + "Runtime filter has a wrong parameter. Maybe FE version is mismatched."); } + + RETURN_IF_ERROR(_init_with_desc( + &filterid_to_desc.second, &query_options, + targetv2_iter == runtime_filter_params.rid_to_target_paramv2.end() + ? std::vector {} + : std::move(targetv2_iter->second), + build_iter->second)); } } return Status::OK(); diff --git a/be/src/runtime/runtime_filter_mgr.h b/be/src/runtime/runtime_filter_mgr.h index 9f4cf5f4e22a07..c54be905f28f08 100644 --- a/be/src/runtime/runtime_filter_mgr.h +++ b/be/src/runtime/runtime_filter_mgr.h @@ -192,12 +192,7 @@ class RuntimeFilterMergeControllerEntity { private: Status _init_with_desc(const TRuntimeFilterDesc* runtime_filter_desc, const TQueryOptions* query_options, - const std::vector* target_info, - const int producer_size); - - Status _init_with_desc(const TRuntimeFilterDesc* runtime_filter_desc, - const TQueryOptions* query_options, - const std::vector* target_info, + const std::vector&& target_info, const int producer_size); UniqueId _query_id; diff --git a/be/src/util/cgroup_util.cpp b/be/src/util/cgroup_util.cpp index 8f64fe699c6062..fc35be3dc35931 100644 --- a/be/src/util/cgroup_util.cpp +++ b/be/src/util/cgroup_util.cpp @@ -218,6 +218,10 @@ std::optional CGroupUtil::get_cgroupsv2_path(const std::string& sub Status CGroupUtil::read_int_line_from_cgroup_file(const std::filesystem::path& file_path, int64_t* val) { std::ifstream file_stream(file_path, std::ios::in); + if (!file_stream.is_open()) { + return Status::CgroupError("Error open {}", file_path.string()); + } + string line; getline(file_stream, line); if (file_stream.fail() || file_stream.bad()) { @@ -264,4 +268,167 @@ void CGroupUtil::read_int_metric_from_cgroup_file( } } +Status CGroupUtil::read_string_line_from_cgroup_file(const std::filesystem::path& file_path, + std::string* line_ptr) { + std::ifstream file_stream(file_path, std::ios::in); + if (!file_stream.is_open()) { + return Status::CgroupError("Error open {}", file_path.string()); + } + string line; + getline(file_stream, line); + if (file_stream.fail() || file_stream.bad()) { + return Status::CgroupError("Error reading {}: {}", file_path.string(), get_str_err_msg()); + } + *line_ptr = line; + return Status::OK(); +} + +Status CGroupUtil::parse_cpuset_line(std::string cpuset_line, int* cpu_count_ptr) { + if (cpuset_line.empty()) { + return Status::CgroupError("cpuset line is empty"); + } + std::vector ranges; + boost::split(ranges, cpuset_line, boost::is_any_of(",")); + int cpu_count = 0; + + for (const std::string& range : ranges) { + std::vector cpu_values; + boost::split(cpu_values, range, boost::is_any_of("-")); + + if (cpu_values.size() == 2) { + int start = std::stoi(cpu_values[0]); + int end = std::stoi(cpu_values[1]); + cpu_count += (end - start) + 1; + } else { + cpu_count++; + } + } + *cpu_count_ptr = cpu_count; + return Status::OK(); +} + +int CGroupUtil::get_cgroup_limited_cpu_number(int physical_cores) { + if (physical_cores <= 0) { + return physical_cores; + } + int ret = physical_cores; +#if defined(OS_LINUX) + // For cgroup v2 + // Child cgroup's cpu.max may bigger than parent group's cpu.max, + // so it should look up from current cgroup to top group. + // For cpuset, child cgroup's cpuset.cpus could not bigger thant parent's cpuset.cpus. + if (CGroupUtil::cgroupsv2_enable()) { + std::string cgroupv2_process_path = CGroupUtil::cgroupv2_of_process(); + if (cgroupv2_process_path.empty()) { + return ret; + } + std::filesystem::path current_cgroup_path = (default_cgroups_mount / cgroupv2_process_path); + ret = get_cgroup_v2_cpu_quota_number(current_cgroup_path, default_cgroups_mount, ret); + + current_cgroup_path = (default_cgroups_mount / cgroupv2_process_path); + ret = get_cgroup_v2_cpuset_number(current_cgroup_path, default_cgroups_mount, ret); + } else if (CGroupUtil::cgroupsv1_enable()) { + // cpu quota, should find first not empty config from current path to top. + // because if a process attach to current cgroup, its cpu quota may not be set. + std::string cpu_quota_path = ""; + Status cpu_quota_ret = CGroupUtil::find_abs_cgroupv1_path("cpu", &cpu_quota_path); + if (cpu_quota_ret.ok() && !cpu_quota_path.empty()) { + std::filesystem::path current_cgroup_path = cpu_quota_path; + ret = get_cgroup_v1_cpu_quota_number(current_cgroup_path, default_cgroups_mount, ret); + } + + //cpuset + // just lookup current process cgroup path is enough + // because if a process attach to current cgroup, its cpuset.cpus must be set. + std::string cpuset_path = ""; + Status cpuset_ret = CGroupUtil::find_abs_cgroupv1_path("cpuset", &cpuset_path); + if (cpuset_ret.ok() && !cpuset_path.empty()) { + std::filesystem::path current_path = cpuset_path; + ret = get_cgroup_v1_cpuset_number(current_path, ret); + } + } +#endif + return ret; +} + +int CGroupUtil::get_cgroup_v2_cpu_quota_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num) { + int ret = cpu_num; + while (current_path != default_cg_mout_path.parent_path()) { + std::ifstream cpu_max_file(current_path / "cpu.max"); + if (cpu_max_file.is_open()) { + std::string cpu_limit_str; + double cpu_period; + cpu_max_file >> cpu_limit_str >> cpu_period; + if (cpu_limit_str != "max" && cpu_period != 0) { + double cpu_limit = std::stod(cpu_limit_str); + ret = std::min(static_cast(std::ceil(cpu_limit / cpu_period)), ret); + } + } + current_path = current_path.parent_path(); + } + return ret; +} + +int CGroupUtil::get_cgroup_v2_cpuset_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num) { + int ret = cpu_num; + while (current_path != default_cg_mout_path.parent_path()) { + std::ifstream cpuset_cpus_file(current_path / "cpuset.cpus.effective"); + current_path = current_path.parent_path(); + if (cpuset_cpus_file.is_open()) { + std::string cpuset_line; + cpuset_cpus_file >> cpuset_line; + if (cpuset_line.empty()) { + continue; + } + int cpus_count = 0; + static_cast(CGroupUtil::parse_cpuset_line(cpuset_line, &cpus_count)); + ret = std::min(cpus_count, ret); + break; + } + } + return ret; +} + +int CGroupUtil::get_cgroup_v1_cpu_quota_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num) { + int ret = cpu_num; + while (current_path != default_cg_mout_path.parent_path()) { + std::ifstream cpu_quota_file(current_path / "cpu.cfs_quota_us"); + std::ifstream cpu_period_file(current_path / "cpu.cfs_period_us"); + if (cpu_quota_file.is_open() && cpu_period_file.is_open()) { + double cpu_quota_value; + double cpu_period_value; + cpu_quota_file >> cpu_quota_value; + cpu_period_file >> cpu_period_value; + if (cpu_quota_value > 0 && cpu_period_value > 0) { + ret = std::min(ret, + static_cast(std::ceil(cpu_quota_value / cpu_period_value))); + break; + } + } + current_path = current_path.parent_path(); + } + return ret; +} + +int CGroupUtil::get_cgroup_v1_cpuset_number(std::filesystem::path& current_path, int cpu_num) { + int ret = cpu_num; + std::string cpuset_line = ""; + Status cpuset_ret = CGroupUtil::read_string_line_from_cgroup_file( + (current_path / "cpuset.cpus"), &cpuset_line); + if (cpuset_ret.ok() && !cpuset_line.empty()) { + int cpuset_count = 0; + static_cast(CGroupUtil::parse_cpuset_line(cpuset_line, &cpuset_count)); + if (cpuset_count > 0) { + ret = std::min(ret, cpuset_count); + } + } + return ret; +} + } // namespace doris diff --git a/be/src/util/cgroup_util.h b/be/src/util/cgroup_util.h index bc1417453f41f6..54fc9494599f15 100644 --- a/be/src/util/cgroup_util.h +++ b/be/src/util/cgroup_util.h @@ -104,5 +104,27 @@ class CGroupUtil { static void read_int_metric_from_cgroup_file( const std::filesystem::path& file_path, std::unordered_map& metrics_map); + + static Status read_string_line_from_cgroup_file(const std::filesystem::path& file_path, + std::string* line_ptr); + + // cpuset_line: 0-4,6,8-10 + static Status parse_cpuset_line(std::string cpuset_line, int* cpu_count_ptr); + + static int get_cgroup_limited_cpu_number(int physical_cores); + + static int get_cgroup_v2_cpu_quota_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num); + + static int get_cgroup_v2_cpuset_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num); + + static int get_cgroup_v1_cpu_quota_number(std::filesystem::path& current_path, + const std::filesystem::path& default_cg_mout_path, + int cpu_num); + + static int get_cgroup_v1_cpuset_number(std::filesystem::path& current_path, int cpu_num); }; } // namespace doris diff --git a/be/src/util/cpu_info.cpp b/be/src/util/cpu_info.cpp index 116dacb8da7ed4..b49985cdc06830 100644 --- a/be/src/util/cpu_info.cpp +++ b/be/src/util/cpu_info.cpp @@ -59,6 +59,7 @@ #include "gflags/gflags.h" #include "gutil/stringprintf.h" #include "gutil/strings/substitute.h" +#include "util/cgroup_util.h" #include "util/pretty_printer.h" using boost::algorithm::contains; @@ -109,58 +110,6 @@ static struct { {"popcnt", CpuInfo::POPCNT}, {"avx", CpuInfo::AVX}, {"avx2", CpuInfo::AVX2}, }; -int cgroup_bandwidth_quota(int physical_cores) { - namespace fs = std::filesystem; - fs::path cpu_max = "/sys/fs/cgroup/cpu.max"; - fs::path cfs_quota = "/sys/fs/cgroup/cpu/cpu.cfs_quota_us"; - fs::path cfs_period = "/sys/fs/cgroup/cpu/cpu.cfs_period_us"; - - int64_t quota, period; - char byte_buffer[1000]; - int64_t read_bytes; - - if (fs::exists(cpu_max)) { - // cgroup v2 - // https://www.kernel.org/doc/html/latest/admin-guide/cgroup-v2.html - std::ifstream file(cpu_max); - file.read(byte_buffer, 999); - read_bytes = file.gcount(); - byte_buffer[read_bytes] = '\0'; - if (sscanf(byte_buffer, "%" SCNd64 " %" SCNd64 "", "a, &period) != 2) { - return physical_cores; - } - } else if (fs::exists(cfs_quota) && fs::exists(cfs_period)) { - // cgroup v1 - // https://www.kernel.org/doc/html/latest/scheduler/sched-bwc.html#management - - // Read the quota, this indicates how many microseconds the CPU can be utilized by this cgroup per period - std::ifstream quota_file(cfs_quota); - quota_file.read(byte_buffer, 999); - read_bytes = quota_file.gcount(); - byte_buffer[read_bytes] = '\0'; - if (sscanf(byte_buffer, "%" SCNd64 "", "a) != 1) { - return physical_cores; - } - - // Read the time period, a cgroup can utilize the CPU up to quota microseconds every period - std::ifstream period_file(cfs_period); - period_file.read(byte_buffer, 999); - read_bytes = period_file.gcount(); - byte_buffer[read_bytes] = '\0'; - if (sscanf(byte_buffer, "%" SCNd64 "", &period) != 1) { - return physical_cores; - } - } else { - // No cgroup quota - return physical_cores; - } - if (quota > 0 && period > 0) { - return int64_t(ceil(double(quota) / double(period))); - } else { - return physical_cores; - } -} - // Helper function to parse for hardware flags. // values contains a list of space-separated flags. check to see if the flags we // care about are present. @@ -212,7 +161,7 @@ void CpuInfo::init() { } } - int num_cores = cgroup_bandwidth_quota(physical_num_cores); + int num_cores = CGroupUtil::get_cgroup_limited_cpu_number(physical_num_cores); if (max_mhz != 0) { cycles_per_ms_ = int64_t(max_mhz) * 1000; } else { diff --git a/be/src/util/debug_util.cpp b/be/src/util/debug_util.cpp index 1cf03d2c22d0e1..0856b10c051709 100644 --- a/be/src/util/debug_util.cpp +++ b/be/src/util/debug_util.cpp @@ -17,6 +17,7 @@ #include "util/debug_util.h" +#include #include #include #include @@ -104,6 +105,16 @@ std::string hexdump(const char* buf, int len) { return ss.str(); } +bvar::Status be_version_metrics("doris_be_version", [] { + std::stringstream ss; + ss << version::doris_build_version_major() << 0 << version::doris_build_version_minor() << 0 + << version::doris_build_version_patch(); + if (version::doris_build_version_hotfix() > 0) { + ss << 0 << version::doris_build_version_hotfix(); + } + return std::strtoul(ss.str().c_str(), nullptr, 10); +}()); + std::string PrintThriftNetworkAddress(const TNetworkAddress& add) { std::stringstream ss; add.printTo(ss); diff --git a/be/src/util/mysql_row_buffer.cpp b/be/src/util/mysql_row_buffer.cpp index 3e20a2d9de72fe..4fd7de13753a95 100644 --- a/be/src/util/mysql_row_buffer.cpp +++ b/be/src/util/mysql_row_buffer.cpp @@ -107,7 +107,11 @@ MysqlRowBuffer::~MysqlRowBuffer() { template void MysqlRowBuffer::open_dynamic_mode() { if (!_dynamic_mode) { - *_pos++ = NEXT_EIGHT_BYTE; + // if _pos now exactly at the end of _buf memory, + // we should reserve 1 byte for _dynamic_mode flag byte to avoid *pos = 254 + // cause _dynamic_mode flag byte be overwritten + reserve(1); + *_pos++ = NEXT_EIGHT_BYTE; // *_pos = 254 ; _pos++ // write length when dynamic mode close _len_pos = (_pos - _buf); _pos = _pos + 8; diff --git a/be/src/util/simd/vstring_function.h b/be/src/util/simd/vstring_function.h index 99313132382e5c..bfa75b728d5620 100644 --- a/be/src/util/simd/vstring_function.h +++ b/be/src/util/simd/vstring_function.h @@ -309,8 +309,11 @@ class VStringFunctions { // is to say, counting bytes which do not match 10xx_xxxx pattern. // All 0xxx_xxxx, 110x_xxxx, 1110_xxxx and 1111_0xxx are greater than 1011_1111 when use int8_t arithmetic, // so just count bytes greater than 1011_1111 in a byte string as the result of utf8_length. - static inline size_t get_char_len(const char* src, size_t len) { - size_t char_len = 0; + // get_char_len is used to return the UTF-8 length of a string. + // The return value will never exceed len. + template + static inline T get_char_len(const char* src, T len) { + T char_len = 0; const char* p = src; const char* end = p + len; #if defined(__SSE2__) || defined(__aarch64__) diff --git a/be/src/util/system_metrics.cpp b/be/src/util/system_metrics.cpp index 973f461d8defe7..fc2cdcc9262b31 100644 --- a/be/src/util/system_metrics.cpp +++ b/be/src/util/system_metrics.cpp @@ -33,18 +33,23 @@ #include "gutil/strings/split.h" // for string split #include "gutil/strtoint.h" // for atoi64 +#include "util/cgroup_util.h" #include "util/mem_info.h" #include "util/perf_counters.h" namespace doris { +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(avail_cpu_num, MetricUnit::NOUNIT); + DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(host_cpu_num, MetricUnit::NOUNIT); struct CpuNumberMetrics { CpuNumberMetrics(MetricEntity* ent) : entity(ent) { INT_ATOMIC_COUNTER_METRIC_REGISTER(entity, host_cpu_num); + INT_ATOMIC_COUNTER_METRIC_REGISTER(entity, avail_cpu_num); } IntAtomicCounter* host_cpu_num {nullptr}; + IntAtomicCounter* avail_cpu_num {nullptr}; MetricEntity* entity = nullptr; }; @@ -1004,6 +1009,14 @@ void SystemMetrics::_update_proc_metrics() { fclose(fp); } +void SystemMetrics::update_be_avail_cpu_num() { + int64_t physical_cpu_num = _cpu_num_metrics->host_cpu_num->value(); + if (physical_cpu_num > 0) { + physical_cpu_num = CGroupUtil::get_cgroup_limited_cpu_number(physical_cpu_num); + _cpu_num_metrics->avail_cpu_num->set_value(physical_cpu_num); + } +} + void SystemMetrics::get_metrics_from_proc_vmstat() { #ifdef BE_TEST FILE* fp = fopen(k_ut_vmstat_path, "r"); diff --git a/be/src/util/system_metrics.h b/be/src/util/system_metrics.h index 29ce8c9c02b359..2c5446b81f4f71 100644 --- a/be/src/util/system_metrics.h +++ b/be/src/util/system_metrics.h @@ -66,6 +66,8 @@ class SystemMetrics { void update_max_network_receive_bytes_rate(int64_t max_receive_bytes_rate); void update_allocator_metrics(); + void update_be_avail_cpu_num(); + private: void _install_cpu_metrics(); // On Intel(R) Xeon(R) CPU E5-2450 0 @ 2.10GHz; diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h index e0ec2bef62fc2a..d761d40c4c932c 100644 --- a/be/src/vec/aggregate_functions/aggregate_function.h +++ b/be/src/vec/aggregate_functions/aggregate_function.h @@ -20,6 +20,8 @@ #pragma once +#include + #include "common/exception.h" #include "common/status.h" #include "util/defer_op.h" @@ -81,7 +83,7 @@ using ConstAggregateDataPtr = const char*; */ class IAggregateFunction { public: - IAggregateFunction(const DataTypes& argument_types_) : argument_types(argument_types_) {} + IAggregateFunction(DataTypes argument_types_) : argument_types(std::move(argument_types_)) {} /// Get main function name. virtual String get_name() const = 0; @@ -225,7 +227,7 @@ class IAggregateFunction { virtual void set_version(const int version_) { version = version_; } - virtual AggregateFunctionPtr transmit_to_stable() { return nullptr; } + virtual IAggregateFunction* transmit_to_stable() { return nullptr; } /// Verify function signature virtual Status verify_result_type(const bool without_key, const DataTypes& argument_types, diff --git a/be/src/vec/aggregate_functions/aggregate_function_distinct.h b/be/src/vec/aggregate_functions/aggregate_function_distinct.h index 46450394627474..a5515145d9d2ad 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_distinct.h +++ b/be/src/vec/aggregate_functions/aggregate_function_distinct.h @@ -341,12 +341,22 @@ class AggregateFunctionDistinct DataTypePtr get_return_type() const override { return nested_func->get_return_type(); } - AggregateFunctionPtr transmit_to_stable() override { - return AggregateFunctionPtr(new AggregateFunctionDistinct( - nested_func, IAggregateFunction::argument_types)); + IAggregateFunction* transmit_to_stable() override { + return new AggregateFunctionDistinct(nested_func, + IAggregateFunction::argument_types); } }; +template +struct FunctionStableTransfer { + using FunctionStable = T; +}; + +template