/root/doris/be/src/pipeline/task_scheduler.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 "task_scheduler.h" |
19 | | |
20 | | #include <fmt/format.h> |
21 | | #include <gen_cpp/Types_types.h> |
22 | | #include <gen_cpp/types.pb.h> |
23 | | #include <glog/logging.h> |
24 | | #include <sched.h> |
25 | | |
26 | | // IWYU pragma: no_include <bits/chrono.h> |
27 | | #include <algorithm> |
28 | | #include <chrono> // IWYU pragma: keep |
29 | | #include <cstddef> |
30 | | #include <functional> |
31 | | #include <memory> |
32 | | #include <mutex> |
33 | | #include <ostream> |
34 | | #include <string> |
35 | | #include <thread> |
36 | | #include <utility> |
37 | | |
38 | | #include "common/logging.h" |
39 | | #include "common/status.h" |
40 | | #include "pipeline/pipeline_task.h" |
41 | | #include "pipeline_fragment_context.h" |
42 | | #include "runtime/exec_env.h" |
43 | | #include "runtime/query_context.h" |
44 | | #include "runtime/thread_context.h" |
45 | | #include "util/thread.h" |
46 | | #include "util/threadpool.h" |
47 | | #include "util/time.h" |
48 | | #include "util/uid_util.h" |
49 | | #include "vec/runtime/vdatetime_value.h" |
50 | | |
51 | | namespace doris::pipeline { |
52 | | #include "common/compile_check_begin.h" |
53 | 1 | TaskScheduler::~TaskScheduler() { |
54 | 1 | stop(); |
55 | 1 | LOG(INFO) << "Task scheduler " << _name << " shutdown"; |
56 | 1 | } |
57 | | |
58 | 1 | Status TaskScheduler::start() { |
59 | 1 | int cores = _task_queue.cores(); |
60 | 1 | RETURN_IF_ERROR(ThreadPoolBuilder(_name) |
61 | 1 | .set_min_threads(cores) |
62 | 1 | .set_max_threads(cores) |
63 | 1 | .set_max_queue_size(0) |
64 | 1 | .set_cgroup_cpu_ctl(_cgroup_cpu_ctl) |
65 | 1 | .build(&_fix_thread_pool)); |
66 | 1 | LOG_INFO("TaskScheduler set cores").tag("size", cores); |
67 | 9 | for (int32_t i = 0; i < cores; ++i) { |
68 | 8 | RETURN_IF_ERROR(_fix_thread_pool->submit_func([this, i] { _do_work(i); })); |
69 | 8 | } |
70 | 1 | return Status::OK(); |
71 | 1 | } |
72 | | |
73 | 0 | Status TaskScheduler::schedule_task(PipelineTaskSPtr task) { |
74 | 0 | return _task_queue.push_back(task); |
75 | 0 | } |
76 | | |
77 | | // after close_task, task maybe destructed. |
78 | 0 | void close_task(PipelineTask* task, Status exec_status, PipelineFragmentContext* ctx) { |
79 | | // Has to attach memory tracker here, because the close task will also release some memory. |
80 | | // Should count the memory to the query or the query's memory will not decrease when part of |
81 | | // task finished. |
82 | 0 | SCOPED_ATTACH_TASK(task->runtime_state()); |
83 | 0 | if (!exec_status.ok()) { |
84 | 0 | ctx->cancel(exec_status); |
85 | 0 | LOG(WARNING) << fmt::format("Pipeline task failed. query_id: {} reason: {}", |
86 | 0 | print_id(ctx->get_query_id()), exec_status.to_string()); |
87 | 0 | } |
88 | 0 | Status status = task->close(exec_status); |
89 | 0 | if (!status.ok()) { |
90 | 0 | ctx->cancel(status); |
91 | 0 | } |
92 | 0 | status = task->finalize(); |
93 | 0 | if (!status.ok()) { |
94 | 0 | ctx->cancel(status); |
95 | 0 | } |
96 | 0 | } |
97 | | |
98 | 8 | void TaskScheduler::_do_work(int index) { |
99 | 8 | while (!_need_to_stop) { |
100 | 8 | auto task = _task_queue.take(index); |
101 | 8 | if (!task) { |
102 | 0 | continue; |
103 | 0 | } |
104 | | |
105 | | // The task is already running, maybe block in now dependency wake up by other thread |
106 | | // but the block thread still hold the task, so put it back to the queue, until the hold |
107 | | // thread set task->set_running(false) |
108 | 8 | if (task->is_running()) { |
109 | 0 | static_cast<void>(_task_queue.push_back(task, index)); |
110 | 0 | continue; |
111 | 0 | } |
112 | 8 | if (task->is_finalized()) { |
113 | 0 | continue; |
114 | 0 | } |
115 | 8 | auto fragment_context = task->fragment_context().lock(); |
116 | 8 | if (!fragment_context) { |
117 | | // Fragment already finishedquery |
118 | 0 | continue; |
119 | 0 | } |
120 | 8 | task->set_running(true); |
121 | 8 | bool done = false; |
122 | 8 | auto status = Status::OK(); |
123 | 8 | Defer task_running_defer {[&]() { |
124 | | // If fragment is finished, fragment context will be de-constructed with all tasks in it. |
125 | 0 | if (done || !status.ok()) { |
126 | 0 | auto id = task->pipeline_id(); |
127 | 0 | close_task(task.get(), status, fragment_context.get()); |
128 | 0 | task->set_running(false); |
129 | 0 | fragment_context->decrement_running_task(id); |
130 | 0 | } else { |
131 | 0 | task->set_running(false); |
132 | 0 | } |
133 | 0 | }}; |
134 | 8 | task->set_task_queue(&_task_queue); |
135 | 8 | task->log_detail_if_need(); |
136 | | |
137 | 8 | bool canceled = fragment_context->is_canceled(); |
138 | | |
139 | | // Close task if canceled |
140 | 8 | if (canceled) { |
141 | 0 | status = fragment_context->get_query_ctx()->exec_status(); |
142 | 0 | DCHECK(!status.ok()); |
143 | 0 | continue; |
144 | 0 | } |
145 | 8 | task->set_core_id(index); |
146 | | |
147 | | // Main logics of execution |
148 | 8 | ASSIGN_STATUS_IF_CATCH_EXCEPTION( |
149 | | //TODO: use a better enclose to abstracting these |
150 | 8 | if (ExecEnv::GetInstance()->pipeline_tracer_context()->enabled()) { |
151 | 8 | TUniqueId query_id = fragment_context->get_query_id(); |
152 | 8 | std::string task_name = task->task_name(); |
153 | | |
154 | 8 | std::thread::id tid = std::this_thread::get_id(); |
155 | 8 | uint64_t thread_id = *reinterpret_cast<uint64_t*>(&tid); |
156 | 8 | uint64_t start_time = MonotonicMicros(); |
157 | | |
158 | 8 | status = task->execute(&done); |
159 | | |
160 | 8 | uint64_t end_time = MonotonicMicros(); |
161 | 8 | ExecEnv::GetInstance()->pipeline_tracer_context()->record( |
162 | 8 | {query_id, task_name, static_cast<uint32_t>(index), thread_id, |
163 | 8 | start_time, end_time}); |
164 | 8 | } else { status = task->execute(&done); }, |
165 | 8 | status); |
166 | 0 | fragment_context->trigger_report_if_necessary(); |
167 | 0 | } |
168 | 8 | } |
169 | | |
170 | 1 | void TaskScheduler::stop() { |
171 | 1 | if (!_shutdown) { |
172 | 1 | _task_queue.close(); |
173 | 1 | if (_fix_thread_pool) { |
174 | 0 | _need_to_stop = true; |
175 | 0 | _fix_thread_pool->shutdown(); |
176 | 0 | _fix_thread_pool->wait(); |
177 | 0 | } |
178 | | // Should set at the ending of the stop to ensure that the |
179 | | // pool is stopped. For example, if there are 2 threads call stop |
180 | | // then if one thread set shutdown = false, then another thread will |
181 | | // not check it and will free task scheduler. |
182 | 1 | _shutdown = true; |
183 | 1 | } |
184 | 1 | } |
185 | | |
186 | | } // namespace doris::pipeline |