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 <brpc/closure_guard.h> |
21 | | #include <gen_cpp/Types_types.h> |
22 | | #include <gen_cpp/types.pb.h> |
23 | | |
24 | | #include <atomic> |
25 | | #include <cstddef> |
26 | | #include <cstdint> |
27 | | #include <functional> |
28 | | #include <memory> |
29 | | #include <mutex> |
30 | | #include <set> |
31 | | #include <string> |
32 | | #include <vector> |
33 | | |
34 | | #include "common/status.h" |
35 | | #include "exec/pipeline/pipeline.h" |
36 | | #include "exec/pipeline/pipeline_task.h" |
37 | | #include "runtime/query_context.h" |
38 | | #include "runtime/runtime_profile.h" |
39 | | #include "runtime/runtime_state.h" |
40 | | #include "runtime/task_execution_context.h" |
41 | | #include "util/stopwatch.hpp" |
42 | | #include "util/uid_util.h" |
43 | | |
44 | | namespace doris { |
45 | | struct ReportStatusRequest; |
46 | | class ExecEnv; |
47 | | class RuntimeFilterMergeControllerEntity; |
48 | | class TDataSink; |
49 | | class TPipelineFragmentParams; |
50 | | |
51 | | class Dependency; |
52 | | |
53 | | class PipelineFragmentContext : public TaskExecutionContext { |
54 | | public: |
55 | | ENABLE_FACTORY_CREATOR(PipelineFragmentContext); |
56 | | PipelineFragmentContext(TUniqueId query_id, const TPipelineFragmentParams& request, |
57 | | std::shared_ptr<QueryContext> query_ctx, ExecEnv* exec_env, |
58 | | const std::function<void(RuntimeState*, Status*)>& call_back); |
59 | | |
60 | | ~PipelineFragmentContext() override; |
61 | | |
62 | | void print_profile(const std::string& extra_info); |
63 | | |
64 | | std::vector<std::shared_ptr<TRuntimeProfileTree>> collect_realtime_profile() const; |
65 | | std::shared_ptr<TRuntimeProfileTree> collect_realtime_load_channel_profile() const; |
66 | | |
67 | | bool is_timeout(timespec now) const; |
68 | | |
69 | 20.3k | uint64_t elapsed_time() const { return _fragment_watcher.elapsed_time(); } |
70 | | |
71 | 0 | int timeout_second() const { return _timeout; } |
72 | | |
73 | | PipelinePtr add_pipeline(PipelinePtr parent = nullptr, int idx = -1); |
74 | | |
75 | 167k | QueryContext* get_query_ctx() { return _query_ctx.get(); } |
76 | 3.19M | [[nodiscard]] bool is_canceled() const { return _query_ctx->is_cancelled(); } |
77 | | |
78 | | Status prepare(ThreadPool* thread_pool); |
79 | | |
80 | | Status submit(); |
81 | | |
82 | 0 | void set_is_report_success(bool is_report_success) { _is_report_success = is_report_success; } |
83 | | |
84 | | void cancel(const Status reason); |
85 | | |
86 | | bool notify_close(); |
87 | | |
88 | 26 | TUniqueId get_query_id() const { return _query_id; } |
89 | | |
90 | 6 | [[nodiscard]] int get_fragment_id() const { return _fragment_id; } |
91 | | |
92 | | void decrement_running_task(PipelineId pipeline_id); |
93 | | |
94 | 2 | uint32_t rec_cte_stage() const { return _rec_cte_stage; } |
95 | 0 | void set_rec_cte_stage(uint32_t stage) { _rec_cte_stage = stage; } |
96 | | |
97 | | Status send_report(bool); |
98 | | |
99 | | void trigger_report_if_necessary(); |
100 | | void refresh_next_report_time(); |
101 | | |
102 | | std::string debug_string(); |
103 | | |
104 | 0 | [[nodiscard]] int next_operator_id() { return _operator_id--; } |
105 | | |
106 | 0 | [[nodiscard]] int max_operator_id() const { return _operator_id; } |
107 | | |
108 | 0 | [[nodiscard]] int next_sink_operator_id() { return _sink_operator_id--; } |
109 | | |
110 | | [[nodiscard]] size_t get_revocable_size(bool* has_running_task) const; |
111 | | |
112 | | [[nodiscard]] std::vector<PipelineTask*> get_revocable_tasks() const; |
113 | | |
114 | 0 | void clear_finished_tasks() { |
115 | 0 | if (_need_notify_close) { |
116 | 0 | return; |
117 | 0 | } |
118 | 0 | for (size_t j = 0; j < _tasks.size(); j++) { |
119 | 0 | for (size_t i = 0; i < _tasks[j].size(); i++) { |
120 | 0 | _tasks[j][i].first->stop_if_finished(); |
121 | 0 | } |
122 | 0 | } |
123 | 0 | } |
124 | | |
125 | | std::string get_load_error_url(); |
126 | | std::string get_first_error_msg(); |
127 | | |
128 | | std::set<int> get_deregister_runtime_filter() const; |
129 | | |
130 | | // Store the brpc ClosureGuard so the RPC response is deferred until this PFC is destroyed. |
131 | | // When need_send_report_on_destruction is true (final_close), send the report immediately |
132 | | // and do not store the guard (let it fire on return to complete the RPC). |
133 | | // |
134 | | // Thread safety: This method is NOT thread-safe. It reads/writes _wait_close_guard without |
135 | | // synchronization. Currently it is only called from rerun_fragment() which is invoked |
136 | | // sequentially by RecCTESourceOperatorX (a serial operator) — one opcode at a time per |
137 | | // fragment. Do NOT call this concurrently from multiple threads. |
138 | | Status listen_wait_close(const std::shared_ptr<brpc::ClosureGuard>& guard, |
139 | 0 | bool need_send_report_on_destruction) { |
140 | 0 | if (_wait_close_guard) { |
141 | 0 | return Status::InternalError("Already listening wait close"); |
142 | 0 | } |
143 | 0 | if (need_send_report_on_destruction) { |
144 | 0 | return send_report(true); |
145 | 0 | } else { |
146 | 0 | _wait_close_guard = guard; |
147 | 0 | } |
148 | 0 | return Status::OK(); |
149 | 0 | } |
150 | | |
151 | | private: |
152 | | void _coordinator_callback(const ReportStatusRequest& req); |
153 | | std::string _to_http_path(const std::string& file_name) const; |
154 | | |
155 | | void _release_resource(); |
156 | | |
157 | | Status _build_and_prepare_full_pipeline(ThreadPool* thread_pool); |
158 | | |
159 | | Status _build_pipelines(ObjectPool* pool, const DescriptorTbl& descs, OperatorPtr* root, |
160 | | PipelinePtr cur_pipe); |
161 | | Status _create_tree_helper(ObjectPool* pool, const std::vector<TPlanNode>& tnodes, |
162 | | const DescriptorTbl& descs, OperatorPtr parent, int* node_idx, |
163 | | OperatorPtr* root, PipelinePtr& cur_pipe, int child_idx, |
164 | | const bool followed_by_shuffled_join, |
165 | | const bool require_bucket_distribution); |
166 | | |
167 | | Status _create_operator(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, |
168 | | OperatorPtr& op, PipelinePtr& cur_pipe, int parent_idx, int child_idx, |
169 | | const bool followed_by_shuffled_join, |
170 | | const bool require_bucket_distribution, OperatorPtr& cache_op); |
171 | | template <bool is_intersect> |
172 | | Status _build_operators_for_set_operation_node(ObjectPool* pool, const TPlanNode& tnode, |
173 | | const DescriptorTbl& descs, OperatorPtr& op, |
174 | | PipelinePtr& cur_pipe, |
175 | | std::vector<DataSinkOperatorPtr>& sink_ops); |
176 | | |
177 | | Status _create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink, |
178 | | const std::vector<TExpr>& output_exprs, |
179 | | const TPipelineFragmentParams& params, const RowDescriptor& row_desc, |
180 | | RuntimeState* state, DescriptorTbl& desc_tbl, |
181 | | PipelineId cur_pipeline_id); |
182 | | Status _plan_local_exchange(int num_buckets, |
183 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
184 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
185 | | Status _plan_local_exchange(int num_buckets, int pip_idx, PipelinePtr pip, |
186 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
187 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
188 | | void _inherit_pipeline_properties(const DataDistribution& data_distribution, |
189 | | PipelinePtr pipe_with_source, PipelinePtr pipe_with_sink); |
190 | | Status _add_local_exchange(int pip_idx, int idx, int node_id, ObjectPool* pool, |
191 | | PipelinePtr cur_pipe, DataDistribution data_distribution, |
192 | | bool* do_local_exchange, int num_buckets, |
193 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
194 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
195 | | Status _add_local_exchange_impl(int idx, ObjectPool* pool, PipelinePtr cur_pipe, |
196 | | PipelinePtr new_pip, DataDistribution data_distribution, |
197 | | bool* do_local_exchange, int num_buckets, |
198 | | const std::map<int, int>& bucket_seq_to_instance_idx, |
199 | | const std::map<int, int>& shuffle_idx_to_instance_idx); |
200 | | |
201 | | Status _build_pipeline_tasks(ThreadPool* thread_pool); |
202 | | Status _build_pipeline_tasks_for_instance( |
203 | | int instance_idx, |
204 | | const std::vector<std::shared_ptr<RuntimeProfile>>& pipeline_id_to_profile); |
205 | | // Close the fragment instance and return true if the caller should call |
206 | | // remove_pipeline_context() **after** releasing _task_mutex. This avoids |
207 | | // holding _task_mutex while acquiring _pipeline_map's shard lock, which |
208 | | // would create an ABBA deadlock with dump_pipeline_tasks(). |
209 | | bool _close_fragment_instance(); |
210 | | void _init_next_report_time(); |
211 | | |
212 | | // Id of this query |
213 | | TUniqueId _query_id; |
214 | | int _fragment_id; |
215 | | |
216 | | ExecEnv* _exec_env = nullptr; |
217 | | |
218 | | std::atomic_bool _prepared = false; |
219 | | bool _submitted = false; |
220 | | |
221 | | Pipelines _pipelines; |
222 | | PipelineId _next_pipeline_id = 0; |
223 | | std::mutex _task_mutex; |
224 | | int _closed_tasks = 0; |
225 | | // After prepared, `_total_tasks` is equal to the size of `_tasks`. |
226 | | // When submit fail, `_total_tasks` is equal to the number of tasks submitted. |
227 | | std::atomic<int> _total_tasks = 0; |
228 | | |
229 | | std::unique_ptr<RuntimeProfile> _fragment_level_profile; |
230 | | bool _is_report_success = false; |
231 | | |
232 | | std::unique_ptr<RuntimeState> _runtime_state; |
233 | | |
234 | | std::shared_ptr<QueryContext> _query_ctx; |
235 | | |
236 | | MonotonicStopWatch _fragment_watcher; |
237 | | RuntimeProfile::Counter* _prepare_timer = nullptr; |
238 | | RuntimeProfile::Counter* _init_context_timer = nullptr; |
239 | | RuntimeProfile::Counter* _build_pipelines_timer = nullptr; |
240 | | RuntimeProfile::Counter* _plan_local_exchanger_timer = nullptr; |
241 | | RuntimeProfile::Counter* _prepare_all_pipelines_timer = nullptr; |
242 | | RuntimeProfile::Counter* _build_tasks_timer = nullptr; |
243 | | |
244 | | std::function<void(RuntimeState*, Status*)> _call_back; |
245 | | std::atomic_bool _is_fragment_instance_closed = false; |
246 | | |
247 | | // If this is set to false, and '_is_report_success' is false as well, |
248 | | // This executor will not report status to FE on being cancelled. |
249 | | bool _is_report_on_cancel; |
250 | | |
251 | | // 0 indicates reporting is in progress or not required |
252 | | std::atomic_bool _disable_period_report = true; |
253 | | std::atomic_uint64_t _previous_report_time = 0; |
254 | | |
255 | | DescriptorTbl* _desc_tbl = nullptr; |
256 | | int _num_instances = 1; |
257 | | |
258 | | int _timeout = -1; |
259 | | bool _use_serial_source = false; |
260 | | |
261 | | OperatorPtr _root_op = nullptr; |
262 | | // |
263 | | /** |
264 | | * Matrix stores tasks with local runtime states. |
265 | | * This is a [n * m] matrix. n is parallelism of pipeline engine and m is the number of pipelines. |
266 | | * |
267 | | * 2-D matrix: |
268 | | * +-------------------------+------------+-------+ |
269 | | * | | Pipeline 0 | Pipeline 1 | ... | |
270 | | * +------------+------------+------------+-------+ |
271 | | * | Instance 0 | task 0-0 | task 0-1 | ... | |
272 | | * +------------+------------+------------+-------+ |
273 | | * | Instance 1 | task 1-0 | task 1-1 | ... | |
274 | | * +------------+------------+------------+-------+ |
275 | | * | ... | |
276 | | * +--------------------------------------+-------+ |
277 | | */ |
278 | | std::vector< |
279 | | std::vector<std::pair<std::shared_ptr<PipelineTask>, std::unique_ptr<RuntimeState>>>> |
280 | | _tasks; |
281 | | |
282 | | // TODO: remove the _sink and _multi_cast_stream_sink_senders to set both |
283 | | // of it in pipeline task not the fragment_context |
284 | | #ifdef __clang__ |
285 | | #pragma clang diagnostic push |
286 | | #pragma clang diagnostic ignored "-Wshadow-field" |
287 | | #endif |
288 | | DataSinkOperatorPtr _sink = nullptr; |
289 | | #ifdef __clang__ |
290 | | #pragma clang diagnostic pop |
291 | | #endif |
292 | | |
293 | | // `_dag` manage dependencies between pipelines by pipeline ID. the indices will be blocked by members |
294 | | std::map<PipelineId, std::vector<PipelineId>> _dag; |
295 | | |
296 | | // We use preorder traversal to create an operator tree. When we meet a join node, we should |
297 | | // build probe operator and build operator in separate pipelines. To do this, we should build |
298 | | // ProbeSide first, and use `_pipelines_to_build` to store which pipeline the build operator |
299 | | // is in, so we can build BuildSide once we complete probe side. |
300 | | struct pipeline_parent_map { |
301 | | std::map<int, std::vector<PipelinePtr>> _build_side_pipelines; |
302 | 0 | void push(int parent_node_id, PipelinePtr pipeline) { |
303 | 0 | if (!_build_side_pipelines.contains(parent_node_id)) { |
304 | 0 | _build_side_pipelines.insert({parent_node_id, {pipeline}}); |
305 | 0 | } else { |
306 | 0 | _build_side_pipelines[parent_node_id].push_back(pipeline); |
307 | 0 | } |
308 | 0 | } |
309 | 0 | void pop(PipelinePtr& cur_pipe, int parent_node_id, int child_idx) { |
310 | 0 | if (!_build_side_pipelines.contains(parent_node_id)) { |
311 | 0 | return; |
312 | 0 | } |
313 | 0 | DCHECK(_build_side_pipelines.contains(parent_node_id)); |
314 | 0 | auto& child_pipeline = _build_side_pipelines[parent_node_id]; |
315 | 0 | DCHECK(child_idx < child_pipeline.size()); |
316 | 0 | cur_pipe = child_pipeline[child_idx]; |
317 | 0 | } |
318 | 0 | void clear() { _build_side_pipelines.clear(); } |
319 | | } _pipeline_parent_map; |
320 | | |
321 | | std::mutex _state_map_lock; |
322 | | |
323 | | int _operator_id = 0; |
324 | | int _sink_operator_id = 0; |
325 | | /** |
326 | | * Some states are shared by tasks in different pipeline task (e.g. local exchange , broadcast join). |
327 | | * |
328 | | * local exchange sink 0 -> -> local exchange source 0 |
329 | | * LocalExchangeSharedState |
330 | | * local exchange sink 1 -> -> local exchange source 1 |
331 | | * |
332 | | * hash join build sink 0 -> -> hash join build source 0 |
333 | | * HashJoinSharedState |
334 | | * hash join build sink 1 -> -> hash join build source 1 |
335 | | * |
336 | | * So we should keep states here. |
337 | | */ |
338 | | std::map<int, |
339 | | std::pair<std::shared_ptr<BasicSharedState>, std::vector<std::shared_ptr<Dependency>>>> |
340 | | _op_id_to_shared_state; |
341 | | |
342 | | std::map<PipelineId, Pipeline*> _pip_id_to_pipeline; |
343 | | std::vector<std::unique_ptr<RuntimeFilterMgr>> _runtime_filter_mgr_map; |
344 | | |
345 | | //Here are two types of runtime states: |
346 | | // - _runtime state is at the Fragment level. |
347 | | // - _task_runtime_states is at the task level, unique to each task. |
348 | | |
349 | | std::vector<TUniqueId> _fragment_instance_ids; |
350 | | |
351 | | // Total instance num running on all BEs |
352 | | int _total_instances = -1; |
353 | | |
354 | | TPipelineFragmentParams _params; |
355 | | int32_t _parallel_instances = 0; |
356 | | |
357 | | std::atomic<bool> _need_notify_close = false; |
358 | | // Holds the brpc ClosureGuard for async wait-close during recursive CTE rerun. |
359 | | // When the PFC finishes closing and is destroyed, the shared_ptr destructor fires |
360 | | // the ClosureGuard, which completes the brpc response to the RecCTESourceOperatorX. |
361 | | // Only written by listen_wait_close() from a single rerun_fragment RPC thread. |
362 | | std::shared_ptr<brpc::ClosureGuard> _wait_close_guard = nullptr; |
363 | | |
364 | | // The recursion round number for recursive CTE fragments. |
365 | | // Incremented each time the fragment is rebuilt via rerun_fragment(rebuild). |
366 | | // Used to stamp runtime filter RPCs so stale messages from old rounds are discarded. |
367 | | uint32_t _rec_cte_stage = 0; |
368 | | }; |
369 | | } // namespace doris |