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