Coverage Report

Created: 2026-03-27 23:51

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