/root/doris/be/src/runtime/query_context.cpp
Line | Count | Source (jump to first uncovered line) |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | #include "runtime/query_context.h" |
19 | | |
20 | | #include <fmt/core.h> |
21 | | #include <gen_cpp/FrontendService_types.h> |
22 | | #include <gen_cpp/RuntimeProfile_types.h> |
23 | | #include <gen_cpp/Types_types.h> |
24 | | #include <glog/logging.h> |
25 | | |
26 | | #include <algorithm> |
27 | | #include <exception> |
28 | | #include <memory> |
29 | | #include <mutex> |
30 | | #include <utility> |
31 | | #include <vector> |
32 | | |
33 | | #include "common/logging.h" |
34 | | #include "common/status.h" |
35 | | #include "olap/olap_common.h" |
36 | | #include "pipeline/dependency.h" |
37 | | #include "pipeline/pipeline_fragment_context.h" |
38 | | #include "runtime/exec_env.h" |
39 | | #include "runtime/fragment_mgr.h" |
40 | | #include "runtime/memory/heap_profiler.h" |
41 | | #include "runtime/runtime_query_statistics_mgr.h" |
42 | | #include "runtime/runtime_state.h" |
43 | | #include "runtime/thread_context.h" |
44 | | #include "runtime/workload_group/workload_group_manager.h" |
45 | | #include "util/mem_info.h" |
46 | | #include "util/uid_util.h" |
47 | | #include "vec/spill/spill_stream_manager.h" |
48 | | |
49 | | namespace doris { |
50 | | |
51 | | class DelayReleaseToken : public Runnable { |
52 | | ENABLE_FACTORY_CREATOR(DelayReleaseToken); |
53 | | |
54 | | public: |
55 | 0 | DelayReleaseToken(std::unique_ptr<ThreadPoolToken>&& token) { token_ = std::move(token); } |
56 | 0 | ~DelayReleaseToken() override = default; |
57 | 0 | void run() override {} |
58 | | std::unique_ptr<ThreadPoolToken> token_; |
59 | | }; |
60 | | |
61 | 0 | const std::string toString(QuerySource queryType) { |
62 | 0 | switch (queryType) { |
63 | 0 | case QuerySource::INTERNAL_FRONTEND: |
64 | 0 | return "INTERNAL_FRONTEND"; |
65 | 0 | case QuerySource::STREAM_LOAD: |
66 | 0 | return "STREAM_LOAD"; |
67 | 0 | case QuerySource::GROUP_COMMIT_LOAD: |
68 | 0 | return "EXTERNAL_QUERY"; |
69 | 0 | case QuerySource::ROUTINE_LOAD: |
70 | 0 | return "ROUTINE_LOAD"; |
71 | 0 | case QuerySource::EXTERNAL_CONNECTOR: |
72 | 0 | return "EXTERNAL_CONNECTOR"; |
73 | 0 | default: |
74 | 0 | return "UNKNOWN"; |
75 | 0 | } |
76 | 0 | } |
77 | | |
78 | 11 | std::unique_ptr<TaskController> QueryContext::QueryTaskController::create(QueryContext* query_ctx) { |
79 | 11 | return QueryContext::QueryTaskController::create_unique(query_ctx->shared_from_this()); |
80 | 11 | } |
81 | | |
82 | 0 | bool QueryContext::QueryTaskController::is_cancelled() const { |
83 | 0 | auto query_ctx = query_ctx_.lock(); |
84 | 0 | if (query_ctx == nullptr) { |
85 | 0 | return true; |
86 | 0 | } |
87 | 0 | return query_ctx->is_cancelled(); |
88 | 0 | } |
89 | | |
90 | 0 | Status QueryContext::QueryTaskController::cancel(const Status& reason, int fragment_id) { |
91 | 0 | auto query_ctx = query_ctx_.lock(); |
92 | 0 | if (query_ctx == nullptr) { |
93 | 0 | return Status::InternalError("QueryContext is destroyed"); |
94 | 0 | } |
95 | 0 | query_ctx->cancel(reason, fragment_id); |
96 | 0 | return Status::OK(); |
97 | 0 | } |
98 | | |
99 | 174 | std::unique_ptr<MemoryContext> QueryContext::QueryMemoryContext::create() { |
100 | 174 | return QueryContext::QueryMemoryContext::create_unique(); |
101 | 174 | } |
102 | | |
103 | | std::shared_ptr<QueryContext> QueryContext::create(TUniqueId query_id, ExecEnv* exec_env, |
104 | | const TQueryOptions& query_options, |
105 | | TNetworkAddress coord_addr, bool is_nereids, |
106 | | TNetworkAddress current_connect_fe, |
107 | 11 | QuerySource query_type) { |
108 | 11 | auto ctx = QueryContext::create_shared(query_id, exec_env, query_options, coord_addr, |
109 | 11 | is_nereids, current_connect_fe, query_type); |
110 | 11 | ctx->init_query_task_controller(); |
111 | 11 | return ctx; |
112 | 11 | } |
113 | | |
114 | | QueryContext::QueryContext(TUniqueId query_id, ExecEnv* exec_env, |
115 | | const TQueryOptions& query_options, TNetworkAddress coord_addr, |
116 | | bool is_nereids, TNetworkAddress current_connect_fe, |
117 | | QuerySource query_source) |
118 | | : _timeout_second(-1), |
119 | | _query_id(std::move(query_id)), |
120 | | _exec_env(exec_env), |
121 | | _is_nereids(is_nereids), |
122 | | _query_options(query_options), |
123 | 174 | _query_source(query_source) { |
124 | 174 | _init_resource_context(); |
125 | 174 | SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_mem_tracker()); |
126 | 174 | _query_watcher.start(); |
127 | 174 | _shared_hash_table_controller.reset(new vectorized::SharedHashTableController()); |
128 | 174 | _execution_dependency = pipeline::QueryGlobalDependency::create_unique("ExecutionDependency"); |
129 | 174 | _memory_sufficient_dependency = |
130 | 174 | pipeline::QueryGlobalDependency::create_unique("MemorySufficientDependency", true); |
131 | | |
132 | 174 | _runtime_filter_mgr = std::make_unique<RuntimeFilterMgr>( |
133 | 174 | TUniqueId(), RuntimeFilterParamsContext::create(this), query_mem_tracker(), true); |
134 | | |
135 | 174 | _timeout_second = query_options.execution_timeout; |
136 | | |
137 | 174 | bool is_query_type_valid = query_options.query_type == TQueryType::SELECT || |
138 | 174 | query_options.query_type == TQueryType::LOAD || |
139 | 174 | query_options.query_type == TQueryType::EXTERNAL; |
140 | 174 | DCHECK_EQ(is_query_type_valid, true); |
141 | | |
142 | 174 | this->coord_addr = coord_addr; |
143 | | // current_connect_fe is used for report query statistics |
144 | 174 | this->current_connect_fe = current_connect_fe; |
145 | | // external query has no current_connect_fe |
146 | 174 | if (query_options.query_type != TQueryType::EXTERNAL) { |
147 | 54 | bool is_report_fe_addr_valid = |
148 | 54 | !this->current_connect_fe.hostname.empty() && this->current_connect_fe.port != 0; |
149 | 54 | DCHECK_EQ(is_report_fe_addr_valid, true); |
150 | 54 | } |
151 | 174 | clock_gettime(CLOCK_MONOTONIC, &this->_query_arrival_timestamp); |
152 | 174 | DorisMetrics::instance()->query_ctx_cnt->increment(1); |
153 | 174 | } |
154 | | |
155 | 174 | void QueryContext::_init_query_mem_tracker() { |
156 | 174 | bool has_query_mem_limit = _query_options.__isset.mem_limit && (_query_options.mem_limit > 0); |
157 | 174 | int64_t bytes_limit = has_query_mem_limit ? _query_options.mem_limit : -1; |
158 | 174 | if (bytes_limit > MemInfo::mem_limit() || bytes_limit == -1) { |
159 | 0 | VLOG_NOTICE << "Query memory limit " << PrettyPrinter::print(bytes_limit, TUnit::BYTES) |
160 | 0 | << " exceeds process memory limit of " |
161 | 0 | << PrettyPrinter::print(MemInfo::mem_limit(), TUnit::BYTES) |
162 | 0 | << " OR is -1. Using process memory limit instead."; |
163 | 0 | bytes_limit = MemInfo::mem_limit(); |
164 | 0 | } |
165 | | // If the query is a pure load task(streamload, routine load, group commit), then it should not use |
166 | | // memlimit per query to limit their memory usage. |
167 | 174 | if (is_pure_load_task()) { |
168 | 120 | bytes_limit = MemInfo::mem_limit(); |
169 | 120 | } |
170 | 174 | std::shared_ptr<MemTrackerLimiter> query_mem_tracker; |
171 | 174 | if (_query_options.query_type == TQueryType::SELECT) { |
172 | 54 | query_mem_tracker = MemTrackerLimiter::create_shared( |
173 | 54 | MemTrackerLimiter::Type::QUERY, fmt::format("Query#Id={}", print_id(_query_id)), |
174 | 54 | bytes_limit); |
175 | 120 | } else if (_query_options.query_type == TQueryType::LOAD) { |
176 | 0 | query_mem_tracker = MemTrackerLimiter::create_shared( |
177 | 0 | MemTrackerLimiter::Type::LOAD, fmt::format("Load#Id={}", print_id(_query_id)), |
178 | 0 | bytes_limit); |
179 | 120 | } else if (_query_options.query_type == TQueryType::EXTERNAL) { // spark/flink/etc.. |
180 | 120 | query_mem_tracker = MemTrackerLimiter::create_shared( |
181 | 120 | MemTrackerLimiter::Type::QUERY, fmt::format("External#Id={}", print_id(_query_id)), |
182 | 120 | bytes_limit); |
183 | 120 | } else { |
184 | 0 | LOG(FATAL) << "__builtin_unreachable"; |
185 | 0 | __builtin_unreachable(); |
186 | 0 | } |
187 | 174 | if (_query_options.__isset.is_report_success && _query_options.is_report_success) { |
188 | 0 | query_mem_tracker->enable_print_log_usage(); |
189 | 0 | } |
190 | | |
191 | 174 | query_mem_tracker->set_enable_reserve_memory(_query_options.__isset.enable_reserve_memory && |
192 | 174 | _query_options.enable_reserve_memory); |
193 | 174 | _user_set_mem_limit = bytes_limit; |
194 | 174 | _adjusted_mem_limit = bytes_limit; |
195 | | |
196 | 174 | _resource_ctx->memory_context()->set_mem_tracker(query_mem_tracker); |
197 | 174 | } |
198 | | |
199 | 174 | void QueryContext::_init_resource_context() { |
200 | 174 | _resource_ctx = ResourceContext::create_shared(); |
201 | 174 | _resource_ctx->set_memory_context(QueryContext::QueryMemoryContext::create()); |
202 | 174 | _init_query_mem_tracker(); |
203 | 174 | } |
204 | | |
205 | 11 | void QueryContext::init_query_task_controller() { |
206 | 11 | _resource_ctx->set_task_controller(QueryContext::QueryTaskController::create(this)); |
207 | 11 | _resource_ctx->task_controller()->set_task_id(_query_id); |
208 | 11 | _resource_ctx->task_controller()->set_fe_addr(current_connect_fe); |
209 | 11 | _resource_ctx->task_controller()->set_query_type(_query_options.query_type); |
210 | | #ifndef BE_TEST |
211 | | _exec_env->runtime_query_statistics_mgr()->register_resource_context(print_id(_query_id), |
212 | | _resource_ctx); |
213 | | #endif |
214 | 11 | } |
215 | | |
216 | 174 | QueryContext::~QueryContext() { |
217 | 174 | SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(query_mem_tracker()); |
218 | | // query mem tracker consumption is equal to 0, it means that after QueryContext is created, |
219 | | // it is found that query already exists in _query_ctx_map, and query mem tracker is not used. |
220 | | // query mem tracker consumption is not equal to 0 after use, because there is memory consumed |
221 | | // on query mem tracker, released on other trackers. |
222 | 174 | std::string mem_tracker_msg; |
223 | 174 | if (query_mem_tracker()->peak_consumption() != 0) { |
224 | 6 | mem_tracker_msg = fmt::format( |
225 | 6 | "deregister query/load memory tracker, queryId={}, Limit={}, CurrUsed={}, " |
226 | 6 | "PeakUsed={}", |
227 | 6 | print_id(_query_id), MemCounter::print_bytes(query_mem_tracker()->limit()), |
228 | 6 | MemCounter::print_bytes(query_mem_tracker()->consumption()), |
229 | 6 | MemCounter::print_bytes(query_mem_tracker()->peak_consumption())); |
230 | 6 | } |
231 | 174 | [[maybe_unused]] uint64_t group_id = 0; |
232 | 174 | if (workload_group()) { |
233 | 10 | group_id = workload_group()->id(); // before remove |
234 | 10 | } |
235 | | |
236 | 174 | _resource_ctx->task_controller()->finish(); |
237 | | |
238 | 174 | if (enable_profile()) { |
239 | 0 | _report_query_profile(); |
240 | 0 | } |
241 | | |
242 | | // Not release the the thread token in query context's dector method, because the query |
243 | | // conext may be dectored in the thread token it self. It is very dangerous and may core. |
244 | | // And also thread token need shutdown, it may take some time, may cause the thread that |
245 | | // release the token hang, the thread maybe a pipeline task scheduler thread. |
246 | 174 | if (_thread_token) { |
247 | 0 | Status submit_st = ExecEnv::GetInstance()->lazy_release_obj_pool()->submit( |
248 | 0 | DelayReleaseToken::create_shared(std::move(_thread_token))); |
249 | 0 | if (!submit_st.ok()) { |
250 | 0 | LOG(WARNING) << "Failed to release query context thread token, query_id " |
251 | 0 | << print_id(_query_id) << ", error status " << submit_st; |
252 | 0 | } |
253 | 0 | } |
254 | | #ifndef BE_TEST |
255 | | if (ExecEnv::GetInstance()->pipeline_tracer_context()->enabled()) [[unlikely]] { |
256 | | try { |
257 | | ExecEnv::GetInstance()->pipeline_tracer_context()->end_query(_query_id, group_id); |
258 | | } catch (std::exception& e) { |
259 | | LOG(WARNING) << "Dump trace log failed bacause " << e.what(); |
260 | | } |
261 | | } |
262 | | #endif |
263 | 174 | _runtime_filter_mgr.reset(); |
264 | 174 | _execution_dependency.reset(); |
265 | 174 | _shared_hash_table_controller.reset(); |
266 | 174 | _runtime_predicates.clear(); |
267 | 174 | file_scan_range_params_map.clear(); |
268 | 174 | obj_pool.clear(); |
269 | 174 | _merge_controller_handler.reset(); |
270 | | |
271 | | #ifndef BE_TEST |
272 | | _exec_env->spill_stream_mgr()->async_cleanup_query(_query_id); |
273 | | #endif |
274 | 174 | DorisMetrics::instance()->query_ctx_cnt->increment(-1); |
275 | | // the only one msg shows query's end. any other msg should append to it if need. |
276 | 174 | LOG_INFO("Query {} deconstructed, mem_tracker: {}", print_id(this->_query_id), mem_tracker_msg); |
277 | 174 | } |
278 | | |
279 | 1 | void QueryContext::set_ready_to_execute(Status reason) { |
280 | 1 | set_execution_dependency_ready(); |
281 | 1 | _exec_status.update(reason); |
282 | 1 | if (query_mem_tracker() && !reason.ok()) { |
283 | 1 | query_mem_tracker()->set_is_query_cancelled(!reason.ok()); |
284 | 1 | } |
285 | 1 | } |
286 | | |
287 | 0 | void QueryContext::set_ready_to_execute_only() { |
288 | 0 | set_execution_dependency_ready(); |
289 | 0 | } |
290 | | |
291 | 1 | void QueryContext::set_execution_dependency_ready() { |
292 | 1 | _execution_dependency->set_ready(); |
293 | 1 | } |
294 | | |
295 | 18 | void QueryContext::set_memory_sufficient(bool sufficient) { |
296 | 18 | if (sufficient) { |
297 | 8 | { |
298 | 8 | _memory_sufficient_dependency->set_ready(); |
299 | 8 | std::lock_guard l(_paused_mutex); |
300 | 8 | _paused_reason = Status::OK(); |
301 | 8 | } |
302 | 10 | } else { |
303 | 10 | _memory_sufficient_dependency->block(); |
304 | 10 | ++_paused_count; |
305 | 10 | } |
306 | 18 | } |
307 | | |
308 | 1 | void QueryContext::cancel(Status new_status, int fragment_id) { |
309 | 1 | if (!_exec_status.update(new_status)) { |
310 | 0 | return; |
311 | 0 | } |
312 | | // Tasks should be always runnable. |
313 | 1 | _execution_dependency->set_always_ready(); |
314 | 1 | _memory_sufficient_dependency->set_always_ready(); |
315 | 1 | if ((new_status.is<ErrorCode::MEM_LIMIT_EXCEEDED>() || |
316 | 1 | new_status.is<ErrorCode::MEM_ALLOC_FAILED>()) && |
317 | 1 | _query_options.__isset.dump_heap_profile_when_mem_limit_exceeded && |
318 | 1 | _query_options.dump_heap_profile_when_mem_limit_exceeded) { |
319 | | // if query is cancelled because of query mem limit exceeded, dump heap profile |
320 | | // at the time of cancellation can get the most accurate memory usage for problem analysis |
321 | 0 | auto wg = workload_group(); |
322 | 0 | auto log_str = fmt::format( |
323 | 0 | "Query {} canceled because of memory limit exceeded, dumping memory " |
324 | 0 | "detail profiles. wg: {}. {}", |
325 | 0 | print_id(_query_id), wg ? wg->debug_string() : "null", |
326 | 0 | doris::ProcessProfile::instance()->memory_profile()->process_memory_detail_str()); |
327 | 0 | LOG_LONG_STRING(INFO, log_str); |
328 | 0 | std::string dot = HeapProfiler::instance()->dump_heap_profile_to_dot(); |
329 | 0 | if (!dot.empty()) { |
330 | 0 | dot += "\n-------------------------------------------------------\n"; |
331 | 0 | dot += "Copy the text after `digraph` in the above output to " |
332 | 0 | "http://www.webgraphviz.com to generate a dot graph.\n" |
333 | 0 | "after start heap profiler, if there is no operation, will print `No nodes " |
334 | 0 | "to " |
335 | 0 | "print`." |
336 | 0 | "If there are many errors: `addr2line: Dwarf Error`," |
337 | 0 | "or other FAQ, reference doc: " |
338 | 0 | "https://doris.apache.org/community/developer-guide/debug-tool/#4-qa\n"; |
339 | 0 | auto log_str = |
340 | 0 | fmt::format("Query {}, dump heap profile to dot: {}", print_id(_query_id), dot); |
341 | 0 | LOG_LONG_STRING(INFO, log_str); |
342 | 0 | } |
343 | 0 | } |
344 | | |
345 | 1 | set_ready_to_execute(new_status); |
346 | 1 | cancel_all_pipeline_context(new_status, fragment_id); |
347 | 1 | } |
348 | | |
349 | 1 | void QueryContext::cancel_all_pipeline_context(const Status& reason, int fragment_id) { |
350 | 1 | std::vector<std::weak_ptr<pipeline::PipelineFragmentContext>> ctx_to_cancel; |
351 | 1 | { |
352 | 1 | std::lock_guard<std::mutex> lock(_pipeline_map_write_lock); |
353 | 1 | for (auto& [f_id, f_context] : _fragment_id_to_pipeline_ctx) { |
354 | 0 | if (fragment_id == f_id) { |
355 | 0 | continue; |
356 | 0 | } |
357 | 0 | ctx_to_cancel.push_back(f_context); |
358 | 0 | } |
359 | 1 | } |
360 | 1 | for (auto& f_context : ctx_to_cancel) { |
361 | 0 | if (auto pipeline_ctx = f_context.lock()) { |
362 | 0 | pipeline_ctx->cancel(reason); |
363 | 0 | } |
364 | 0 | } |
365 | 1 | } |
366 | | |
367 | 0 | std::string QueryContext::print_all_pipeline_context() { |
368 | 0 | std::vector<std::weak_ptr<pipeline::PipelineFragmentContext>> ctx_to_print; |
369 | 0 | fmt::memory_buffer debug_string_buffer; |
370 | 0 | size_t i = 0; |
371 | 0 | { |
372 | 0 | fmt::format_to(debug_string_buffer, "{} pipeline fragment contexts in query {}. \n", |
373 | 0 | _fragment_id_to_pipeline_ctx.size(), print_id(_query_id)); |
374 | |
|
375 | 0 | { |
376 | 0 | std::lock_guard<std::mutex> lock(_pipeline_map_write_lock); |
377 | 0 | for (auto& [f_id, f_context] : _fragment_id_to_pipeline_ctx) { |
378 | 0 | ctx_to_print.push_back(f_context); |
379 | 0 | } |
380 | 0 | } |
381 | 0 | for (auto& f_context : ctx_to_print) { |
382 | 0 | if (auto pipeline_ctx = f_context.lock()) { |
383 | 0 | auto elapsed = pipeline_ctx->elapsed_time() / 1000000000.0; |
384 | 0 | fmt::format_to(debug_string_buffer, |
385 | 0 | "No.{} (elapse_second={}s, fragment_id={}) : {}\n", i, elapsed, |
386 | 0 | pipeline_ctx->get_fragment_id(), pipeline_ctx->debug_string()); |
387 | 0 | i++; |
388 | 0 | } |
389 | 0 | } |
390 | 0 | } |
391 | 0 | return fmt::to_string(debug_string_buffer); |
392 | 0 | } |
393 | | |
394 | | void QueryContext::set_pipeline_context( |
395 | 0 | const int fragment_id, std::shared_ptr<pipeline::PipelineFragmentContext> pip_ctx) { |
396 | 0 | std::lock_guard<std::mutex> lock(_pipeline_map_write_lock); |
397 | 0 | _fragment_id_to_pipeline_ctx.insert({fragment_id, pip_ctx}); |
398 | 0 | } |
399 | | |
400 | 0 | doris::pipeline::TaskScheduler* QueryContext::get_pipe_exec_scheduler() { |
401 | 0 | if (workload_group()) { |
402 | 0 | if (_task_scheduler) { |
403 | 0 | return _task_scheduler; |
404 | 0 | } |
405 | 0 | } |
406 | 0 | return _exec_env->pipeline_task_scheduler(); |
407 | 0 | } |
408 | | |
409 | 0 | ThreadPool* QueryContext::get_memtable_flush_pool() { |
410 | 0 | if (workload_group()) { |
411 | 0 | return _memtable_flush_pool; |
412 | 0 | } else { |
413 | 0 | return nullptr; |
414 | 0 | } |
415 | 0 | } |
416 | | |
417 | 10 | void QueryContext::set_workload_group(WorkloadGroupPtr& wg) { |
418 | 10 | _resource_ctx->set_workload_group(wg); |
419 | | // Should add query first, then the workload group will not be deleted. |
420 | | // see task_group_manager::delete_workload_group_by_ids |
421 | 10 | workload_group()->add_mem_tracker_limiter(query_mem_tracker()); |
422 | 10 | workload_group()->get_query_scheduler(&_task_scheduler, &_scan_task_scheduler, |
423 | 10 | &_memtable_flush_pool, &_remote_scan_task_scheduler); |
424 | 10 | } |
425 | | |
426 | | void QueryContext::add_fragment_profile( |
427 | | int fragment_id, const std::vector<std::shared_ptr<TRuntimeProfileTree>>& pipeline_profiles, |
428 | 0 | std::shared_ptr<TRuntimeProfileTree> load_channel_profile) { |
429 | 0 | if (pipeline_profiles.empty()) { |
430 | 0 | std::string msg = fmt::format("Add pipeline profile failed, query {}, fragment {}", |
431 | 0 | print_id(this->_query_id), fragment_id); |
432 | 0 | LOG_ERROR(msg); |
433 | 0 | DCHECK(false) << msg; |
434 | 0 | return; |
435 | 0 | } |
436 | | |
437 | 0 | #ifndef NDEBUG |
438 | 0 | for (const auto& p : pipeline_profiles) { |
439 | 0 | DCHECK(p != nullptr) << fmt::format("Add pipeline profile failed, query {}, fragment {}", |
440 | 0 | print_id(this->_query_id), fragment_id); |
441 | 0 | } |
442 | 0 | #endif |
443 | |
|
444 | 0 | std::lock_guard<std::mutex> l(_profile_mutex); |
445 | 0 | VLOG_ROW << fmt::format( |
446 | 0 | "Query add fragment profile, query {}, fragment {}, pipeline profile count {} ", |
447 | 0 | print_id(this->_query_id), fragment_id, pipeline_profiles.size()); |
448 | |
|
449 | 0 | _profile_map.insert(std::make_pair(fragment_id, pipeline_profiles)); |
450 | |
|
451 | 0 | if (load_channel_profile != nullptr) { |
452 | 0 | _load_channel_profile_map.insert(std::make_pair(fragment_id, load_channel_profile)); |
453 | 0 | } |
454 | 0 | } |
455 | | |
456 | 0 | void QueryContext::_report_query_profile() { |
457 | 0 | std::lock_guard<std::mutex> lg(_profile_mutex); |
458 | |
|
459 | 0 | for (auto& [fragment_id, fragment_profile] : _profile_map) { |
460 | 0 | std::shared_ptr<TRuntimeProfileTree> load_channel_profile = nullptr; |
461 | |
|
462 | 0 | if (_load_channel_profile_map.contains(fragment_id)) { |
463 | 0 | load_channel_profile = _load_channel_profile_map[fragment_id]; |
464 | 0 | } |
465 | |
|
466 | 0 | ExecEnv::GetInstance()->runtime_query_statistics_mgr()->register_fragment_profile( |
467 | 0 | _query_id, this->coord_addr, fragment_id, fragment_profile, load_channel_profile); |
468 | 0 | } |
469 | |
|
470 | 0 | ExecEnv::GetInstance()->runtime_query_statistics_mgr()->trigger_report_profile(); |
471 | 0 | } |
472 | | |
473 | | void QueryContext::get_revocable_info(size_t* revocable_size, size_t* memory_usage, |
474 | 203 | bool* has_running_task) const { |
475 | 203 | *revocable_size = 0; |
476 | 203 | for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { |
477 | 0 | auto fragment_ctx = fragment_wptr.lock(); |
478 | 0 | if (!fragment_ctx) { |
479 | 0 | continue; |
480 | 0 | } |
481 | | |
482 | 0 | *revocable_size += fragment_ctx->get_revocable_size(has_running_task); |
483 | | |
484 | | // Should wait for all tasks are not running before revoking memory. |
485 | 0 | if (*has_running_task) { |
486 | 0 | break; |
487 | 0 | } |
488 | 0 | } |
489 | | |
490 | 203 | *memory_usage = query_mem_tracker()->consumption(); |
491 | 203 | } |
492 | | |
493 | 0 | size_t QueryContext::get_revocable_size() const { |
494 | 0 | size_t revocable_size = 0; |
495 | 0 | for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { |
496 | 0 | auto fragment_ctx = fragment_wptr.lock(); |
497 | 0 | if (!fragment_ctx) { |
498 | 0 | continue; |
499 | 0 | } |
500 | | |
501 | 0 | bool has_running_task = false; |
502 | 0 | revocable_size += fragment_ctx->get_revocable_size(&has_running_task); |
503 | | |
504 | | // Should wait for all tasks are not running before revoking memory. |
505 | 0 | if (has_running_task) { |
506 | 0 | return 0; |
507 | 0 | } |
508 | 0 | } |
509 | 0 | return revocable_size; |
510 | 0 | } |
511 | | |
512 | 0 | Status QueryContext::revoke_memory() { |
513 | 0 | std::vector<std::pair<size_t, pipeline::PipelineTask*>> tasks; |
514 | 0 | std::vector<std::shared_ptr<pipeline::PipelineFragmentContext>> fragments; |
515 | 0 | for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { |
516 | 0 | auto fragment_ctx = fragment_wptr.lock(); |
517 | 0 | if (!fragment_ctx) { |
518 | 0 | continue; |
519 | 0 | } |
520 | | |
521 | 0 | auto tasks_of_fragment = fragment_ctx->get_revocable_tasks(); |
522 | 0 | for (auto* task : tasks_of_fragment) { |
523 | 0 | tasks.emplace_back(task->get_revocable_size(), task); |
524 | 0 | } |
525 | 0 | fragments.emplace_back(std::move(fragment_ctx)); |
526 | 0 | } |
527 | |
|
528 | 0 | std::sort(tasks.begin(), tasks.end(), [](auto&& l, auto&& r) { return l.first > r.first; }); |
529 | | |
530 | | // Do not use memlimit, use current memory usage. |
531 | | // For example, if current limit is 1.6G, but current used is 1G, if reserve failed |
532 | | // should free 200MB memory, not 300MB |
533 | 0 | const auto target_revoking_size = (int64_t)(query_mem_tracker()->consumption() * 0.2); |
534 | 0 | size_t revoked_size = 0; |
535 | 0 | size_t total_revokable_size = 0; |
536 | |
|
537 | 0 | std::vector<pipeline::PipelineTask*> chosen_tasks; |
538 | 0 | for (auto&& [revocable_size, task] : tasks) { |
539 | | // Only revoke the largest task to ensure memory is used as much as possible |
540 | | // break; |
541 | 0 | if (revoked_size < target_revoking_size) { |
542 | 0 | chosen_tasks.emplace_back(task); |
543 | 0 | revoked_size += revocable_size; |
544 | 0 | } |
545 | 0 | total_revokable_size += revocable_size; |
546 | 0 | } |
547 | |
|
548 | 0 | std::weak_ptr<QueryContext> this_ctx = shared_from_this(); |
549 | 0 | auto spill_context = std::make_shared<pipeline::SpillContext>( |
550 | 0 | chosen_tasks.size(), _query_id, [this_ctx](pipeline::SpillContext* context) { |
551 | 0 | auto query_context = this_ctx.lock(); |
552 | 0 | if (!query_context) { |
553 | 0 | return; |
554 | 0 | } |
555 | | |
556 | 0 | LOG(INFO) << query_context->debug_string() << ", context: " << ((void*)context) |
557 | 0 | << " all spill tasks done, resume it."; |
558 | 0 | query_context->set_memory_sufficient(true); |
559 | 0 | }); |
560 | |
|
561 | 0 | LOG(INFO) << fmt::format( |
562 | 0 | "{}, spill context: {}, revokable mem: {}/{}, tasks count: {}/{}", this->debug_string(), |
563 | 0 | ((void*)spill_context.get()), PrettyPrinter::print_bytes(revoked_size), |
564 | 0 | PrettyPrinter::print_bytes(total_revokable_size), chosen_tasks.size(), tasks.size()); |
565 | |
|
566 | 0 | for (auto* task : chosen_tasks) { |
567 | 0 | RETURN_IF_ERROR(task->revoke_memory(spill_context)); |
568 | 0 | } |
569 | 0 | return Status::OK(); |
570 | 0 | } |
571 | | |
572 | 0 | void QueryContext::decrease_revoking_tasks_count() { |
573 | 0 | _revoking_tasks_count.fetch_sub(1); |
574 | 0 | } |
575 | | |
576 | 203 | std::vector<pipeline::PipelineTask*> QueryContext::get_revocable_tasks() const { |
577 | 203 | std::vector<pipeline::PipelineTask*> tasks; |
578 | 203 | for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { |
579 | 0 | auto fragment_ctx = fragment_wptr.lock(); |
580 | 0 | if (!fragment_ctx) { |
581 | 0 | continue; |
582 | 0 | } |
583 | 0 | auto tasks_of_fragment = fragment_ctx->get_revocable_tasks(); |
584 | 0 | tasks.insert(tasks.end(), tasks_of_fragment.cbegin(), tasks_of_fragment.cend()); |
585 | 0 | } |
586 | 203 | return tasks; |
587 | 203 | } |
588 | | |
589 | 212 | std::string QueryContext::debug_string() { |
590 | 212 | std::lock_guard l(_paused_mutex); |
591 | 212 | return fmt::format( |
592 | 212 | "QueryId={}, Memory [Used={}, Limit={}, Peak={}], Spill[RunningSpillTaskCnt={}, " |
593 | 212 | "TotalPausedPeriodSecs={}, LatestPausedReason={}]", |
594 | 212 | print_id(_query_id), |
595 | 212 | PrettyPrinter::print(query_mem_tracker()->consumption(), TUnit::BYTES), |
596 | 212 | PrettyPrinter::print(query_mem_tracker()->limit(), TUnit::BYTES), |
597 | 212 | PrettyPrinter::print(query_mem_tracker()->peak_consumption(), TUnit::BYTES), |
598 | 212 | _revoking_tasks_count, |
599 | 212 | _memory_sufficient_dependency->watcher_elapse_time() / NANOS_PER_SEC, |
600 | 212 | _paused_reason.to_string()); |
601 | 212 | } |
602 | | |
603 | | std::unordered_map<int, std::vector<std::shared_ptr<TRuntimeProfileTree>>> |
604 | 0 | QueryContext::_collect_realtime_query_profile() const { |
605 | 0 | std::unordered_map<int, std::vector<std::shared_ptr<TRuntimeProfileTree>>> res; |
606 | |
|
607 | 0 | for (const auto& [fragment_id, fragment_ctx_wptr] : _fragment_id_to_pipeline_ctx) { |
608 | 0 | if (auto fragment_ctx = fragment_ctx_wptr.lock()) { |
609 | 0 | if (fragment_ctx == nullptr) { |
610 | 0 | std::string msg = |
611 | 0 | fmt::format("PipelineFragmentContext is nullptr, query {} fragment_id: {}", |
612 | 0 | print_id(_query_id), fragment_id); |
613 | 0 | LOG_ERROR(msg); |
614 | 0 | DCHECK(false) << msg; |
615 | 0 | continue; |
616 | 0 | } |
617 | | |
618 | 0 | auto profile = fragment_ctx->collect_realtime_profile(); |
619 | |
|
620 | 0 | if (profile.empty()) { |
621 | 0 | std::string err_msg = fmt::format( |
622 | 0 | "Get nothing when collecting profile, query {}, fragment_id: {}", |
623 | 0 | print_id(_query_id), fragment_id); |
624 | 0 | LOG_ERROR(err_msg); |
625 | 0 | DCHECK(false) << err_msg; |
626 | 0 | continue; |
627 | 0 | } |
628 | | |
629 | 0 | res.insert(std::make_pair(fragment_id, profile)); |
630 | 0 | } |
631 | 0 | } |
632 | |
|
633 | 0 | return res; |
634 | 0 | } |
635 | | |
636 | 0 | TReportExecStatusParams QueryContext::get_realtime_exec_status() const { |
637 | 0 | TReportExecStatusParams exec_status; |
638 | |
|
639 | 0 | auto realtime_query_profile = _collect_realtime_query_profile(); |
640 | 0 | std::vector<std::shared_ptr<TRuntimeProfileTree>> load_channel_profiles; |
641 | |
|
642 | 0 | for (auto load_channel_profile : _load_channel_profile_map) { |
643 | 0 | if (load_channel_profile.second != nullptr) { |
644 | 0 | load_channel_profiles.push_back(load_channel_profile.second); |
645 | 0 | } |
646 | 0 | } |
647 | |
|
648 | 0 | exec_status = RuntimeQueryStatisticsMgr::create_report_exec_status_params( |
649 | 0 | this->_query_id, std::move(realtime_query_profile), std::move(load_channel_profiles), |
650 | 0 | /*is_done=*/false); |
651 | |
|
652 | 0 | return exec_status; |
653 | 0 | } |
654 | | |
655 | | } // namespace doris |