-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[opt](pipeline) Make pipeline fragment context send_report asynchronized #23142
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
3e4cd1a
793d246
b779092
bc00bc8
f90b900
0bf7de3
4b0b246
3839fc6
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -28,6 +28,9 @@ | |||||
| #include <pthread.h> | ||||||
| #include <stdlib.h> | ||||||
| // IWYU pragma: no_include <bits/chrono.h> | ||||||
| #include <fmt/format.h> | ||||||
| #include <fmt/ranges.h> | ||||||
|
|
||||||
| #include <chrono> // IWYU pragma: keep | ||||||
| #include <map> | ||||||
| #include <ostream> | ||||||
|
|
@@ -125,14 +128,12 @@ PipelineFragmentContext::PipelineFragmentContext( | |||||
| _exec_env(exec_env), | ||||||
| _query_ctx(std::move(query_ctx)), | ||||||
| _call_back(call_back), | ||||||
| _report_thread_active(false), | ||||||
| _report_status_cb(report_status_cb), | ||||||
| _is_report_on_cancel(true), | ||||||
| _report_status_cb(report_status_cb), | ||||||
| _group_commit(group_commit) { | ||||||
| if (_query_ctx->get_task_group()) { | ||||||
| _task_group_entity = _query_ctx->get_task_group()->task_entity(); | ||||||
| } | ||||||
| _report_thread_future = _report_thread_promise.get_future(); | ||||||
| _fragment_watcher.start(); | ||||||
| } | ||||||
|
|
||||||
|
|
@@ -146,16 +147,29 @@ PipelineFragmentContext::~PipelineFragmentContext() { | |||||
| } else { | ||||||
| _call_back(_runtime_state.get(), &st); | ||||||
| } | ||||||
| DCHECK(!_report_thread_active); | ||||||
| } | ||||||
|
|
||||||
| void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, | ||||||
| const std::string& msg) { | ||||||
| LOG_INFO("PipelineFragmentContext::cancel") | ||||||
| .tag("query_id", print_id(_query_ctx->query_id())) | ||||||
| .tag("fragment_id", _fragment_id) | ||||||
| .tag("instance_id", print_id(_runtime_state->fragment_instance_id())) | ||||||
| .tag("reason", PPlanFragmentCancelReason_Name(reason)) | ||||||
| .tag("message", msg); | ||||||
|
|
||||||
| if (_query_ctx->cancel(true, msg, Status::Cancelled(msg))) { | ||||||
| LOG(WARNING) << "PipelineFragmentContext " | ||||||
| << PrintInstanceStandardInfo(_query_id, _fragment_id, _fragment_instance_id) | ||||||
| << " is canceled, cancel message: " << msg; | ||||||
| if (reason != PPlanFragmentCancelReason::LIMIT_REACH) { | ||||||
| LOG(WARNING) << "PipelineFragmentContext " | ||||||
| << PrintInstanceStandardInfo(_query_id, _fragment_id, | ||||||
| _fragment_instance_id) | ||||||
| << " is canceled, cancel message: " << msg; | ||||||
|
|
||||||
| } else { | ||||||
| _set_is_report_on_cancel(false); // TODO bug llj fix this not projected by lock | ||||||
| } | ||||||
|
|
||||||
| _runtime_state->set_process_status(_query_ctx->exec_status()); | ||||||
| // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe | ||||||
| // For stream load the fragment's query_id == load id, it is set in FE. | ||||||
| auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id); | ||||||
|
|
@@ -164,7 +178,8 @@ void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason, | |||||
| } | ||||||
|
|
||||||
| // must close stream_mgr to avoid dead lock in Exchange Node | ||||||
| _exec_env->vstream_mgr()->cancel(_fragment_instance_id); | ||||||
| // TODO bug llj fix this other instance will not cancel | ||||||
| _exec_env->vstream_mgr()->cancel(_fragment_instance_id, Status::Cancelled(msg)); | ||||||
| // Cancel the result queue manager used by spark doris connector | ||||||
| // TODO pipeline incomp | ||||||
| // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg)); | ||||||
|
|
@@ -199,6 +214,7 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re | |||||
|
|
||||||
| LOG_INFO("PipelineFragmentContext::prepare") | ||||||
| .tag("query_id", print_id(_query_id)) | ||||||
| .tag("fragment_id", _fragment_id) | ||||||
| .tag("instance_id", print_id(local_params.fragment_instance_id)) | ||||||
| .tag("backend_num", local_params.backend_num) | ||||||
| .tag("pthread_id", (uintptr_t)pthread_self()); | ||||||
|
|
@@ -311,6 +327,8 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re | |||||
| _runtime_state->runtime_profile()->add_child(_root_plan->runtime_profile(), true, nullptr); | ||||||
| _runtime_state->runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); | ||||||
|
|
||||||
| _init_next_report_time(); | ||||||
|
|
||||||
| _prepared = true; | ||||||
| return Status::OK(); | ||||||
| } | ||||||
|
|
@@ -344,54 +362,56 @@ Status PipelineFragmentContext::_build_pipeline_tasks( | |||||
| return Status::OK(); | ||||||
| } | ||||||
|
|
||||||
| void PipelineFragmentContext::_stop_report_thread() { | ||||||
| if (!_report_thread_active) { | ||||||
| return; | ||||||
| void PipelineFragmentContext::_init_next_report_time() { | ||||||
| auto interval_s = config::pipeline_status_report_interval; | ||||||
| if (_is_report_success && interval_s > 0 && _query_ctx->timeout_second > interval_s) { | ||||||
| std::vector<string> ins_ids; | ||||||
| instance_ids(ins_ids); | ||||||
| VLOG_FILE << "enable period report: instance_id=" | ||||||
| << fmt::format("{}", fmt::join(ins_ids, ", ")); | ||||||
| uint64_t report_fragment_offset = (uint64_t)(rand() % interval_s) * NANOS_PER_SEC; | ||||||
| // We don't want to wait longer than it takes to run the entire fragment. | ||||||
| _previous_report_time = | ||||||
| MonotonicNanos() + report_fragment_offset - (uint64_t)(interval_s)*NANOS_PER_SEC; | ||||||
| _disable_period_report = false; | ||||||
| } | ||||||
| } | ||||||
|
|
||||||
| _report_thread_active = false; | ||||||
|
|
||||||
| _stop_report_thread_cv.notify_one(); | ||||||
| // Wait infinitly to ensure that the report task is finished and the this variable | ||||||
| // is not used in report thread. | ||||||
| _report_thread_future.wait(); | ||||||
| void PipelineFragmentContext::refresh_next_report_time() { | ||||||
| auto disable = _disable_period_report.load(std::memory_order_acquire); | ||||||
| DCHECK(disable == true); | ||||||
| _previous_report_time.store(MonotonicNanos(), std::memory_order_release); | ||||||
| _disable_period_report.compare_exchange_strong(disable, false); | ||||||
| } | ||||||
|
|
||||||
| void PipelineFragmentContext::report_profile() { | ||||||
| SCOPED_ATTACH_TASK(_runtime_state.get()); | ||||||
| VLOG_FILE << "report_profile(): instance_id=" << _runtime_state->fragment_instance_id(); | ||||||
|
|
||||||
| _report_thread_active = true; | ||||||
|
|
||||||
| std::unique_lock<std::mutex> l(_report_thread_lock); | ||||||
| // tell Open() that we started | ||||||
| _report_thread_started_cv.notify_one(); | ||||||
|
|
||||||
| // Jitter the reporting time of remote fragments by a random amount between | ||||||
| // 0 and the report_interval. This way, the coordinator doesn't get all the | ||||||
| // updates at once so its better for contention as well as smoother progress | ||||||
| // reporting. | ||||||
| int report_fragment_offset = rand() % config::status_report_interval; | ||||||
| // We don't want to wait longer than it takes to run the entire fragment. | ||||||
| _stop_report_thread_cv.wait_for(l, std::chrono::seconds(report_fragment_offset)); | ||||||
| while (_report_thread_active) { | ||||||
| if (config::status_report_interval > 0) { | ||||||
| // wait_for can return because the timeout occurred or the condition variable | ||||||
| // was signaled. We can't rely on its return value to distinguish between the | ||||||
| // two cases (e.g. there is a race here where the wait timed out but before grabbing | ||||||
| // the lock, the condition variable was signaled). Instead, we will use an external | ||||||
| // flag, _report_thread_active, to coordinate this. | ||||||
| _stop_report_thread_cv.wait_for(l, | ||||||
| std::chrono::seconds(config::status_report_interval)); | ||||||
| } else { | ||||||
| LOG(WARNING) << "config::status_report_interval is equal to or less than zero, exiting " | ||||||
| "reporting thread."; | ||||||
| break; | ||||||
| void PipelineFragmentContext::trigger_report_if_necessary() { | ||||||
| if (!_is_report_success) { | ||||||
| return; | ||||||
| } | ||||||
| auto disable = _disable_period_report.load(std::memory_order_acquire); | ||||||
| if (disable) { | ||||||
| return; | ||||||
| } | ||||||
| int32_t interval_s = config::pipeline_status_report_interval; | ||||||
| if (interval_s <= 0) { | ||||||
| LOG(WARNING) | ||||||
| << "config::status_report_interval is equal to or less than zero, do not trigger " | ||||||
| "report."; | ||||||
| } | ||||||
| uint64_t next_report_time = _previous_report_time.load(std::memory_order_acquire) + | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. warning: variable 'next_report_time' is not initialized [cppcoreguidelines-init-variables]
Suggested change
|
||||||
| (uint64_t)(interval_s)*NANOS_PER_SEC; | ||||||
| if (MonotonicNanos() > next_report_time) { | ||||||
| if (!_disable_period_report.compare_exchange_strong(disable, true, | ||||||
| std::memory_order_acq_rel)) { | ||||||
| return; | ||||||
| } | ||||||
|
|
||||||
| if (VLOG_FILE_IS_ON) { | ||||||
| VLOG_FILE << "Reporting " << (!_report_thread_active ? "final " : " ") | ||||||
| << "profile for instance " << _runtime_state->fragment_instance_id(); | ||||||
| std::vector<string> ins_ids; | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. warning: variable 'ins_ids' is not initialized [cppcoreguidelines-init-variables] be/src/pipeline/pipeline_fragment_context.cpp:410: - ;
+ = 0; |
||||||
| instance_ids(ins_ids); | ||||||
| VLOG_FILE << "Reporting " | ||||||
| << "profile for query_id " << print_id(_query_id) | ||||||
| << ", instance ids: " << fmt::format("{}", fmt::join(ins_ids, ", ")); | ||||||
|
|
||||||
| std::stringstream ss; | ||||||
| _runtime_state->runtime_profile()->compute_time_in_profile(); | ||||||
| _runtime_state->runtime_profile()->pretty_print(&ss); | ||||||
|
|
@@ -401,15 +421,13 @@ void PipelineFragmentContext::report_profile() { | |||||
| } | ||||||
| VLOG_FILE << ss.str(); | ||||||
| } | ||||||
|
|
||||||
| if (!_report_thread_active) { | ||||||
| break; | ||||||
| auto st = send_report(false); | ||||||
| if (!st.ok()) { | ||||||
| disable = true; | ||||||
| _disable_period_report.compare_exchange_strong(disable, false, | ||||||
| std::memory_order_acq_rel); | ||||||
| } | ||||||
|
|
||||||
| send_report(false); | ||||||
| } | ||||||
|
|
||||||
| VLOG_FILE << "exiting reporting thread: instance_id=" << _runtime_state->fragment_instance_id(); | ||||||
| } | ||||||
|
|
||||||
| // TODO: use virtual function to do abstruct | ||||||
|
|
@@ -815,7 +833,6 @@ Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thr | |||||
| void PipelineFragmentContext::_close_action() { | ||||||
| _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time()); | ||||||
| send_report(true); | ||||||
| _stop_report_thread(); | ||||||
| // all submitted tasks done | ||||||
| _exec_env->fragment_mgr()->remove_pipeline_context(shared_from_this()); | ||||||
| } | ||||||
|
|
@@ -828,7 +845,7 @@ void PipelineFragmentContext::close_a_pipeline() { | |||||
| } | ||||||
| } | ||||||
|
|
||||||
| void PipelineFragmentContext::send_report(bool done) { | ||||||
| Status PipelineFragmentContext::send_report(bool done) { | ||||||
| Status exec_status = Status::OK(); | ||||||
| { | ||||||
| std::lock_guard<std::mutex> l(_status_lock); | ||||||
|
|
@@ -838,18 +855,18 @@ void PipelineFragmentContext::send_report(bool done) { | |||||
| // If plan is done successfully, but _is_report_success is false, | ||||||
| // no need to send report. | ||||||
| if (!_is_report_success && done && exec_status.ok()) { | ||||||
| return; | ||||||
| return Status::NeedSendAgain(""); | ||||||
| } | ||||||
|
|
||||||
| // If both _is_report_success and _is_report_on_cancel are false, | ||||||
| // which means no matter query is success or failed, no report is needed. | ||||||
| // This may happen when the query limit reached and | ||||||
| // a internal cancellation being processed | ||||||
| if (!_is_report_success && !_is_report_on_cancel) { | ||||||
| return; | ||||||
| return Status::NeedSendAgain(""); | ||||||
| } | ||||||
|
|
||||||
| _report_status_cb( | ||||||
| return _report_status_cb( | ||||||
| {false, | ||||||
| exec_status, | ||||||
| {}, | ||||||
|
|
@@ -864,7 +881,8 @@ void PipelineFragmentContext::send_report(bool done) { | |||||
| _runtime_state.get(), | ||||||
| std::bind(&PipelineFragmentContext::update_status, this, std::placeholders::_1), | ||||||
| std::bind(&PipelineFragmentContext::cancel, this, std::placeholders::_1, | ||||||
| std::placeholders::_2)}); | ||||||
| std::placeholders::_2)}, | ||||||
| shared_from_this()); | ||||||
| } | ||||||
|
|
||||||
| } // namespace doris::pipeline | ||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
warning: variable 'ins_ids' is not initialized [cppcoreguidelines-init-variables]
be/src/pipeline/pipeline_fragment_context.cpp:369: