be/src/exec/pipeline/pipeline_fragment_context.h
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 | | #pragma once |
19 | | |
20 | | #include <gen_cpp/Types_types.h> |
21 | | #include <gen_cpp/types.pb.h> |
22 | | |
23 | | #include <atomic> |
24 | | #include <cstddef> |
25 | | #include <cstdint> |
26 | | #include <functional> |
27 | | #include <memory> |
28 | | #include <mutex> |
29 | | #include <string> |
30 | | #include <vector> |
31 | | |
32 | | #include "common/status.h" |
33 | | #include "exec/pipeline/pipeline.h" |
34 | | #include "exec/pipeline/pipeline_task.h" |
35 | | #include "runtime/query_context.h" |
36 | | #include "runtime/runtime_profile.h" |
37 | | #include "runtime/runtime_state.h" |
38 | | #include "runtime/task_execution_context.h" |
39 | | #include "util/stopwatch.hpp" |
40 | | |
41 | | namespace doris { |
42 | | struct ReportStatusRequest; |
43 | | class ExecEnv; |
44 | | class RuntimeFilterMergeControllerEntity; |
45 | | class TDataSink; |
46 | | class TPipelineFragmentParams; |
47 | | |
48 | | class Dependency; |
49 | | |
50 | | class PipelineFragmentContext : public TaskExecutionContext { |
51 | | public: |
52 | | ENABLE_FACTORY_CREATOR(PipelineFragmentContext); |
53 | | // Callback to report execution status of plan fragment. |
54 | | // 'profile' is the cumulative profile, 'done' indicates whether the execution |
55 | | // is done or still continuing. |
56 | | // Note: this does not take a const RuntimeProfile&, because it might need to call |
57 | | // functions like PrettyPrint() or to_thrift(), neither of which is const |
58 | | // because they take locks. |
59 | | using report_status_callback = std::function<Status( |
60 | | const ReportStatusRequest, std::shared_ptr<PipelineFragmentContext>&&)>; |
61 | | PipelineFragmentContext(TUniqueId query_id, const TPipelineFragmentParams& request, |
62 | | std::shared_ptr<QueryContext> query_ctx, ExecEnv* exec_env, |
63 | | const std::function<void(RuntimeState*, Status*)>& call_back, |
64 | | report_status_callback report_status_cb); |
65 | | |
66 | | ~PipelineFragmentContext() override; |
67 | | |
68 | | void print_profile(const std::string& extra_info); |
69 | | |
70 | | std::vector<std::shared_ptr<TRuntimeProfileTree>> collect_realtime_profile() const; |
71 | | std::shared_ptr<TRuntimeProfileTree> collect_realtime_load_channel_profile() const; |
72 | | |
73 | | bool is_timeout(timespec now) const; |
74 | | |
75 | 58.7k | uint64_t elapsed_time() const { return _fragment_watcher.elapsed_time(); } |
76 | | |
77 | 534 | int timeout_second() const { return _timeout; } |
78 | | |
79 | | PipelinePtr add_pipeline(PipelinePtr parent = nullptr, int idx = -1); |
80 | | |
81 | 16.4M | QueryContext* get_query_ctx() { return _query_ctx.get(); } |
82 | 95.9M | [[nodiscard]] bool is_canceled() const { return _query_ctx->is_cancelled(); } |
83 | | |
84 | | Status prepare(ThreadPool* thread_pool); |
85 | | |
86 | | Status submit(); |
87 | | |
88 | 426k | void set_is_report_success(bool is_report_success) { _is_report_success = is_report_success; } |
89 | | |
90 | | void cancel(const Status reason); |
91 | | |
92 | 2.28M | TUniqueId get_query_id() const { return _query_id; } |
93 | | |
94 | 6 | [[nodiscard]] int get_fragment_id() const { return _fragment_id; } |
95 | | |
96 | | void decrement_running_task(PipelineId pipeline_id); |
97 | | |
98 | | Status send_report(bool); |
99 | | |
100 | | void trigger_report_if_necessary(); |
101 | | void refresh_next_report_time(); |
102 | | |
103 | | std::string debug_string(); |
104 | | |
105 | 798k | [[nodiscard]] int next_operator_id() { return _operator_id--; } |
106 | | |
107 | 4.55M | [[nodiscard]] int max_operator_id() const { return _operator_id; } |
108 | | |
109 | 679k | [[nodiscard]] int next_sink_operator_id() { return _sink_operator_id--; } |
110 | | |
111 | | [[nodiscard]] size_t get_revocable_size(bool* has_running_task) const; |
112 | | |
113 | | [[nodiscard]] std::vector<PipelineTask*> get_revocable_tasks() const; |
114 | | |
115 | 119k | void clear_finished_tasks() { |
116 | 119k | if (_need_notify_close) { |
117 | 30.0k | return; |
118 | 30.0k | } |
119 | 535k | for (size_t j = 0; j < _tasks.size(); j++) { |
120 | 1.57M | for (size_t i = 0; i < _tasks[j].size(); i++) { |
121 | 1.13M | _tasks[j][i].first->stop_if_finished(); |
122 | 1.13M | } |
123 | 446k | } |
124 | 89.1k | } |
125 | | |
126 | | std::string get_load_error_url(); |
127 | | std::string get_first_error_msg(); |
128 | | |
129 | | Status wait_close(bool close); |
130 | | Status rebuild(ThreadPool* thread_pool); |
131 | | Status set_to_rerun(); |
132 | | |
133 | 0 | bool need_notify_close() const { return _need_notify_close; } |
134 | | |
135 | | private: |
136 | | void _release_resource(); |
137 | | |
138 | | Status _build_and_prepare_full_pipeline(ThreadPool* thread_pool); |
139 | | |
140 | | Status _build_pipelines(ObjectPool* pool, const DescriptorTbl& descs, OperatorPtr* root, |
141 | | PipelinePtr cur_pipe); |
142 | | Status _create_tree_helper(ObjectPool* pool, const std::vector<TPlanNode>& tnodes, |
143 | | const DescriptorTbl& descs, OperatorPtr parent, int* node_idx, |
144 | | OperatorPtr* root, PipelinePtr& cur_pipe, int child_idx, |
145 | | const bool followed_by_shuffled_join, |
146 | | const bool require_bucket_distribution); |
147 | | |
148 | | Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, |
149 | | OperatorPtr& op, PipelinePtr& cur_pipe, int parent_idx, int child_idx, |
150 | | const bool followed_by_shuffled_join, |
151 | | const bool require_bucket_distribution); |
152 | | template <bool is_intersect> |
153 | | Status _build_operators_for_set_operation_node(ObjectPool* pool, const TPlanNode& tnode, |
154 | | const DescriptorTbl& descs, OperatorPtr& op, |
155 | | PipelinePtr& cur_pipe, |
156 | | std::vector<DataSinkOperatorPtr>& sink_ops); |
157 | | |
158 | | Status _create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, |
159 | | const std::vector<TExpr>& output_exprs, |
160 | | const TPipelineFragmentParams& params, const RowDescriptor& row_desc, |
161 | | RuntimeState* state, DescriptorTbl& desc_tbl, |
162 | | PipelineId cur_pipeline_id); |
163 | | Status _plan_local_exchange(int num_buckets, |
164 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
165 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
166 | | Status _plan_local_exchange(int num_buckets, int pip_idx, PipelinePtr pip, |
167 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
168 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
169 | | void _inherit_pipeline_properties(const DataDistribution& data_distribution, |
170 | | PipelinePtr pipe_with_source, PipelinePtr pipe_with_sink); |
171 | | Status _add_local_exchange(int pip_idx, int idx, int node_id, ObjectPool* pool, |
172 | | PipelinePtr cur_pipe, DataDistribution data_distribution, |
173 | | bool* do_local_exchange, int num_buckets, |
174 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
175 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
176 | | Status _add_local_exchange_impl(int idx, ObjectPool* pool, PipelinePtr cur_pipe, |
177 | | PipelinePtr new_pip, DataDistribution data_distribution, |
178 | | bool* do_local_exchange, int num_buckets, |
179 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
180 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
181 | | |
182 | | Status _build_pipeline_tasks(ThreadPool* thread_pool); |
183 | | Status _build_pipeline_tasks_for_instance( |
184 | | int instance_idx, |
185 | | const std::vector<std::shared_ptr<RuntimeProfile>>& pipeline_id_to_profile); |
186 | | void _close_fragment_instance(); |
187 | | void _init_next_report_time(); |
188 | | |
189 | | // Id of this query |
190 | | TUniqueId _query_id; |
191 | | int _fragment_id; |
192 | | |
193 | | ExecEnv* _exec_env = nullptr; |
194 | | |
195 | | std::atomic_bool _prepared = false; |
196 | | bool _submitted = false; |
197 | | |
198 | | Pipelines _pipelines; |
199 | | PipelineId _next_pipeline_id = 0; |
200 | | std::mutex _task_mutex; |
201 | | int _closed_tasks = 0; |
202 | | // After prepared, `_total_tasks` is equal to the size of `_tasks`. |
203 | | // When submit fail, `_total_tasks` is equal to the number of tasks submitted. |
204 | | std::atomic<int> _total_tasks = 0; |
205 | | |
206 | | std::unique_ptr<RuntimeProfile> _fragment_level_profile; |
207 | | bool _is_report_success = false; |
208 | | |
209 | | std::unique_ptr<RuntimeState> _runtime_state; |
210 | | |
211 | | std::shared_ptr<QueryContext> _query_ctx; |
212 | | |
213 | | MonotonicStopWatch _fragment_watcher; |
214 | | RuntimeProfile::Counter* _prepare_timer = nullptr; |
215 | | RuntimeProfile::Counter* _init_context_timer = nullptr; |
216 | | RuntimeProfile::Counter* _build_pipelines_timer = nullptr; |
217 | | RuntimeProfile::Counter* _plan_local_exchanger_timer = nullptr; |
218 | | RuntimeProfile::Counter* _prepare_all_pipelines_timer = nullptr; |
219 | | RuntimeProfile::Counter* _build_tasks_timer = nullptr; |
220 | | |
221 | | std::function<void(RuntimeState*, Status*)> _call_back; |
222 | | std::atomic_bool _is_fragment_instance_closed = false; |
223 | | |
224 | | // If this is set to false, and '_is_report_success' is false as well, |
225 | | // This executor will not report status to FE on being cancelled. |
226 | | bool _is_report_on_cancel; |
227 | | |
228 | | // 0 indicates reporting is in progress or not required |
229 | | std::atomic_bool _disable_period_report = true; |
230 | | std::atomic_uint64_t _previous_report_time = 0; |
231 | | |
232 | | // This callback is used to notify the FE of the status of the fragment. |
233 | | // For example: |
234 | | // 1. when the fragment is cancelled, it will be called. |
235 | | // 2. when the fragment is finished, it will be called. especially, when the fragment is |
236 | | // a insert into select statement, it should notfiy FE every fragment's status. |
237 | | // And also, this callback is called periodly to notify FE the load process. |
238 | | report_status_callback _report_status_cb; |
239 | | |
240 | | DescriptorTbl* _desc_tbl = nullptr; |
241 | | int _num_instances = 1; |
242 | | |
243 | | int _timeout = -1; |
244 | | bool _use_serial_source = false; |
245 | | |
246 | | OperatorPtr _root_op = nullptr; |
247 | | // |
248 | | /** |
249 | | * Matrix stores tasks with local runtime states. |
250 | | * This is a [n * m] matrix. n is parallelism of pipeline engine and m is the number of pipelines. |
251 | | * |
252 | | * 2-D matrix: |
253 | | * +-------------------------+------------+-------+ |
254 | | * | | Pipeline 0 | Pipeline 1 | ... | |
255 | | * +------------+------------+------------+-------+ |
256 | | * | Instance 0 | task 0-0 | task 0-1 | ... | |
257 | | * +------------+------------+------------+-------+ |
258 | | * | Instance 1 | task 1-0 | task 1-1 | ... | |
259 | | * +------------+------------+------------+-------+ |
260 | | * | ... | |
261 | | * +--------------------------------------+-------+ |
262 | | */ |
263 | | std::vector< |
264 | | std::vector<std::pair<std::shared_ptr<PipelineTask>, std::unique_ptr<RuntimeState>>>> |
265 | | _tasks; |
266 | | |
267 | | // TODO: remove the _sink and _multi_cast_stream_sink_senders to set both |
268 | | // of it in pipeline task not the fragment_context |
269 | | #ifdef __clang__ |
270 | | #pragma clang diagnostic push |
271 | | #pragma clang diagnostic ignored "-Wshadow-field" |
272 | | #endif |
273 | | DataSinkOperatorPtr _sink = nullptr; |
274 | | #ifdef __clang__ |
275 | | #pragma clang diagnostic pop |
276 | | #endif |
277 | | |
278 | | // `_dag` manage dependencies between pipelines by pipeline ID. the indices will be blocked by members |
279 | | std::map<PipelineId, std::vector<PipelineId>> _dag; |
280 | | |
281 | | // We use preorder traversal to create an operator tree. When we meet a join node, we should |
282 | | // build probe operator and build operator in separate pipelines. To do this, we should build |
283 | | // ProbeSide first, and use `_pipelines_to_build` to store which pipeline the build operator |
284 | | // is in, so we can build BuildSide once we complete probe side. |
285 | | struct pipeline_parent_map { |
286 | | std::map<int, std::vector<PipelinePtr>> _build_side_pipelines; |
287 | 26.3k | void push(int parent_node_id, PipelinePtr pipeline) { |
288 | 26.3k | if (!_build_side_pipelines.contains(parent_node_id)) { |
289 | 13.0k | _build_side_pipelines.insert({parent_node_id, {pipeline}}); |
290 | 13.2k | } else { |
291 | 13.2k | _build_side_pipelines[parent_node_id].push_back(pipeline); |
292 | 13.2k | } |
293 | 26.3k | } |
294 | 672k | void pop(PipelinePtr& cur_pipe, int parent_node_id, int child_idx) { |
295 | 672k | if (!_build_side_pipelines.contains(parent_node_id)) { |
296 | 646k | return; |
297 | 646k | } |
298 | 672k | DCHECK(_build_side_pipelines.contains(parent_node_id)); |
299 | 25.6k | auto& child_pipeline = _build_side_pipelines[parent_node_id]; |
300 | 25.6k | DCHECK(child_idx < child_pipeline.size()); |
301 | 25.6k | cur_pipe = child_pipeline[child_idx]; |
302 | 25.6k | } |
303 | 429k | void clear() { _build_side_pipelines.clear(); } |
304 | | } _pipeline_parent_map; |
305 | | |
306 | | std::mutex _state_map_lock; |
307 | | |
308 | | int _operator_id = 0; |
309 | | int _sink_operator_id = 0; |
310 | | /** |
311 | | * Some states are shared by tasks in different pipeline task (e.g. local exchange , broadcast join). |
312 | | * |
313 | | * local exchange sink 0 -> -> local exchange source 0 |
314 | | * LocalExchangeSharedState |
315 | | * local exchange sink 1 -> -> local exchange source 1 |
316 | | * |
317 | | * hash join build sink 0 -> -> hash join build source 0 |
318 | | * HashJoinSharedState |
319 | | * hash join build sink 1 -> -> hash join build source 1 |
320 | | * |
321 | | * So we should keep states here. |
322 | | */ |
323 | | std::map<int, |
324 | | std::pair<std::shared_ptr<BasicSharedState>, std::vector<std::shared_ptr<Dependency>>>> |
325 | | _op_id_to_shared_state; |
326 | | |
327 | | std::map<PipelineId, Pipeline*> _pip_id_to_pipeline; |
328 | | std::vector<std::unique_ptr<RuntimeFilterMgr>> _runtime_filter_mgr_map; |
329 | | |
330 | | //Here are two types of runtime states: |
331 | | // - _runtime state is at the Fragment level. |
332 | | // - _task_runtime_states is at the task level, unique to each task. |
333 | | |
334 | | std::vector<TUniqueId> _fragment_instance_ids; |
335 | | |
336 | | // Total instance num running on all BEs |
337 | | int _total_instances = -1; |
338 | | |
339 | | TPipelineFragmentParams _params; |
340 | | int32_t _parallel_instances = 0; |
341 | | |
342 | | bool _need_notify_close = false; |
343 | | }; |
344 | | } // namespace doris |