Coverage Report

Created: 2026-04-10 06:24

next uncovered line (L), next uncovered region (R), next uncovered branch (B)
be/src/exec/pipeline/pipeline_fragment_context.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/pipeline_fragment_context.h"
19
20
#include <gen_cpp/DataSinks_types.h>
21
#include <gen_cpp/PaloInternalService_types.h>
22
#include <gen_cpp/PlanNodes_types.h>
23
#include <pthread.h>
24
25
#include <algorithm>
26
#include <cstdlib>
27
// IWYU pragma: no_include <bits/chrono.h>
28
#include <fmt/format.h>
29
30
#include <chrono> // IWYU pragma: keep
31
#include <map>
32
#include <memory>
33
#include <ostream>
34
#include <utility>
35
36
#include "cloud/config.h"
37
#include "common/cast_set.h"
38
#include "common/config.h"
39
#include "common/exception.h"
40
#include "common/logging.h"
41
#include "common/status.h"
42
#include "exec/exchange/local_exchange_sink_operator.h"
43
#include "exec/exchange/local_exchange_source_operator.h"
44
#include "exec/exchange/local_exchanger.h"
45
#include "exec/exchange/vdata_stream_mgr.h"
46
#include "exec/operator/aggregation_sink_operator.h"
47
#include "exec/operator/aggregation_source_operator.h"
48
#include "exec/operator/analytic_sink_operator.h"
49
#include "exec/operator/analytic_source_operator.h"
50
#include "exec/operator/assert_num_rows_operator.h"
51
#include "exec/operator/blackhole_sink_operator.h"
52
#include "exec/operator/cache_sink_operator.h"
53
#include "exec/operator/cache_source_operator.h"
54
#include "exec/operator/datagen_operator.h"
55
#include "exec/operator/dict_sink_operator.h"
56
#include "exec/operator/distinct_streaming_aggregation_operator.h"
57
#include "exec/operator/empty_set_operator.h"
58
#include "exec/operator/es_scan_operator.h"
59
#include "exec/operator/exchange_sink_operator.h"
60
#include "exec/operator/exchange_source_operator.h"
61
#include "exec/operator/file_scan_operator.h"
62
#include "exec/operator/group_commit_block_sink_operator.h"
63
#include "exec/operator/group_commit_scan_operator.h"
64
#include "exec/operator/hashjoin_build_sink.h"
65
#include "exec/operator/hashjoin_probe_operator.h"
66
#include "exec/operator/hive_table_sink_operator.h"
67
#include "exec/operator/iceberg_delete_sink_operator.h"
68
#include "exec/operator/iceberg_merge_sink_operator.h"
69
#include "exec/operator/iceberg_table_sink_operator.h"
70
#include "exec/operator/jdbc_scan_operator.h"
71
#include "exec/operator/jdbc_table_sink_operator.h"
72
#include "exec/operator/local_merge_sort_source_operator.h"
73
#include "exec/operator/materialization_opertor.h"
74
#include "exec/operator/maxcompute_table_sink_operator.h"
75
#include "exec/operator/memory_scratch_sink_operator.h"
76
#include "exec/operator/meta_scan_operator.h"
77
#include "exec/operator/multi_cast_data_stream_sink.h"
78
#include "exec/operator/multi_cast_data_stream_source.h"
79
#include "exec/operator/nested_loop_join_build_operator.h"
80
#include "exec/operator/nested_loop_join_probe_operator.h"
81
#include "exec/operator/olap_scan_operator.h"
82
#include "exec/operator/olap_table_sink_operator.h"
83
#include "exec/operator/olap_table_sink_v2_operator.h"
84
#include "exec/operator/partition_sort_sink_operator.h"
85
#include "exec/operator/partition_sort_source_operator.h"
86
#include "exec/operator/partitioned_aggregation_sink_operator.h"
87
#include "exec/operator/partitioned_aggregation_source_operator.h"
88
#include "exec/operator/partitioned_hash_join_probe_operator.h"
89
#include "exec/operator/partitioned_hash_join_sink_operator.h"
90
#include "exec/operator/rec_cte_anchor_sink_operator.h"
91
#include "exec/operator/rec_cte_scan_operator.h"
92
#include "exec/operator/rec_cte_sink_operator.h"
93
#include "exec/operator/rec_cte_source_operator.h"
94
#include "exec/operator/repeat_operator.h"
95
#include "exec/operator/result_file_sink_operator.h"
96
#include "exec/operator/result_sink_operator.h"
97
#include "exec/operator/schema_scan_operator.h"
98
#include "exec/operator/select_operator.h"
99
#include "exec/operator/set_probe_sink_operator.h"
100
#include "exec/operator/set_sink_operator.h"
101
#include "exec/operator/set_source_operator.h"
102
#include "exec/operator/sort_sink_operator.h"
103
#include "exec/operator/sort_source_operator.h"
104
#include "exec/operator/spill_iceberg_table_sink_operator.h"
105
#include "exec/operator/spill_sort_sink_operator.h"
106
#include "exec/operator/spill_sort_source_operator.h"
107
#include "exec/operator/streaming_aggregation_operator.h"
108
#include "exec/operator/table_function_operator.h"
109
#include "exec/operator/tvf_table_sink_operator.h"
110
#include "exec/operator/union_sink_operator.h"
111
#include "exec/operator/union_source_operator.h"
112
#include "exec/pipeline/dependency.h"
113
#include "exec/pipeline/pipeline_task.h"
114
#include "exec/pipeline/task_scheduler.h"
115
#include "exec/runtime_filter/runtime_filter_mgr.h"
116
#include "exec/sort/topn_sorter.h"
117
#include "exec/spill/spill_file.h"
118
#include "io/fs/stream_load_pipe.h"
119
#include "load/stream_load/new_load_stream_mgr.h"
120
#include "runtime/exec_env.h"
121
#include "runtime/fragment_mgr.h"
122
#include "runtime/result_buffer_mgr.h"
123
#include "runtime/runtime_state.h"
124
#include "runtime/thread_context.h"
125
#include "util/countdown_latch.h"
126
#include "util/debug_util.h"
127
#include "util/uid_util.h"
128
129
namespace doris {
130
#include "common/compile_check_begin.h"
131
PipelineFragmentContext::PipelineFragmentContext(
132
        TUniqueId query_id, const TPipelineFragmentParams& request,
133
        std::shared_ptr<QueryContext> query_ctx, ExecEnv* exec_env,
134
        const std::function<void(RuntimeState*, Status*)>& call_back,
135
        report_status_callback report_status_cb)
136
429k
        : _query_id(std::move(query_id)),
137
429k
          _fragment_id(request.fragment_id),
138
429k
          _exec_env(exec_env),
139
429k
          _query_ctx(std::move(query_ctx)),
140
429k
          _call_back(call_back),
141
429k
          _is_report_on_cancel(true),
142
429k
          _report_status_cb(std::move(report_status_cb)),
143
429k
          _params(request),
144
429k
          _parallel_instances(_params.__isset.parallel_instances ? _params.parallel_instances : 0),
145
429k
          _need_notify_close(request.__isset.need_notify_close ? request.need_notify_close
146
429k
                                                               : false) {
147
429k
    _fragment_watcher.start();
148
429k
}
149
150
429k
PipelineFragmentContext::~PipelineFragmentContext() {
151
429k
    LOG_INFO("PipelineFragmentContext::~PipelineFragmentContext")
152
429k
            .tag("query_id", print_id(_query_id))
153
429k
            .tag("fragment_id", _fragment_id);
154
429k
    _release_resource();
155
429k
    {
156
        // The memory released by the query end is recorded in the query mem tracker.
157
429k
        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_ctx->query_mem_tracker());
158
429k
        _runtime_state.reset();
159
429k
        _query_ctx.reset();
160
429k
    }
161
429k
}
162
163
113
bool PipelineFragmentContext::is_timeout(timespec now) const {
164
113
    if (_timeout <= 0) {
165
0
        return false;
166
0
    }
167
113
    return _fragment_watcher.elapsed_time_seconds(now) > _timeout;
168
113
}
169
170
// notify_close() transitions the PFC from "waiting for external close notification" to
171
// "self-managed close". For recursive CTE fragments, the old PFC is kept alive until
172
// the rerun_fragment(wait_for_destroy) RPC calls this to trigger shutdown.
173
// Returns true if all tasks have already closed (i.e., the PFC can be safely destroyed).
174
9.52k
bool PipelineFragmentContext::notify_close() {
175
9.52k
    bool all_closed = false;
176
9.52k
    bool need_remove = false;
177
9.52k
    {
178
9.52k
        std::lock_guard<std::mutex> l(_task_mutex);
179
9.52k
        if (_closed_tasks >= _total_tasks) {
180
3.45k
            if (_need_notify_close) {
181
                // Fragment was cancelled and waiting for notify to close.
182
                // Record that we need to remove from fragment mgr, but do it
183
                // after releasing _task_mutex to avoid ABBA deadlock with
184
                // dump_pipeline_tasks() (which acquires _pipeline_map lock
185
                // first, then _task_mutex via debug_string()).
186
3.38k
                need_remove = true;
187
3.38k
            }
188
3.45k
            all_closed = true;
189
3.45k
        }
190
        // make fragment release by self after cancel
191
9.52k
        _need_notify_close = false;
192
9.52k
    }
193
9.52k
    if (need_remove) {
194
3.38k
        _exec_env->fragment_mgr()->remove_pipeline_context({_query_id, _fragment_id});
195
3.38k
    }
196
9.52k
    return all_closed;
197
9.52k
}
198
199
// Must not add lock in this method. Because it will call query ctx cancel. And
200
// QueryCtx cancel will call fragment ctx cancel. And Also Fragment ctx's running
201
// Method like exchange sink buffer will call query ctx cancel. If we add lock here
202
// There maybe dead lock.
203
6.04k
void PipelineFragmentContext::cancel(const Status reason) {
204
6.04k
    LOG_INFO("PipelineFragmentContext::cancel")
205
6.04k
            .tag("query_id", print_id(_query_id))
206
6.04k
            .tag("fragment_id", _fragment_id)
207
6.04k
            .tag("reason", reason.to_string());
208
6.04k
    if (notify_close()) {
209
80
        return;
210
80
    }
211
    // Timeout is a special error code, we need print current stack to debug timeout issue.
212
5.96k
    if (reason.is<ErrorCode::TIMEOUT>()) {
213
8
        auto dbg_str = fmt::format("PipelineFragmentContext is cancelled due to timeout:\n{}",
214
8
                                   debug_string());
215
8
        LOG_LONG_STRING(WARNING, dbg_str);
216
8
    }
217
218
    // `ILLEGAL_STATE` means queries this fragment belongs to was not found in FE (maybe finished)
219
5.96k
    if (reason.is<ErrorCode::ILLEGAL_STATE>()) {
220
0
        LOG_WARNING("PipelineFragmentContext is cancelled due to illegal state : {}",
221
0
                    debug_string());
222
0
    }
223
224
5.96k
    if (reason.is<ErrorCode::MEM_LIMIT_EXCEEDED>() || reason.is<ErrorCode::MEM_ALLOC_FAILED>()) {
225
0
        print_profile("cancel pipeline, reason: " + reason.to_string());
226
0
    }
227
228
5.96k
    if (auto error_url = get_load_error_url(); !error_url.empty()) {
229
24
        _query_ctx->set_load_error_url(error_url);
230
24
    }
231
232
5.96k
    if (auto first_error_msg = get_first_error_msg(); !first_error_msg.empty()) {
233
24
        _query_ctx->set_first_error_msg(first_error_msg);
234
24
    }
235
236
5.96k
    _query_ctx->cancel(reason, _fragment_id);
237
5.96k
    if (reason.is<ErrorCode::LIMIT_REACH>()) {
238
479
        _is_report_on_cancel = false;
239
5.48k
    } else {
240
20.7k
        for (auto& id : _fragment_instance_ids) {
241
20.7k
            LOG(WARNING) << "PipelineFragmentContext cancel instance: " << print_id(id);
242
20.7k
        }
243
5.48k
    }
244
    // Get pipe from new load stream manager and send cancel to it or the fragment may hang to wait read from pipe
245
    // For stream load the fragment's query_id == load id, it is set in FE.
246
5.96k
    auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id);
247
5.96k
    if (stream_load_ctx != nullptr) {
248
32
        stream_load_ctx->pipe->cancel(reason.to_string());
249
        // Set error URL here because after pipe is cancelled, stream load execution may return early.
250
        // We need to set the error URL at this point to ensure error information is properly
251
        // propagated to the client.
252
32
        stream_load_ctx->error_url = get_load_error_url();
253
32
        stream_load_ctx->first_error_msg = get_first_error_msg();
254
32
    }
255
256
21.9k
    for (auto& tasks : _tasks) {
257
50.9k
        for (auto& task : tasks) {
258
50.9k
            task.first->unblock_all_dependencies();
259
50.9k
        }
260
21.9k
    }
261
5.96k
}
262
263
687k
PipelinePtr PipelineFragmentContext::add_pipeline(PipelinePtr parent, int idx) {
264
687k
    PipelineId id = _next_pipeline_id++;
265
687k
    auto pipeline = std::make_shared<Pipeline>(
266
687k
            id, parent ? std::min(parent->num_tasks(), _num_instances) : _num_instances,
267
687k
            parent ? parent->num_tasks() : _num_instances);
268
687k
    if (idx >= 0) {
269
131k
        _pipelines.insert(_pipelines.begin() + idx, pipeline);
270
555k
    } else {
271
555k
        _pipelines.emplace_back(pipeline);
272
555k
    }
273
687k
    if (parent) {
274
252k
        parent->set_children(pipeline);
275
252k
    }
276
687k
    return pipeline;
277
687k
}
278
279
428k
Status PipelineFragmentContext::_build_and_prepare_full_pipeline(ThreadPool* thread_pool) {
280
428k
    {
281
428k
        SCOPED_TIMER(_build_pipelines_timer);
282
        // 2. Build pipelines with operators in this fragment.
283
428k
        auto root_pipeline = add_pipeline();
284
428k
        RETURN_IF_ERROR(_build_pipelines(_runtime_state->obj_pool(), *_query_ctx->desc_tbl,
285
428k
                                         &_root_op, root_pipeline));
286
287
        // 3. Create sink operator
288
428k
        if (!_params.fragment.__isset.output_sink) {
289
0
            return Status::InternalError("No output sink in this fragment!");
290
0
        }
291
428k
        RETURN_IF_ERROR(_create_data_sink(_runtime_state->obj_pool(), _params.fragment.output_sink,
292
428k
                                          _params.fragment.output_exprs, _params,
293
428k
                                          root_pipeline->output_row_desc(), _runtime_state.get(),
294
428k
                                          *_desc_tbl, root_pipeline->id()));
295
428k
        RETURN_IF_ERROR(_sink->init(_params.fragment.output_sink));
296
428k
        RETURN_IF_ERROR(root_pipeline->set_sink(_sink));
297
298
555k
        for (PipelinePtr& pipeline : _pipelines) {
299
18.4E
            DCHECK(pipeline->sink() != nullptr) << pipeline->operators().size();
300
555k
            RETURN_IF_ERROR(pipeline->sink()->set_child(pipeline->operators().back()));
301
555k
        }
302
428k
    }
303
    // 4. Build local exchanger
304
428k
    if (_runtime_state->enable_local_shuffle()) {
305
426k
        SCOPED_TIMER(_plan_local_exchanger_timer);
306
426k
        RETURN_IF_ERROR(_plan_local_exchange(_params.num_buckets,
307
426k
                                             _params.bucket_seq_to_instance_idx,
308
426k
                                             _params.shuffle_idx_to_instance_idx));
309
426k
    }
310
311
    // 5. Initialize global states in pipelines.
312
687k
    for (PipelinePtr& pipeline : _pipelines) {
313
687k
        SCOPED_TIMER(_prepare_all_pipelines_timer);
314
687k
        pipeline->children().clear();
315
687k
        RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get()));
316
687k
    }
317
318
427k
    {
319
427k
        SCOPED_TIMER(_build_tasks_timer);
320
        // 6. Build pipeline tasks and initialize local state.
321
427k
        RETURN_IF_ERROR(_build_pipeline_tasks(thread_pool));
322
427k
    }
323
324
427k
    return Status::OK();
325
427k
}
326
327
429k
Status PipelineFragmentContext::prepare(ThreadPool* thread_pool) {
328
429k
    if (_prepared) {
329
0
        return Status::InternalError("Already prepared");
330
0
    }
331
429k
    if (_params.__isset.query_options && _params.query_options.__isset.execution_timeout) {
332
429k
        _timeout = _params.query_options.execution_timeout;
333
429k
    }
334
335
429k
    _fragment_level_profile = std::make_unique<RuntimeProfile>("PipelineContext");
336
429k
    _prepare_timer = ADD_TIMER(_fragment_level_profile, "PrepareTime");
337
429k
    SCOPED_TIMER(_prepare_timer);
338
429k
    _build_pipelines_timer = ADD_TIMER(_fragment_level_profile, "BuildPipelinesTime");
339
429k
    _init_context_timer = ADD_TIMER(_fragment_level_profile, "InitContextTime");
340
429k
    _plan_local_exchanger_timer = ADD_TIMER(_fragment_level_profile, "PlanLocalLocalExchangerTime");
341
429k
    _build_tasks_timer = ADD_TIMER(_fragment_level_profile, "BuildTasksTime");
342
429k
    _prepare_all_pipelines_timer = ADD_TIMER(_fragment_level_profile, "PrepareAllPipelinesTime");
343
429k
    {
344
429k
        SCOPED_TIMER(_init_context_timer);
345
429k
        cast_set(_num_instances, _params.local_params.size());
346
429k
        _total_instances =
347
429k
                _params.__isset.total_instances ? _params.total_instances : _num_instances;
348
349
429k
        auto* fragment_context = this;
350
351
429k
        if (_params.query_options.__isset.is_report_success) {
352
428k
            fragment_context->set_is_report_success(_params.query_options.is_report_success);
353
428k
        }
354
355
        // 1. Set up the global runtime state.
356
429k
        _runtime_state = RuntimeState::create_unique(
357
429k
                _params.query_id, _params.fragment_id, _params.query_options,
358
429k
                _query_ctx->query_globals, _exec_env, _query_ctx.get());
359
429k
        _runtime_state->set_task_execution_context(shared_from_this());
360
429k
        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_runtime_state->query_mem_tracker());
361
429k
        if (_params.__isset.backend_id) {
362
426k
            _runtime_state->set_backend_id(_params.backend_id);
363
426k
        }
364
429k
        if (_params.__isset.import_label) {
365
236
            _runtime_state->set_import_label(_params.import_label);
366
236
        }
367
429k
        if (_params.__isset.db_name) {
368
188
            _runtime_state->set_db_name(_params.db_name);
369
188
        }
370
429k
        if (_params.__isset.load_job_id) {
371
0
            _runtime_state->set_load_job_id(_params.load_job_id);
372
0
        }
373
374
429k
        if (_params.is_simplified_param) {
375
143k
            _desc_tbl = _query_ctx->desc_tbl;
376
285k
        } else {
377
285k
            DCHECK(_params.__isset.desc_tbl);
378
285k
            RETURN_IF_ERROR(DescriptorTbl::create(_runtime_state->obj_pool(), _params.desc_tbl,
379
285k
                                                  &_desc_tbl));
380
285k
        }
381
429k
        _runtime_state->set_desc_tbl(_desc_tbl);
382
429k
        _runtime_state->set_num_per_fragment_instances(_params.num_senders);
383
429k
        _runtime_state->set_load_stream_per_node(_params.load_stream_per_node);
384
429k
        _runtime_state->set_total_load_streams(_params.total_load_streams);
385
429k
        _runtime_state->set_num_local_sink(_params.num_local_sink);
386
387
        // init fragment_instance_ids
388
429k
        const auto target_size = _params.local_params.size();
389
429k
        _fragment_instance_ids.resize(target_size);
390
1.67M
        for (size_t i = 0; i < _params.local_params.size(); i++) {
391
1.24M
            auto fragment_instance_id = _params.local_params[i].fragment_instance_id;
392
1.24M
            _fragment_instance_ids[i] = fragment_instance_id;
393
1.24M
        }
394
429k
    }
395
396
429k
    RETURN_IF_ERROR(_build_and_prepare_full_pipeline(thread_pool));
397
398
427k
    _init_next_report_time();
399
400
427k
    _prepared = true;
401
427k
    return Status::OK();
402
429k
}
403
404
Status PipelineFragmentContext::_build_pipeline_tasks_for_instance(
405
        int instance_idx,
406
1.24M
        const std::vector<std::shared_ptr<RuntimeProfile>>& pipeline_id_to_profile) {
407
1.24M
    const auto& local_params = _params.local_params[instance_idx];
408
1.24M
    auto fragment_instance_id = local_params.fragment_instance_id;
409
1.24M
    auto runtime_filter_mgr = std::make_unique<RuntimeFilterMgr>(false);
410
1.24M
    std::map<PipelineId, PipelineTask*> pipeline_id_to_task;
411
1.24M
    auto get_shared_state = [&](PipelinePtr pipeline)
412
1.24M
            -> std::map<int, std::pair<std::shared_ptr<BasicSharedState>,
413
2.13M
                                       std::vector<std::shared_ptr<Dependency>>>> {
414
2.13M
        std::map<int, std::pair<std::shared_ptr<BasicSharedState>,
415
2.13M
                                std::vector<std::shared_ptr<Dependency>>>>
416
2.13M
                shared_state_map;
417
2.76M
        for (auto& op : pipeline->operators()) {
418
2.76M
            auto source_id = op->operator_id();
419
2.76M
            if (auto iter = _op_id_to_shared_state.find(source_id);
420
2.76M
                iter != _op_id_to_shared_state.end()) {
421
863k
                shared_state_map.insert({source_id, iter->second});
422
863k
            }
423
2.76M
        }
424
2.13M
        for (auto sink_to_source_id : pipeline->sink()->dests_id()) {
425
2.13M
            if (auto iter = _op_id_to_shared_state.find(sink_to_source_id);
426
2.13M
                iter != _op_id_to_shared_state.end()) {
427
374k
                shared_state_map.insert({sink_to_source_id, iter->second});
428
374k
            }
429
2.13M
        }
430
2.13M
        return shared_state_map;
431
2.13M
    };
432
433
3.86M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
434
2.62M
        auto& pipeline = _pipelines[pip_idx];
435
2.62M
        if (pipeline->num_tasks() > 1 || instance_idx == 0) {
436
2.13M
            auto task_runtime_state = RuntimeState::create_unique(
437
2.13M
                    local_params.fragment_instance_id, _params.query_id, _params.fragment_id,
438
2.13M
                    _params.query_options, _query_ctx->query_globals, _exec_env, _query_ctx.get());
439
2.13M
            {
440
                // Initialize runtime state for this task
441
2.13M
                task_runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker());
442
443
2.13M
                task_runtime_state->set_task_execution_context(shared_from_this());
444
2.13M
                task_runtime_state->set_be_number(local_params.backend_num);
445
446
2.13M
                if (_params.__isset.backend_id) {
447
2.13M
                    task_runtime_state->set_backend_id(_params.backend_id);
448
2.13M
                }
449
2.13M
                if (_params.__isset.import_label) {
450
237
                    task_runtime_state->set_import_label(_params.import_label);
451
237
                }
452
2.13M
                if (_params.__isset.db_name) {
453
189
                    task_runtime_state->set_db_name(_params.db_name);
454
189
                }
455
2.13M
                if (_params.__isset.load_job_id) {
456
0
                    task_runtime_state->set_load_job_id(_params.load_job_id);
457
0
                }
458
2.13M
                if (_params.__isset.wal_id) {
459
112
                    task_runtime_state->set_wal_id(_params.wal_id);
460
112
                }
461
2.13M
                if (_params.__isset.content_length) {
462
31
                    task_runtime_state->set_content_length(_params.content_length);
463
31
                }
464
465
2.13M
                task_runtime_state->set_desc_tbl(_desc_tbl);
466
2.13M
                task_runtime_state->set_per_fragment_instance_idx(local_params.sender_id);
467
2.13M
                task_runtime_state->set_num_per_fragment_instances(_params.num_senders);
468
2.13M
                task_runtime_state->resize_op_id_to_local_state(max_operator_id());
469
2.13M
                task_runtime_state->set_max_operator_id(max_operator_id());
470
2.13M
                task_runtime_state->set_load_stream_per_node(_params.load_stream_per_node);
471
2.13M
                task_runtime_state->set_total_load_streams(_params.total_load_streams);
472
2.13M
                task_runtime_state->set_num_local_sink(_params.num_local_sink);
473
474
2.13M
                task_runtime_state->set_runtime_filter_mgr(runtime_filter_mgr.get());
475
2.13M
            }
476
2.13M
            auto cur_task_id = _total_tasks++;
477
2.13M
            task_runtime_state->set_task_id(cur_task_id);
478
2.13M
            task_runtime_state->set_task_num(pipeline->num_tasks());
479
2.13M
            auto task = std::make_shared<PipelineTask>(
480
2.13M
                    pipeline, cur_task_id, task_runtime_state.get(),
481
2.13M
                    std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this()),
482
2.13M
                    pipeline_id_to_profile[pip_idx].get(), get_shared_state(pipeline),
483
2.13M
                    instance_idx);
484
2.13M
            pipeline->incr_created_tasks(instance_idx, task.get());
485
2.13M
            pipeline_id_to_task.insert({pipeline->id(), task.get()});
486
2.13M
            _tasks[instance_idx].emplace_back(
487
2.13M
                    std::pair<std::shared_ptr<PipelineTask>, std::unique_ptr<RuntimeState>> {
488
2.13M
                            std::move(task), std::move(task_runtime_state)});
489
2.13M
        }
490
2.62M
    }
491
492
    /**
493
         * Build DAG for pipeline tasks.
494
         * For example, we have
495
         *
496
         *   ExchangeSink (Pipeline1)     JoinBuildSink (Pipeline2)
497
         *            \                      /
498
         *          JoinProbeOperator1 (Pipeline1)    JoinBuildSink (Pipeline3)
499
         *                 \                          /
500
         *               JoinProbeOperator2 (Pipeline1)
501
         *
502
         * In this fragment, we have three pipelines and pipeline 1 depends on pipeline 2 and pipeline 3.
503
         * To build this DAG, `_dag` manage dependencies between pipelines by pipeline ID and
504
         * `pipeline_id_to_task` is used to find the task by a unique pipeline ID.
505
         *
506
         * Finally, we have two upstream dependencies in Pipeline1 corresponding to JoinProbeOperator1
507
         * and JoinProbeOperator2.
508
         */
509
2.62M
    for (auto& _pipeline : _pipelines) {
510
2.62M
        if (pipeline_id_to_task.contains(_pipeline->id())) {
511
2.13M
            auto* task = pipeline_id_to_task[_pipeline->id()];
512
2.13M
            DCHECK(task != nullptr);
513
514
            // If this task has upstream dependency, then inject it into this task.
515
2.13M
            if (_dag.contains(_pipeline->id())) {
516
1.37M
                auto& deps = _dag[_pipeline->id()];
517
2.22M
                for (auto& dep : deps) {
518
2.22M
                    if (pipeline_id_to_task.contains(dep)) {
519
1.24M
                        auto ss = pipeline_id_to_task[dep]->get_sink_shared_state();
520
1.24M
                        if (ss) {
521
507k
                            task->inject_shared_state(ss);
522
737k
                        } else {
523
737k
                            pipeline_id_to_task[dep]->inject_shared_state(
524
737k
                                    task->get_source_shared_state());
525
737k
                        }
526
1.24M
                    }
527
2.22M
                }
528
1.37M
            }
529
2.13M
        }
530
2.62M
    }
531
3.86M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
532
2.62M
        if (pipeline_id_to_task.contains(_pipelines[pip_idx]->id())) {
533
2.13M
            auto* task = pipeline_id_to_task[_pipelines[pip_idx]->id()];
534
2.13M
            DCHECK(pipeline_id_to_profile[pip_idx]);
535
2.13M
            std::vector<TScanRangeParams> scan_ranges;
536
2.13M
            auto node_id = _pipelines[pip_idx]->operators().front()->node_id();
537
2.13M
            if (local_params.per_node_scan_ranges.contains(node_id)) {
538
341k
                scan_ranges = local_params.per_node_scan_ranges.find(node_id)->second;
539
341k
            }
540
2.13M
            RETURN_IF_ERROR_OR_CATCH_EXCEPTION(task->prepare(scan_ranges, local_params.sender_id,
541
2.13M
                                                             _params.fragment.output_sink));
542
2.13M
        }
543
2.62M
    }
544
1.24M
    {
545
1.24M
        std::lock_guard<std::mutex> l(_state_map_lock);
546
1.24M
        _runtime_filter_mgr_map[instance_idx] = std::move(runtime_filter_mgr);
547
1.24M
    }
548
1.24M
    return Status::OK();
549
1.24M
}
550
551
428k
Status PipelineFragmentContext::_build_pipeline_tasks(ThreadPool* thread_pool) {
552
428k
    _total_tasks = 0;
553
428k
    _closed_tasks = 0;
554
428k
    const auto target_size = _params.local_params.size();
555
428k
    _tasks.resize(target_size);
556
428k
    _runtime_filter_mgr_map.resize(target_size);
557
1.11M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
558
686k
        _pip_id_to_pipeline[_pipelines[pip_idx]->id()] = _pipelines[pip_idx].get();
559
686k
    }
560
428k
    auto pipeline_id_to_profile = _runtime_state->build_pipeline_profile(_pipelines.size());
561
562
428k
    if (target_size > 1 &&
563
428k
        (_runtime_state->query_options().__isset.parallel_prepare_threshold &&
564
148k
         target_size > _runtime_state->query_options().parallel_prepare_threshold)) {
565
        // If instances parallelism is big enough ( > parallel_prepare_threshold), we will prepare all tasks by multi-threads
566
28.5k
        std::vector<Status> prepare_status(target_size);
567
28.5k
        int submitted_tasks = 0;
568
28.5k
        Status submit_status;
569
28.5k
        CountDownLatch latch((int)target_size);
570
333k
        for (int i = 0; i < target_size; i++) {
571
304k
            submit_status = thread_pool->submit_func([&, i]() {
572
304k
                SCOPED_ATTACH_TASK(_query_ctx.get());
573
304k
                prepare_status[i] = _build_pipeline_tasks_for_instance(i, pipeline_id_to_profile);
574
304k
                latch.count_down();
575
304k
            });
576
304k
            if (LIKELY(submit_status.ok())) {
577
304k
                submitted_tasks++;
578
18.4E
            } else {
579
18.4E
                break;
580
18.4E
            }
581
304k
        }
582
28.5k
        latch.arrive_and_wait(target_size - submitted_tasks);
583
28.5k
        if (UNLIKELY(!submit_status.ok())) {
584
0
            return submit_status;
585
0
        }
586
333k
        for (int i = 0; i < submitted_tasks; i++) {
587
304k
            if (!prepare_status[i].ok()) {
588
0
                return prepare_status[i];
589
0
            }
590
304k
        }
591
399k
    } else {
592
1.33M
        for (int i = 0; i < target_size; i++) {
593
937k
            RETURN_IF_ERROR(_build_pipeline_tasks_for_instance(i, pipeline_id_to_profile));
594
937k
        }
595
399k
    }
596
428k
    _pipeline_parent_map.clear();
597
428k
    _op_id_to_shared_state.clear();
598
599
428k
    return Status::OK();
600
428k
}
601
602
427k
void PipelineFragmentContext::_init_next_report_time() {
603
427k
    auto interval_s = config::pipeline_status_report_interval;
604
427k
    if (_is_report_success && interval_s > 0 && _timeout > interval_s) {
605
40.3k
        VLOG_FILE << "enable period report: fragment id=" << _fragment_id;
606
40.3k
        uint64_t report_fragment_offset = (uint64_t)(rand() % interval_s) * NANOS_PER_SEC;
607
        // We don't want to wait longer than it takes to run the entire fragment.
608
40.3k
        _previous_report_time =
609
40.3k
                MonotonicNanos() + report_fragment_offset - (uint64_t)(interval_s)*NANOS_PER_SEC;
610
40.3k
        _disable_period_report = false;
611
40.3k
    }
612
427k
}
613
614
4.82k
void PipelineFragmentContext::refresh_next_report_time() {
615
4.82k
    auto disable = _disable_period_report.load(std::memory_order_acquire);
616
4.82k
    DCHECK(disable == true);
617
4.82k
    _previous_report_time.store(MonotonicNanos(), std::memory_order_release);
618
4.82k
    _disable_period_report.compare_exchange_strong(disable, false);
619
4.82k
}
620
621
7.64M
void PipelineFragmentContext::trigger_report_if_necessary() {
622
7.64M
    if (!_is_report_success) {
623
7.12M
        return;
624
7.12M
    }
625
523k
    auto disable = _disable_period_report.load(std::memory_order_acquire);
626
523k
    if (disable) {
627
8.36k
        return;
628
8.36k
    }
629
515k
    int32_t interval_s = config::pipeline_status_report_interval;
630
515k
    if (interval_s <= 0) {
631
0
        LOG(WARNING) << "config::status_report_interval is equal to or less than zero, do not "
632
0
                        "trigger "
633
0
                        "report.";
634
0
    }
635
515k
    uint64_t next_report_time = _previous_report_time.load(std::memory_order_acquire) +
636
515k
                                (uint64_t)(interval_s)*NANOS_PER_SEC;
637
515k
    if (MonotonicNanos() > next_report_time) {
638
4.83k
        if (!_disable_period_report.compare_exchange_strong(disable, true,
639
4.83k
                                                            std::memory_order_acq_rel)) {
640
10
            return;
641
10
        }
642
4.82k
        if (VLOG_FILE_IS_ON) {
643
0
            VLOG_FILE << "Reporting "
644
0
                      << "profile for query_id " << print_id(_query_id)
645
0
                      << ", fragment id: " << _fragment_id;
646
647
0
            std::stringstream ss;
648
0
            _runtime_state->runtime_profile()->compute_time_in_profile();
649
0
            _runtime_state->runtime_profile()->pretty_print(&ss);
650
0
            if (_runtime_state->load_channel_profile()) {
651
0
                _runtime_state->load_channel_profile()->pretty_print(&ss);
652
0
            }
653
654
0
            VLOG_FILE << "Query " << print_id(get_query_id()) << " fragment " << get_fragment_id()
655
0
                      << " profile:\n"
656
0
                      << ss.str();
657
0
        }
658
4.82k
        auto st = send_report(false);
659
4.82k
        if (!st.ok()) {
660
0
            disable = true;
661
0
            _disable_period_report.compare_exchange_strong(disable, false,
662
0
                                                           std::memory_order_acq_rel);
663
0
        }
664
4.82k
    }
665
515k
}
666
667
Status PipelineFragmentContext::_build_pipelines(ObjectPool* pool, const DescriptorTbl& descs,
668
428k
                                                 OperatorPtr* root, PipelinePtr cur_pipe) {
669
428k
    if (_params.fragment.plan.nodes.empty()) {
670
0
        throw Exception(ErrorCode::INTERNAL_ERROR, "Invalid plan which has no plan node!");
671
0
    }
672
673
428k
    int node_idx = 0;
674
675
428k
    RETURN_IF_ERROR(_create_tree_helper(pool, _params.fragment.plan.nodes, descs, nullptr,
676
428k
                                        &node_idx, root, cur_pipe, 0, false, false));
677
678
428k
    if (node_idx + 1 != _params.fragment.plan.nodes.size()) {
679
0
        return Status::InternalError(
680
0
                "Plan tree only partially reconstructed. Not all thrift nodes were used.");
681
0
    }
682
428k
    return Status::OK();
683
428k
}
684
685
Status PipelineFragmentContext::_create_tree_helper(
686
        ObjectPool* pool, const std::vector<TPlanNode>& tnodes, const DescriptorTbl& descs,
687
        OperatorPtr parent, int* node_idx, OperatorPtr* root, PipelinePtr& cur_pipe, int child_idx,
688
660k
        const bool followed_by_shuffled_operator, const bool require_bucket_distribution) {
689
    // propagate error case
690
660k
    if (*node_idx >= tnodes.size()) {
691
0
        return Status::InternalError(
692
0
                "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}",
693
0
                *node_idx, tnodes.size());
694
0
    }
695
660k
    const TPlanNode& tnode = tnodes[*node_idx];
696
697
660k
    int num_children = tnodes[*node_idx].num_children;
698
660k
    bool current_followed_by_shuffled_operator = followed_by_shuffled_operator;
699
660k
    bool current_require_bucket_distribution = require_bucket_distribution;
700
    // TODO: Create CacheOperator is confused now
701
660k
    OperatorPtr op = nullptr;
702
660k
    OperatorPtr cache_op = nullptr;
703
660k
    RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], descs, op, cur_pipe,
704
660k
                                     parent == nullptr ? -1 : parent->node_id(), child_idx,
705
660k
                                     followed_by_shuffled_operator,
706
660k
                                     current_require_bucket_distribution, cache_op));
707
    // Initialization must be done here. For example, group by expressions in agg will be used to
708
    // decide if a local shuffle should be planed, so it must be initialized here.
709
660k
    RETURN_IF_ERROR(op->init(tnode, _runtime_state.get()));
710
    // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr));
711
660k
    if (parent != nullptr) {
712
        // add to parent's child(s)
713
232k
        RETURN_IF_ERROR(parent->set_child(cache_op ? cache_op : op));
714
428k
    } else {
715
428k
        *root = op;
716
428k
    }
717
    /**
718
     * `ExchangeType::HASH_SHUFFLE` should be used if an operator is followed by a shuffled operator (shuffled hash join, union operator followed by co-located operators).
719
     *
720
     * For plan:
721
     * LocalExchange(id=0) -> Aggregation(id=1) -> ShuffledHashJoin(id=2)
722
     *                           Exchange(id=3) -> ShuffledHashJoinBuild(id=2)
723
     * We must ensure data distribution of `LocalExchange(id=0)` is same as Exchange(id=3).
724
     *
725
     * If an operator's is followed by a local exchange without shuffle (e.g. passthrough), a
726
     * shuffled local exchanger will be used before join so it is not followed by shuffle join.
727
     */
728
660k
    auto required_data_distribution =
729
660k
            cur_pipe->operators().empty()
730
660k
                    ? cur_pipe->sink()->required_data_distribution(_runtime_state.get())
731
660k
                    : op->required_data_distribution(_runtime_state.get());
732
660k
    current_followed_by_shuffled_operator =
733
660k
            ((followed_by_shuffled_operator ||
734
660k
              (cur_pipe->operators().empty() ? cur_pipe->sink()->is_shuffled_operator()
735
600k
                                             : op->is_shuffled_operator())) &&
736
660k
             Pipeline::is_hash_exchange(required_data_distribution.distribution_type)) ||
737
660k
            (followed_by_shuffled_operator &&
738
549k
             required_data_distribution.distribution_type == ExchangeType::NOOP);
739
740
660k
    current_require_bucket_distribution =
741
660k
            ((require_bucket_distribution ||
742
660k
              (cur_pipe->operators().empty() ? cur_pipe->sink()->is_colocated_operator()
743
605k
                                             : op->is_colocated_operator())) &&
744
660k
             Pipeline::is_hash_exchange(required_data_distribution.distribution_type)) ||
745
660k
            (require_bucket_distribution &&
746
554k
             required_data_distribution.distribution_type == ExchangeType::NOOP);
747
748
660k
    if (num_children == 0) {
749
442k
        _use_serial_source = op->is_serial_operator();
750
442k
    }
751
    // rely on that tnodes is preorder of the plan
752
893k
    for (int i = 0; i < num_children; i++) {
753
232k
        ++*node_idx;
754
232k
        RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, descs, op, node_idx, nullptr, cur_pipe, i,
755
232k
                                            current_followed_by_shuffled_operator,
756
232k
                                            current_require_bucket_distribution));
757
758
        // we are expecting a child, but have used all nodes
759
        // this means we have been given a bad tree and must fail
760
232k
        if (*node_idx >= tnodes.size()) {
761
0
            return Status::InternalError(
762
0
                    "Failed to reconstruct plan tree from thrift. Node id: {}, number of "
763
0
                    "nodes: {}",
764
0
                    *node_idx, tnodes.size());
765
0
        }
766
232k
    }
767
768
660k
    return Status::OK();
769
660k
}
770
771
void PipelineFragmentContext::_inherit_pipeline_properties(
772
        const DataDistribution& data_distribution, PipelinePtr pipe_with_source,
773
131k
        PipelinePtr pipe_with_sink) {
774
131k
    pipe_with_sink->set_num_tasks(pipe_with_source->num_tasks());
775
131k
    pipe_with_source->set_num_tasks(_num_instances);
776
131k
    pipe_with_source->set_data_distribution(data_distribution);
777
131k
}
778
779
Status PipelineFragmentContext::_add_local_exchange_impl(
780
        int idx, ObjectPool* pool, PipelinePtr cur_pipe, PipelinePtr new_pip,
781
        DataDistribution data_distribution, bool* do_local_exchange, int num_buckets,
782
        const std::map<int, int>& bucket_seq_to_instance_idx,
783
131k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
784
131k
    auto& operators = cur_pipe->operators();
785
131k
    const auto downstream_pipeline_id = cur_pipe->id();
786
131k
    auto local_exchange_id = next_operator_id();
787
    // 1. Create a new pipeline with local exchange sink.
788
131k
    DataSinkOperatorPtr sink;
789
131k
    auto sink_id = next_sink_operator_id();
790
791
    /**
792
     * `bucket_seq_to_instance_idx` is empty if no scan operator is contained in this fragment.
793
     * So co-located operators(e.g. Agg, Analytic) should use `HASH_SHUFFLE` instead of `BUCKET_HASH_SHUFFLE`.
794
     */
795
131k
    const bool followed_by_shuffled_operator =
796
131k
            operators.size() > idx ? operators[idx]->followed_by_shuffled_operator()
797
131k
                                   : cur_pipe->sink()->followed_by_shuffled_operator();
798
131k
    const bool use_global_hash_shuffle = bucket_seq_to_instance_idx.empty() &&
799
131k
                                         !shuffle_idx_to_instance_idx.contains(-1) &&
800
131k
                                         followed_by_shuffled_operator && !_use_serial_source;
801
131k
    sink = std::make_shared<LocalExchangeSinkOperatorX>(
802
131k
            sink_id, local_exchange_id, use_global_hash_shuffle ? _total_instances : _num_instances,
803
131k
            data_distribution.partition_exprs, bucket_seq_to_instance_idx);
804
131k
    if (bucket_seq_to_instance_idx.empty() &&
805
131k
        data_distribution.distribution_type == ExchangeType::BUCKET_HASH_SHUFFLE) {
806
6
        data_distribution.distribution_type = ExchangeType::HASH_SHUFFLE;
807
6
    }
808
131k
    RETURN_IF_ERROR(new_pip->set_sink(sink));
809
131k
    RETURN_IF_ERROR(new_pip->sink()->init(_runtime_state.get(), data_distribution.distribution_type,
810
131k
                                          num_buckets, use_global_hash_shuffle,
811
131k
                                          shuffle_idx_to_instance_idx));
812
813
    // 2. Create and initialize LocalExchangeSharedState.
814
131k
    std::shared_ptr<LocalExchangeSharedState> shared_state =
815
131k
            LocalExchangeSharedState::create_shared(_num_instances);
816
131k
    switch (data_distribution.distribution_type) {
817
27.5k
    case ExchangeType::HASH_SHUFFLE:
818
27.5k
        shared_state->exchanger = ShuffleExchanger::create_unique(
819
27.5k
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances,
820
27.5k
                use_global_hash_shuffle ? _total_instances : _num_instances,
821
27.5k
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
822
27.5k
                        ? cast_set<int>(
823
27.5k
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
824
18.4E
                        : 0);
825
27.5k
        break;
826
508
    case ExchangeType::BUCKET_HASH_SHUFFLE:
827
508
        shared_state->exchanger = BucketShuffleExchanger::create_unique(
828
508
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, num_buckets,
829
508
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
830
508
                        ? cast_set<int>(
831
508
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
832
508
                        : 0);
833
508
        break;
834
99.7k
    case ExchangeType::PASSTHROUGH:
835
99.7k
        shared_state->exchanger = PassthroughExchanger::create_unique(
836
99.7k
                cur_pipe->num_tasks(), _num_instances,
837
99.7k
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
838
99.7k
                        ? cast_set<int>(
839
99.7k
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
840
18.4E
                        : 0);
841
99.7k
        break;
842
405
    case ExchangeType::BROADCAST:
843
405
        shared_state->exchanger = BroadcastExchanger::create_unique(
844
405
                cur_pipe->num_tasks(), _num_instances,
845
405
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
846
405
                        ? cast_set<int>(
847
405
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
848
405
                        : 0);
849
405
        break;
850
2.74k
    case ExchangeType::PASS_TO_ONE:
851
2.74k
        if (_runtime_state->enable_share_hash_table_for_broadcast_join()) {
852
            // If shared hash table is enabled for BJ, hash table will be built by only one task
853
1.65k
            shared_state->exchanger = PassToOneExchanger::create_unique(
854
1.65k
                    cur_pipe->num_tasks(), _num_instances,
855
1.65k
                    _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
856
1.65k
                            ? cast_set<int>(_runtime_state->query_options()
857
1.65k
                                                    .local_exchange_free_blocks_limit)
858
1.65k
                            : 0);
859
1.65k
        } else {
860
1.08k
            shared_state->exchanger = BroadcastExchanger::create_unique(
861
1.08k
                    cur_pipe->num_tasks(), _num_instances,
862
1.08k
                    _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
863
1.08k
                            ? cast_set<int>(_runtime_state->query_options()
864
1.08k
                                                    .local_exchange_free_blocks_limit)
865
1.08k
                            : 0);
866
1.08k
        }
867
2.74k
        break;
868
773
    case ExchangeType::ADAPTIVE_PASSTHROUGH:
869
773
        shared_state->exchanger = AdaptivePassthroughExchanger::create_unique(
870
773
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances,
871
773
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
872
773
                        ? cast_set<int>(
873
773
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
874
773
                        : 0);
875
773
        break;
876
0
    default:
877
0
        return Status::InternalError("Unsupported local exchange type : " +
878
0
                                     std::to_string((int)data_distribution.distribution_type));
879
131k
    }
880
131k
    shared_state->create_source_dependencies(_num_instances, local_exchange_id, local_exchange_id,
881
131k
                                             "LOCAL_EXCHANGE_OPERATOR");
882
131k
    shared_state->create_sink_dependency(sink_id, local_exchange_id, "LOCAL_EXCHANGE_SINK");
883
131k
    _op_id_to_shared_state.insert({local_exchange_id, {shared_state, shared_state->sink_deps}});
884
885
    // 3. Set two pipelines' operator list. For example, split pipeline [Scan - AggSink] to
886
    // pipeline1 [Scan - LocalExchangeSink] and pipeline2 [LocalExchangeSource - AggSink].
887
888
    // 3.1 Initialize new pipeline's operator list.
889
131k
    std::copy(operators.begin(), operators.begin() + idx,
890
131k
              std::inserter(new_pip->operators(), new_pip->operators().end()));
891
892
    // 3.2 Erase unused operators in previous pipeline.
893
131k
    operators.erase(operators.begin(), operators.begin() + idx);
894
895
    // 4. Initialize LocalExchangeSource and insert it into this pipeline.
896
131k
    OperatorPtr source_op;
897
131k
    source_op = std::make_shared<LocalExchangeSourceOperatorX>(pool, local_exchange_id);
898
131k
    RETURN_IF_ERROR(source_op->set_child(new_pip->operators().back()));
899
131k
    RETURN_IF_ERROR(source_op->init(data_distribution.distribution_type));
900
131k
    if (!operators.empty()) {
901
49.6k
        RETURN_IF_ERROR(operators.front()->set_child(nullptr));
902
49.6k
        RETURN_IF_ERROR(operators.front()->set_child(source_op));
903
49.6k
    }
904
131k
    operators.insert(operators.begin(), source_op);
905
906
    // 5. Set children for two pipelines separately.
907
131k
    std::vector<std::shared_ptr<Pipeline>> new_children;
908
131k
    std::vector<PipelineId> edges_with_source;
909
150k
    for (auto child : cur_pipe->children()) {
910
150k
        bool found = false;
911
164k
        for (auto op : new_pip->operators()) {
912
164k
            if (child->sink()->node_id() == op->node_id()) {
913
12.8k
                new_pip->set_children(child);
914
12.8k
                found = true;
915
12.8k
            };
916
164k
        }
917
150k
        if (!found) {
918
137k
            new_children.push_back(child);
919
137k
            edges_with_source.push_back(child->id());
920
137k
        }
921
150k
    }
922
131k
    new_children.push_back(new_pip);
923
131k
    edges_with_source.push_back(new_pip->id());
924
925
    // 6. Set DAG for new pipelines.
926
131k
    if (!new_pip->children().empty()) {
927
7.32k
        std::vector<PipelineId> edges_with_sink;
928
12.8k
        for (auto child : new_pip->children()) {
929
12.8k
            edges_with_sink.push_back(child->id());
930
12.8k
        }
931
7.32k
        _dag.insert({new_pip->id(), edges_with_sink});
932
7.32k
    }
933
131k
    cur_pipe->set_children(new_children);
934
131k
    _dag[downstream_pipeline_id] = edges_with_source;
935
131k
    RETURN_IF_ERROR(new_pip->sink()->set_child(new_pip->operators().back()));
936
131k
    RETURN_IF_ERROR(cur_pipe->sink()->set_child(nullptr));
937
131k
    RETURN_IF_ERROR(cur_pipe->sink()->set_child(cur_pipe->operators().back()));
938
939
    // 7. Inherit properties from current pipeline.
940
131k
    _inherit_pipeline_properties(data_distribution, cur_pipe, new_pip);
941
131k
    return Status::OK();
942
131k
}
943
944
Status PipelineFragmentContext::_add_local_exchange(
945
        int pip_idx, int idx, int node_id, ObjectPool* pool, PipelinePtr cur_pipe,
946
        DataDistribution data_distribution, bool* do_local_exchange, int num_buckets,
947
        const std::map<int, int>& bucket_seq_to_instance_idx,
948
189k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
949
189k
    if (_num_instances <= 1 || cur_pipe->num_tasks_of_parent() <= 1) {
950
33.8k
        return Status::OK();
951
33.8k
    }
952
953
156k
    if (!cur_pipe->need_to_local_exchange(data_distribution, idx)) {
954
52.1k
        return Status::OK();
955
52.1k
    }
956
103k
    *do_local_exchange = true;
957
958
103k
    auto& operators = cur_pipe->operators();
959
103k
    auto total_op_num = operators.size();
960
103k
    auto new_pip = add_pipeline(cur_pipe, pip_idx + 1);
961
103k
    RETURN_IF_ERROR(_add_local_exchange_impl(
962
103k
            idx, pool, cur_pipe, new_pip, data_distribution, do_local_exchange, num_buckets,
963
103k
            bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx));
964
965
103k
    CHECK(total_op_num + 1 == cur_pipe->operators().size() + new_pip->operators().size())
966
142
            << "total_op_num: " << total_op_num
967
142
            << " cur_pipe->operators().size(): " << cur_pipe->operators().size()
968
142
            << " new_pip->operators().size(): " << new_pip->operators().size();
969
970
    // There are some local shuffles with relatively heavy operations on the sink.
971
    // If the local sink concurrency is 1 and the local source concurrency is n, the sink becomes a bottleneck.
972
    // Therefore, local passthrough is used to increase the concurrency of the sink.
973
    // op -> local sink(1) -> local source (n)
974
    // op -> local passthrough(1) -> local passthrough(n) ->  local sink(n) -> local source (n)
975
103k
    if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 &&
976
103k
        Pipeline::heavy_operations_on_the_sink(data_distribution.distribution_type)) {
977
27.8k
        RETURN_IF_ERROR(_add_local_exchange_impl(
978
27.8k
                cast_set<int>(new_pip->operators().size()), pool, new_pip,
979
27.8k
                add_pipeline(new_pip, pip_idx + 2), DataDistribution(ExchangeType::PASSTHROUGH),
980
27.8k
                do_local_exchange, num_buckets, bucket_seq_to_instance_idx,
981
27.8k
                shuffle_idx_to_instance_idx));
982
27.8k
    }
983
103k
    return Status::OK();
984
103k
}
985
986
Status PipelineFragmentContext::_plan_local_exchange(
987
        int num_buckets, const std::map<int, int>& bucket_seq_to_instance_idx,
988
426k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
989
978k
    for (int pip_idx = cast_set<int>(_pipelines.size()) - 1; pip_idx >= 0; pip_idx--) {
990
552k
        _pipelines[pip_idx]->init_data_distribution(_runtime_state.get());
991
        // Set property if child pipeline is not join operator's child.
992
552k
        if (!_pipelines[pip_idx]->children().empty()) {
993
121k
            for (auto& child : _pipelines[pip_idx]->children()) {
994
121k
                if (child->sink()->node_id() ==
995
121k
                    _pipelines[pip_idx]->operators().front()->node_id()) {
996
108k
                    _pipelines[pip_idx]->set_data_distribution(child->data_distribution());
997
108k
                }
998
121k
            }
999
116k
        }
1000
1001
        // if 'num_buckets == 0' means the fragment is colocated by exchange node not the
1002
        // scan node. so here use `_num_instance` to replace the `num_buckets` to prevent dividing 0
1003
        // still keep colocate plan after local shuffle
1004
552k
        RETURN_IF_ERROR(_plan_local_exchange(num_buckets, pip_idx, _pipelines[pip_idx],
1005
552k
                                             bucket_seq_to_instance_idx,
1006
552k
                                             shuffle_idx_to_instance_idx));
1007
552k
    }
1008
426k
    return Status::OK();
1009
426k
}
1010
1011
Status PipelineFragmentContext::_plan_local_exchange(
1012
        int num_buckets, int pip_idx, PipelinePtr pip,
1013
        const std::map<int, int>& bucket_seq_to_instance_idx,
1014
552k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
1015
552k
    int idx = 1;
1016
552k
    bool do_local_exchange = false;
1017
602k
    do {
1018
602k
        auto& ops = pip->operators();
1019
602k
        do_local_exchange = false;
1020
        // Plan local exchange for each operator.
1021
664k
        for (; idx < ops.size();) {
1022
111k
            if (ops[idx]->required_data_distribution(_runtime_state.get()).need_local_exchange()) {
1023
103k
                RETURN_IF_ERROR(_add_local_exchange(
1024
103k
                        pip_idx, idx, ops[idx]->node_id(), _runtime_state->obj_pool(), pip,
1025
103k
                        ops[idx]->required_data_distribution(_runtime_state.get()),
1026
103k
                        &do_local_exchange, num_buckets, bucket_seq_to_instance_idx,
1027
103k
                        shuffle_idx_to_instance_idx));
1028
103k
            }
1029
111k
            if (do_local_exchange) {
1030
                // If local exchange is needed for current operator, we will split this pipeline to
1031
                // two pipelines by local exchange sink/source. And then we need to process remaining
1032
                // operators in this pipeline so we set idx to 2 (0 is local exchange source and 1
1033
                // is current operator was already processed) and continue to plan local exchange.
1034
49.6k
                idx = 2;
1035
49.6k
                break;
1036
49.6k
            }
1037
62.2k
            idx++;
1038
62.2k
        }
1039
602k
    } while (do_local_exchange);
1040
552k
    if (pip->sink()->required_data_distribution(_runtime_state.get()).need_local_exchange()) {
1041
86.3k
        RETURN_IF_ERROR(_add_local_exchange(
1042
86.3k
                pip_idx, idx, pip->sink()->node_id(), _runtime_state->obj_pool(), pip,
1043
86.3k
                pip->sink()->required_data_distribution(_runtime_state.get()), &do_local_exchange,
1044
86.3k
                num_buckets, bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx));
1045
86.3k
    }
1046
552k
    return Status::OK();
1047
552k
}
1048
1049
Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink,
1050
                                                  const std::vector<TExpr>& output_exprs,
1051
                                                  const TPipelineFragmentParams& params,
1052
                                                  const RowDescriptor& row_desc,
1053
                                                  RuntimeState* state, DescriptorTbl& desc_tbl,
1054
428k
                                                  PipelineId cur_pipeline_id) {
1055
428k
    switch (thrift_sink.type) {
1056
141k
    case TDataSinkType::DATA_STREAM_SINK: {
1057
141k
        if (!thrift_sink.__isset.stream_sink) {
1058
0
            return Status::InternalError("Missing data stream sink.");
1059
0
        }
1060
141k
        _sink = std::make_shared<ExchangeSinkOperatorX>(
1061
141k
                state, row_desc, next_sink_operator_id(), thrift_sink.stream_sink,
1062
141k
                params.destinations, _fragment_instance_ids);
1063
141k
        break;
1064
141k
    }
1065
250k
    case TDataSinkType::RESULT_SINK: {
1066
250k
        if (!thrift_sink.__isset.result_sink) {
1067
0
            return Status::InternalError("Missing data buffer sink.");
1068
0
        }
1069
1070
250k
        _sink = std::make_shared<ResultSinkOperatorX>(next_sink_operator_id(), row_desc,
1071
250k
                                                      output_exprs, thrift_sink.result_sink);
1072
250k
        break;
1073
250k
    }
1074
105
    case TDataSinkType::DICTIONARY_SINK: {
1075
105
        if (!thrift_sink.__isset.dictionary_sink) {
1076
0
            return Status::InternalError("Missing dict sink.");
1077
0
        }
1078
1079
105
        _sink = std::make_shared<DictSinkOperatorX>(next_sink_operator_id(), row_desc, output_exprs,
1080
105
                                                    thrift_sink.dictionary_sink);
1081
105
        break;
1082
105
    }
1083
0
    case TDataSinkType::GROUP_COMMIT_OLAP_TABLE_SINK:
1084
30.0k
    case TDataSinkType::OLAP_TABLE_SINK: {
1085
30.0k
        if (state->query_options().enable_memtable_on_sink_node &&
1086
30.0k
            !_has_inverted_index_v1_or_partial_update(thrift_sink.olap_table_sink) &&
1087
30.0k
            !config::is_cloud_mode()) {
1088
2.08k
            _sink = std::make_shared<OlapTableSinkV2OperatorX>(pool, next_sink_operator_id(),
1089
2.08k
                                                               row_desc, output_exprs);
1090
27.9k
        } else {
1091
27.9k
            _sink = std::make_shared<OlapTableSinkOperatorX>(pool, next_sink_operator_id(),
1092
27.9k
                                                             row_desc, output_exprs);
1093
27.9k
        }
1094
30.0k
        break;
1095
0
    }
1096
164
    case TDataSinkType::GROUP_COMMIT_BLOCK_SINK: {
1097
164
        DCHECK(thrift_sink.__isset.olap_table_sink);
1098
164
        DCHECK(state->get_query_ctx() != nullptr);
1099
164
        state->get_query_ctx()->query_mem_tracker()->is_group_commit_load = true;
1100
164
        _sink = std::make_shared<GroupCommitBlockSinkOperatorX>(next_sink_operator_id(), row_desc,
1101
164
                                                                output_exprs);
1102
164
        break;
1103
0
    }
1104
1.46k
    case TDataSinkType::HIVE_TABLE_SINK: {
1105
1.46k
        if (!thrift_sink.__isset.hive_table_sink) {
1106
0
            return Status::InternalError("Missing hive table sink.");
1107
0
        }
1108
1.46k
        _sink = std::make_shared<HiveTableSinkOperatorX>(pool, next_sink_operator_id(), row_desc,
1109
1.46k
                                                         output_exprs);
1110
1.46k
        break;
1111
1.46k
    }
1112
1.72k
    case TDataSinkType::ICEBERG_TABLE_SINK: {
1113
1.72k
        if (!thrift_sink.__isset.iceberg_table_sink) {
1114
0
            return Status::InternalError("Missing iceberg table sink.");
1115
0
        }
1116
1.72k
        if (thrift_sink.iceberg_table_sink.__isset.sort_info) {
1117
0
            _sink = std::make_shared<SpillIcebergTableSinkOperatorX>(pool, next_sink_operator_id(),
1118
0
                                                                     row_desc, output_exprs);
1119
1.72k
        } else {
1120
1.72k
            _sink = std::make_shared<IcebergTableSinkOperatorX>(pool, next_sink_operator_id(),
1121
1.72k
                                                                row_desc, output_exprs);
1122
1.72k
        }
1123
1.72k
        break;
1124
1.72k
    }
1125
20
    case TDataSinkType::ICEBERG_DELETE_SINK: {
1126
20
        if (!thrift_sink.__isset.iceberg_delete_sink) {
1127
0
            return Status::InternalError("Missing iceberg delete sink.");
1128
0
        }
1129
20
        _sink = std::make_shared<IcebergDeleteSinkOperatorX>(pool, next_sink_operator_id(),
1130
20
                                                             row_desc, output_exprs);
1131
20
        break;
1132
20
    }
1133
80
    case TDataSinkType::ICEBERG_MERGE_SINK: {
1134
80
        if (!thrift_sink.__isset.iceberg_merge_sink) {
1135
0
            return Status::InternalError("Missing iceberg merge sink.");
1136
0
        }
1137
80
        _sink = std::make_shared<IcebergMergeSinkOperatorX>(pool, next_sink_operator_id(), row_desc,
1138
80
                                                            output_exprs);
1139
80
        break;
1140
80
    }
1141
0
    case TDataSinkType::MAXCOMPUTE_TABLE_SINK: {
1142
0
        if (!thrift_sink.__isset.max_compute_table_sink) {
1143
0
            return Status::InternalError("Missing max compute table sink.");
1144
0
        }
1145
0
        _sink = std::make_shared<MCTableSinkOperatorX>(pool, next_sink_operator_id(), row_desc,
1146
0
                                                       output_exprs);
1147
0
        break;
1148
0
    }
1149
80
    case TDataSinkType::JDBC_TABLE_SINK: {
1150
80
        if (!thrift_sink.__isset.jdbc_table_sink) {
1151
0
            return Status::InternalError("Missing data jdbc sink.");
1152
0
        }
1153
80
        if (config::enable_java_support) {
1154
80
            _sink = std::make_shared<JdbcTableSinkOperatorX>(row_desc, next_sink_operator_id(),
1155
80
                                                             output_exprs);
1156
80
        } else {
1157
0
            return Status::InternalError(
1158
0
                    "Jdbc table sink is not enabled, you can change be config "
1159
0
                    "enable_java_support to true and restart be.");
1160
0
        }
1161
80
        break;
1162
80
    }
1163
80
    case TDataSinkType::MEMORY_SCRATCH_SINK: {
1164
3
        if (!thrift_sink.__isset.memory_scratch_sink) {
1165
0
            return Status::InternalError("Missing data buffer sink.");
1166
0
        }
1167
1168
3
        _sink = std::make_shared<MemoryScratchSinkOperatorX>(row_desc, next_sink_operator_id(),
1169
3
                                                             output_exprs);
1170
3
        break;
1171
3
    }
1172
502
    case TDataSinkType::RESULT_FILE_SINK: {
1173
502
        if (!thrift_sink.__isset.result_file_sink) {
1174
0
            return Status::InternalError("Missing result file sink.");
1175
0
        }
1176
1177
        // Result file sink is not the top sink
1178
502
        if (params.__isset.destinations && !params.destinations.empty()) {
1179
0
            _sink = std::make_shared<ResultFileSinkOperatorX>(
1180
0
                    next_sink_operator_id(), row_desc, thrift_sink.result_file_sink,
1181
0
                    params.destinations, output_exprs, desc_tbl);
1182
502
        } else {
1183
502
            _sink = std::make_shared<ResultFileSinkOperatorX>(next_sink_operator_id(), row_desc,
1184
502
                                                              output_exprs);
1185
502
        }
1186
502
        break;
1187
502
    }
1188
1.91k
    case TDataSinkType::MULTI_CAST_DATA_STREAM_SINK: {
1189
1.91k
        DCHECK(thrift_sink.__isset.multi_cast_stream_sink);
1190
1.91k
        DCHECK_GT(thrift_sink.multi_cast_stream_sink.sinks.size(), 0);
1191
1.91k
        auto sink_id = next_sink_operator_id();
1192
1.91k
        const int multi_cast_node_id = sink_id;
1193
1.91k
        auto sender_size = thrift_sink.multi_cast_stream_sink.sinks.size();
1194
        // one sink has multiple sources.
1195
1.91k
        std::vector<int> sources;
1196
7.43k
        for (int i = 0; i < sender_size; ++i) {
1197
5.52k
            auto source_id = next_operator_id();
1198
5.52k
            sources.push_back(source_id);
1199
5.52k
        }
1200
1201
1.91k
        _sink = std::make_shared<MultiCastDataStreamSinkOperatorX>(
1202
1.91k
                sink_id, multi_cast_node_id, sources, pool, thrift_sink.multi_cast_stream_sink);
1203
7.43k
        for (int i = 0; i < sender_size; ++i) {
1204
5.52k
            auto new_pipeline = add_pipeline();
1205
            // use to exchange sink
1206
5.52k
            RowDescriptor* exchange_row_desc = nullptr;
1207
5.52k
            {
1208
5.52k
                const auto& tmp_row_desc =
1209
5.52k
                        !thrift_sink.multi_cast_stream_sink.sinks[i].output_exprs.empty()
1210
5.52k
                                ? RowDescriptor(state->desc_tbl(),
1211
5.52k
                                                {thrift_sink.multi_cast_stream_sink.sinks[i]
1212
5.52k
                                                         .output_tuple_id})
1213
5.52k
                                : row_desc;
1214
5.52k
                exchange_row_desc = pool->add(new RowDescriptor(tmp_row_desc));
1215
5.52k
            }
1216
5.52k
            auto source_id = sources[i];
1217
5.52k
            OperatorPtr source_op;
1218
            // 1. create and set the source operator of multi_cast_data_stream_source for new pipeline
1219
5.52k
            source_op = std::make_shared<MultiCastDataStreamerSourceOperatorX>(
1220
5.52k
                    /*node_id*/ source_id, /*consumer_id*/ i, pool,
1221
5.52k
                    thrift_sink.multi_cast_stream_sink.sinks[i], row_desc,
1222
5.52k
                    /*operator_id=*/source_id);
1223
5.52k
            RETURN_IF_ERROR(new_pipeline->add_operator(
1224
5.52k
                    source_op, params.__isset.parallel_instances ? params.parallel_instances : 0));
1225
            // 2. create and set sink operator of data stream sender for new pipeline
1226
1227
5.52k
            DataSinkOperatorPtr sink_op;
1228
5.52k
            sink_op = std::make_shared<ExchangeSinkOperatorX>(
1229
5.52k
                    state, *exchange_row_desc, next_sink_operator_id(),
1230
5.52k
                    thrift_sink.multi_cast_stream_sink.sinks[i],
1231
5.52k
                    thrift_sink.multi_cast_stream_sink.destinations[i], _fragment_instance_ids);
1232
1233
5.52k
            RETURN_IF_ERROR(new_pipeline->set_sink(sink_op));
1234
5.52k
            {
1235
5.52k
                TDataSink* t = pool->add(new TDataSink());
1236
5.52k
                t->stream_sink = thrift_sink.multi_cast_stream_sink.sinks[i];
1237
5.52k
                RETURN_IF_ERROR(sink_op->init(*t));
1238
5.52k
            }
1239
1240
            // 3. set dependency dag
1241
5.52k
            _dag[new_pipeline->id()].push_back(cur_pipeline_id);
1242
5.52k
        }
1243
1.91k
        if (sources.empty()) {
1244
0
            return Status::InternalError("size of sources must be greater than 0");
1245
0
        }
1246
1.91k
        break;
1247
1.91k
    }
1248
1.91k
    case TDataSinkType::BLACKHOLE_SINK: {
1249
13
        if (!thrift_sink.__isset.blackhole_sink) {
1250
0
            return Status::InternalError("Missing blackhole sink.");
1251
0
        }
1252
1253
13
        _sink.reset(new BlackholeSinkOperatorX(next_sink_operator_id()));
1254
13
        break;
1255
13
    }
1256
156
    case TDataSinkType::TVF_TABLE_SINK: {
1257
156
        if (!thrift_sink.__isset.tvf_table_sink) {
1258
0
            return Status::InternalError("Missing TVF table sink.");
1259
0
        }
1260
156
        _sink = std::make_shared<TVFTableSinkOperatorX>(pool, next_sink_operator_id(), row_desc,
1261
156
                                                        output_exprs);
1262
156
        break;
1263
156
    }
1264
0
    default:
1265
0
        return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type);
1266
428k
    }
1267
428k
    return Status::OK();
1268
428k
}
1269
1270
// NOLINTBEGIN(readability-function-size)
1271
// NOLINTBEGIN(readability-function-cognitive-complexity)
1272
Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNode& tnode,
1273
                                                 const DescriptorTbl& descs, OperatorPtr& op,
1274
                                                 PipelinePtr& cur_pipe, int parent_idx,
1275
                                                 int child_idx,
1276
                                                 const bool followed_by_shuffled_operator,
1277
                                                 const bool require_bucket_distribution,
1278
661k
                                                 OperatorPtr& cache_op) {
1279
661k
    std::vector<DataSinkOperatorPtr> sink_ops;
1280
661k
    Defer defer = Defer([&]() {
1281
661k
        if (op) {
1282
661k
            op->update_operator(tnode, followed_by_shuffled_operator, require_bucket_distribution);
1283
661k
        }
1284
661k
        for (auto& s : sink_ops) {
1285
120k
            s->update_operator(tnode, followed_by_shuffled_operator, require_bucket_distribution);
1286
120k
        }
1287
661k
    });
1288
    // We directly construct the operator from Thrift because the given array is in the order of preorder traversal.
1289
    // Therefore, here we need to use a stack-like structure.
1290
661k
    _pipeline_parent_map.pop(cur_pipe, parent_idx, child_idx);
1291
661k
    std::stringstream error_msg;
1292
661k
    bool enable_query_cache = _params.fragment.__isset.query_cache_param;
1293
1294
661k
    bool fe_with_old_version = false;
1295
661k
    switch (tnode.node_type) {
1296
208k
    case TPlanNodeType::OLAP_SCAN_NODE: {
1297
208k
        op = std::make_shared<OlapScanOperatorX>(
1298
208k
                pool, tnode, next_operator_id(), descs, _num_instances,
1299
208k
                enable_query_cache ? _params.fragment.query_cache_param : TQueryCacheParam {});
1300
208k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1301
208k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1302
208k
        break;
1303
208k
    }
1304
77
    case TPlanNodeType::GROUP_COMMIT_SCAN_NODE: {
1305
77
        DCHECK(_query_ctx != nullptr);
1306
77
        _query_ctx->query_mem_tracker()->is_group_commit_load = true;
1307
77
        op = std::make_shared<GroupCommitOperatorX>(pool, tnode, next_operator_id(), descs,
1308
77
                                                    _num_instances);
1309
77
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1310
77
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1311
77
        break;
1312
77
    }
1313
0
    case TPlanNodeType::JDBC_SCAN_NODE: {
1314
0
        if (config::enable_java_support) {
1315
0
            op = std::make_shared<JDBCScanOperatorX>(pool, tnode, next_operator_id(), descs,
1316
0
                                                     _num_instances);
1317
0
            RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1318
0
        } else {
1319
0
            return Status::InternalError(
1320
0
                    "Jdbc scan node is disabled, you can change be config enable_java_support "
1321
0
                    "to true and restart be.");
1322
0
        }
1323
0
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1324
0
        break;
1325
0
    }
1326
23.0k
    case TPlanNodeType::FILE_SCAN_NODE: {
1327
23.0k
        op = std::make_shared<FileScanOperatorX>(pool, tnode, next_operator_id(), descs,
1328
23.0k
                                                 _num_instances);
1329
23.0k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1330
23.0k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1331
23.0k
        break;
1332
23.0k
    }
1333
0
    case TPlanNodeType::ES_SCAN_NODE:
1334
592
    case TPlanNodeType::ES_HTTP_SCAN_NODE: {
1335
592
        op = std::make_shared<EsScanOperatorX>(pool, tnode, next_operator_id(), descs,
1336
592
                                               _num_instances);
1337
592
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1338
592
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1339
592
        break;
1340
592
    }
1341
145k
    case TPlanNodeType::EXCHANGE_NODE: {
1342
145k
        int num_senders = _params.per_exch_num_senders.contains(tnode.node_id)
1343
145k
                                  ? _params.per_exch_num_senders.find(tnode.node_id)->second
1344
18.4E
                                  : 0;
1345
145k
        DCHECK_GT(num_senders, 0);
1346
145k
        op = std::make_shared<ExchangeSourceOperatorX>(pool, tnode, next_operator_id(), descs,
1347
145k
                                                       num_senders);
1348
145k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1349
145k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1350
145k
        break;
1351
145k
    }
1352
155k
    case TPlanNodeType::AGGREGATION_NODE: {
1353
155k
        if (tnode.agg_node.grouping_exprs.empty() &&
1354
155k
            descs.get_tuple_descriptor(tnode.agg_node.output_tuple_id)->slots().empty()) {
1355
0
            return Status::InternalError("Illegal aggregate node " + std::to_string(tnode.node_id) +
1356
0
                                         ": group by and output is empty");
1357
0
        }
1358
155k
        bool need_create_cache_op =
1359
155k
                enable_query_cache && tnode.node_id == _params.fragment.query_cache_param.node_id;
1360
155k
        auto create_query_cache_operator = [&](PipelinePtr& new_pipe) {
1361
10
            auto cache_node_id = _params.local_params[0].per_node_scan_ranges.begin()->first;
1362
10
            auto cache_source_id = next_operator_id();
1363
10
            op = std::make_shared<CacheSourceOperatorX>(pool, cache_node_id, cache_source_id,
1364
10
                                                        _params.fragment.query_cache_param);
1365
10
            RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1366
1367
10
            const auto downstream_pipeline_id = cur_pipe->id();
1368
10
            if (!_dag.contains(downstream_pipeline_id)) {
1369
10
                _dag.insert({downstream_pipeline_id, {}});
1370
10
            }
1371
10
            new_pipe = add_pipeline(cur_pipe);
1372
10
            _dag[downstream_pipeline_id].push_back(new_pipe->id());
1373
1374
10
            DataSinkOperatorPtr cache_sink(new CacheSinkOperatorX(
1375
10
                    next_sink_operator_id(), op->node_id(), op->operator_id()));
1376
10
            RETURN_IF_ERROR(new_pipe->set_sink(cache_sink));
1377
10
            return Status::OK();
1378
10
        };
1379
155k
        const bool group_by_limit_opt =
1380
155k
                tnode.agg_node.__isset.agg_sort_info_by_group_key && tnode.limit > 0;
1381
1382
        /// PartitionedAggSourceOperatorX does not support "group by limit opt(#29641)" yet.
1383
        /// If `group_by_limit_opt` is true, then it might not need to spill at all.
1384
155k
        const bool enable_spill = _runtime_state->enable_spill() &&
1385
155k
                                  !tnode.agg_node.grouping_exprs.empty() && !group_by_limit_opt;
1386
155k
        const bool is_streaming_agg = tnode.agg_node.__isset.use_streaming_preaggregation &&
1387
155k
                                      tnode.agg_node.use_streaming_preaggregation &&
1388
155k
                                      !tnode.agg_node.grouping_exprs.empty();
1389
        // TODO: distinct streaming agg does not support spill.
1390
155k
        const bool can_use_distinct_streaming_agg =
1391
155k
                (!enable_spill || is_streaming_agg) && tnode.agg_node.aggregate_functions.empty() &&
1392
155k
                !tnode.agg_node.__isset.agg_sort_info_by_group_key &&
1393
155k
                _params.query_options.__isset.enable_distinct_streaming_aggregation &&
1394
155k
                _params.query_options.enable_distinct_streaming_aggregation;
1395
1396
155k
        if (can_use_distinct_streaming_agg) {
1397
91.7k
            if (need_create_cache_op) {
1398
8
                PipelinePtr new_pipe;
1399
8
                RETURN_IF_ERROR(create_query_cache_operator(new_pipe));
1400
1401
8
                cache_op = op;
1402
8
                op = std::make_shared<DistinctStreamingAggOperatorX>(pool, next_operator_id(),
1403
8
                                                                     tnode, descs);
1404
8
                RETURN_IF_ERROR(new_pipe->add_operator(op, _parallel_instances));
1405
8
                RETURN_IF_ERROR(cur_pipe->operators().front()->set_child(op));
1406
8
                cur_pipe = new_pipe;
1407
91.7k
            } else {
1408
91.7k
                op = std::make_shared<DistinctStreamingAggOperatorX>(pool, next_operator_id(),
1409
91.7k
                                                                     tnode, descs);
1410
91.7k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1411
91.7k
            }
1412
91.7k
        } else if (is_streaming_agg) {
1413
3.16k
            if (need_create_cache_op) {
1414
0
                PipelinePtr new_pipe;
1415
0
                RETURN_IF_ERROR(create_query_cache_operator(new_pipe));
1416
0
                cache_op = op;
1417
0
                op = std::make_shared<StreamingAggOperatorX>(pool, next_operator_id(), tnode,
1418
0
                                                             descs);
1419
0
                RETURN_IF_ERROR(cur_pipe->operators().front()->set_child(op));
1420
0
                RETURN_IF_ERROR(new_pipe->add_operator(op, _parallel_instances));
1421
0
                cur_pipe = new_pipe;
1422
3.16k
            } else {
1423
3.16k
                op = std::make_shared<StreamingAggOperatorX>(pool, next_operator_id(), tnode,
1424
3.16k
                                                             descs);
1425
3.16k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1426
3.16k
            }
1427
60.3k
        } else {
1428
            // create new pipeline to add query cache operator
1429
60.3k
            PipelinePtr new_pipe;
1430
60.3k
            if (need_create_cache_op) {
1431
2
                RETURN_IF_ERROR(create_query_cache_operator(new_pipe));
1432
2
                cache_op = op;
1433
2
            }
1434
1435
60.3k
            if (enable_spill) {
1436
83
                op = std::make_shared<PartitionedAggSourceOperatorX>(pool, tnode,
1437
83
                                                                     next_operator_id(), descs);
1438
60.3k
            } else {
1439
60.3k
                op = std::make_shared<AggSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1440
60.3k
            }
1441
60.3k
            if (need_create_cache_op) {
1442
2
                RETURN_IF_ERROR(cur_pipe->operators().front()->set_child(op));
1443
2
                RETURN_IF_ERROR(new_pipe->add_operator(op, _parallel_instances));
1444
2
                cur_pipe = new_pipe;
1445
60.3k
            } else {
1446
60.3k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1447
60.3k
            }
1448
1449
60.3k
            const auto downstream_pipeline_id = cur_pipe->id();
1450
60.3k
            if (!_dag.contains(downstream_pipeline_id)) {
1451
58.4k
                _dag.insert({downstream_pipeline_id, {}});
1452
58.4k
            }
1453
60.3k
            cur_pipe = add_pipeline(cur_pipe);
1454
60.3k
            _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1455
1456
60.3k
            if (enable_spill) {
1457
83
                sink_ops.push_back(std::make_shared<PartitionedAggSinkOperatorX>(
1458
83
                        pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1459
60.3k
            } else {
1460
60.3k
                sink_ops.push_back(std::make_shared<AggSinkOperatorX>(
1461
60.3k
                        pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1462
60.3k
            }
1463
60.3k
            RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1464
60.3k
            RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1465
60.3k
        }
1466
155k
        break;
1467
155k
    }
1468
155k
    case TPlanNodeType::HASH_JOIN_NODE: {
1469
7.82k
        const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join &&
1470
7.82k
                                       tnode.hash_join_node.is_broadcast_join;
1471
7.82k
        const auto enable_spill = _runtime_state->enable_spill();
1472
7.82k
        if (enable_spill && !is_broadcast_join) {
1473
0
            auto tnode_ = tnode;
1474
0
            tnode_.runtime_filters.clear();
1475
0
            auto inner_probe_operator =
1476
0
                    std::make_shared<HashJoinProbeOperatorX>(pool, tnode_, 0, descs);
1477
1478
            // probe side inner sink operator is used to build hash table on probe side when data is spilled.
1479
            // So here use `tnode_` which has no runtime filters.
1480
0
            auto probe_side_inner_sink_operator =
1481
0
                    std::make_shared<HashJoinBuildSinkOperatorX>(pool, 0, 0, tnode_, descs);
1482
1483
0
            RETURN_IF_ERROR(inner_probe_operator->init(tnode_, _runtime_state.get()));
1484
0
            RETURN_IF_ERROR(probe_side_inner_sink_operator->init(tnode_, _runtime_state.get()));
1485
1486
0
            auto probe_operator = std::make_shared<PartitionedHashJoinProbeOperatorX>(
1487
0
                    pool, tnode_, next_operator_id(), descs);
1488
0
            probe_operator->set_inner_operators(probe_side_inner_sink_operator,
1489
0
                                                inner_probe_operator);
1490
0
            op = std::move(probe_operator);
1491
0
            RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1492
1493
0
            const auto downstream_pipeline_id = cur_pipe->id();
1494
0
            if (!_dag.contains(downstream_pipeline_id)) {
1495
0
                _dag.insert({downstream_pipeline_id, {}});
1496
0
            }
1497
0
            PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1498
0
            _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1499
1500
0
            auto inner_sink_operator =
1501
0
                    std::make_shared<HashJoinBuildSinkOperatorX>(pool, 0, 0, tnode, descs);
1502
0
            auto sink_operator = std::make_shared<PartitionedHashJoinSinkOperatorX>(
1503
0
                    pool, next_sink_operator_id(), op->operator_id(), tnode_, descs);
1504
0
            RETURN_IF_ERROR(inner_sink_operator->init(tnode, _runtime_state.get()));
1505
1506
0
            sink_operator->set_inner_operators(inner_sink_operator, inner_probe_operator);
1507
0
            sink_ops.push_back(std::move(sink_operator));
1508
0
            RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1509
0
            RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode_, _runtime_state.get()));
1510
1511
0
            _pipeline_parent_map.push(op->node_id(), cur_pipe);
1512
0
            _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1513
7.82k
        } else {
1514
7.82k
            op = std::make_shared<HashJoinProbeOperatorX>(pool, tnode, next_operator_id(), descs);
1515
7.82k
            RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1516
1517
7.82k
            const auto downstream_pipeline_id = cur_pipe->id();
1518
7.82k
            if (!_dag.contains(downstream_pipeline_id)) {
1519
7.01k
                _dag.insert({downstream_pipeline_id, {}});
1520
7.01k
            }
1521
7.82k
            PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1522
7.82k
            _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1523
1524
7.82k
            sink_ops.push_back(std::make_shared<HashJoinBuildSinkOperatorX>(
1525
7.82k
                    pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1526
7.82k
            RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1527
7.82k
            RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode, _runtime_state.get()));
1528
1529
7.82k
            _pipeline_parent_map.push(op->node_id(), cur_pipe);
1530
7.82k
            _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1531
7.82k
        }
1532
7.82k
        if (is_broadcast_join && _runtime_state->enable_share_hash_table_for_broadcast_join()) {
1533
2.93k
            std::shared_ptr<HashJoinSharedState> shared_state =
1534
2.93k
                    HashJoinSharedState::create_shared(_num_instances);
1535
21.9k
            for (int i = 0; i < _num_instances; i++) {
1536
19.0k
                auto sink_dep = std::make_shared<Dependency>(op->operator_id(), op->node_id(),
1537
19.0k
                                                             "HASH_JOIN_BUILD_DEPENDENCY");
1538
19.0k
                sink_dep->set_shared_state(shared_state.get());
1539
19.0k
                shared_state->sink_deps.push_back(sink_dep);
1540
19.0k
            }
1541
2.93k
            shared_state->create_source_dependencies(_num_instances, op->operator_id(),
1542
2.93k
                                                     op->node_id(), "HASH_JOIN_PROBE");
1543
2.93k
            _op_id_to_shared_state.insert(
1544
2.93k
                    {op->operator_id(), {shared_state, shared_state->sink_deps}});
1545
2.93k
        }
1546
7.82k
        break;
1547
7.82k
    }
1548
4.54k
    case TPlanNodeType::CROSS_JOIN_NODE: {
1549
4.54k
        op = std::make_shared<NestedLoopJoinProbeOperatorX>(pool, tnode, next_operator_id(), descs);
1550
4.54k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1551
1552
4.54k
        const auto downstream_pipeline_id = cur_pipe->id();
1553
4.54k
        if (!_dag.contains(downstream_pipeline_id)) {
1554
4.31k
            _dag.insert({downstream_pipeline_id, {}});
1555
4.31k
        }
1556
4.54k
        PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1557
4.54k
        _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1558
1559
4.54k
        sink_ops.push_back(std::make_shared<NestedLoopJoinBuildSinkOperatorX>(
1560
4.54k
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1561
4.54k
        RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1562
4.54k
        RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode, _runtime_state.get()));
1563
4.54k
        _pipeline_parent_map.push(op->node_id(), cur_pipe);
1564
4.54k
        _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1565
4.54k
        break;
1566
4.54k
    }
1567
52.4k
    case TPlanNodeType::UNION_NODE: {
1568
52.4k
        int child_count = tnode.num_children;
1569
52.4k
        op = std::make_shared<UnionSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1570
52.4k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1571
1572
52.4k
        const auto downstream_pipeline_id = cur_pipe->id();
1573
52.4k
        if (!_dag.contains(downstream_pipeline_id)) {
1574
52.2k
            _dag.insert({downstream_pipeline_id, {}});
1575
52.2k
        }
1576
54.1k
        for (int i = 0; i < child_count; i++) {
1577
1.67k
            PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1578
1.67k
            _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1579
1.67k
            sink_ops.push_back(std::make_shared<UnionSinkOperatorX>(
1580
1.67k
                    i, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1581
1.67k
            RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1582
1.67k
            RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode, _runtime_state.get()));
1583
            // preset children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build.
1584
1.67k
            _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1585
1.67k
        }
1586
52.4k
        break;
1587
52.4k
    }
1588
52.4k
    case TPlanNodeType::SORT_NODE: {
1589
44.1k
        const auto should_spill = _runtime_state->enable_spill() &&
1590
44.1k
                                  tnode.sort_node.algorithm == TSortAlgorithm::FULL_SORT;
1591
44.1k
        const bool use_local_merge =
1592
44.1k
                tnode.sort_node.__isset.use_local_merge && tnode.sort_node.use_local_merge;
1593
44.1k
        if (should_spill) {
1594
9
            op = std::make_shared<SpillSortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1595
44.1k
        } else if (use_local_merge) {
1596
42.0k
            op = std::make_shared<LocalMergeSortSourceOperatorX>(pool, tnode, next_operator_id(),
1597
42.0k
                                                                 descs);
1598
42.0k
        } else {
1599
2.13k
            op = std::make_shared<SortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1600
2.13k
        }
1601
44.1k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1602
1603
44.1k
        const auto downstream_pipeline_id = cur_pipe->id();
1604
44.1k
        if (!_dag.contains(downstream_pipeline_id)) {
1605
44.1k
            _dag.insert({downstream_pipeline_id, {}});
1606
44.1k
        }
1607
44.1k
        cur_pipe = add_pipeline(cur_pipe);
1608
44.1k
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1609
1610
44.1k
        if (should_spill) {
1611
9
            sink_ops.push_back(std::make_shared<SpillSortSinkOperatorX>(
1612
9
                    pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1613
44.1k
        } else {
1614
44.1k
            sink_ops.push_back(std::make_shared<SortSinkOperatorX>(
1615
44.1k
                    pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1616
44.1k
        }
1617
44.1k
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1618
44.1k
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1619
44.1k
        break;
1620
44.1k
    }
1621
44.1k
    case TPlanNodeType::PARTITION_SORT_NODE: {
1622
63
        op = std::make_shared<PartitionSortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1623
63
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1624
1625
63
        const auto downstream_pipeline_id = cur_pipe->id();
1626
63
        if (!_dag.contains(downstream_pipeline_id)) {
1627
63
            _dag.insert({downstream_pipeline_id, {}});
1628
63
        }
1629
63
        cur_pipe = add_pipeline(cur_pipe);
1630
63
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1631
1632
63
        sink_ops.push_back(std::make_shared<PartitionSortSinkOperatorX>(
1633
63
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1634
63
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1635
63
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1636
63
        break;
1637
63
    }
1638
1.47k
    case TPlanNodeType::ANALYTIC_EVAL_NODE: {
1639
1.47k
        op = std::make_shared<AnalyticSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1640
1.47k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1641
1642
1.47k
        const auto downstream_pipeline_id = cur_pipe->id();
1643
1.47k
        if (!_dag.contains(downstream_pipeline_id)) {
1644
1.46k
            _dag.insert({downstream_pipeline_id, {}});
1645
1.46k
        }
1646
1.47k
        cur_pipe = add_pipeline(cur_pipe);
1647
1.47k
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1648
1649
1.47k
        sink_ops.push_back(std::make_shared<AnalyticSinkOperatorX>(
1650
1.47k
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1651
1.47k
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1652
1.47k
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1653
1.47k
        break;
1654
1.47k
    }
1655
1.61k
    case TPlanNodeType::MATERIALIZATION_NODE: {
1656
1.61k
        op = std::make_shared<MaterializationOperator>(pool, tnode, next_operator_id(), descs);
1657
1.61k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1658
1.61k
        break;
1659
1.61k
    }
1660
1.61k
    case TPlanNodeType::INTERSECT_NODE: {
1661
114
        RETURN_IF_ERROR(_build_operators_for_set_operation_node<true>(pool, tnode, descs, op,
1662
114
                                                                      cur_pipe, sink_ops));
1663
114
        break;
1664
114
    }
1665
125
    case TPlanNodeType::EXCEPT_NODE: {
1666
125
        RETURN_IF_ERROR(_build_operators_for_set_operation_node<false>(pool, tnode, descs, op,
1667
125
                                                                       cur_pipe, sink_ops));
1668
125
        break;
1669
125
    }
1670
313
    case TPlanNodeType::REPEAT_NODE: {
1671
313
        op = std::make_shared<RepeatOperatorX>(pool, tnode, next_operator_id(), descs);
1672
313
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1673
313
        break;
1674
313
    }
1675
921
    case TPlanNodeType::TABLE_FUNCTION_NODE: {
1676
921
        op = std::make_shared<TableFunctionOperatorX>(pool, tnode, next_operator_id(), descs);
1677
921
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1678
921
        break;
1679
921
    }
1680
921
    case TPlanNodeType::ASSERT_NUM_ROWS_NODE: {
1681
218
        op = std::make_shared<AssertNumRowsOperatorX>(pool, tnode, next_operator_id(), descs);
1682
218
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1683
218
        break;
1684
218
    }
1685
1.53k
    case TPlanNodeType::EMPTY_SET_NODE: {
1686
1.53k
        op = std::make_shared<EmptySetSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1687
1.53k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1688
1.53k
        break;
1689
1.53k
    }
1690
1.53k
    case TPlanNodeType::DATA_GEN_SCAN_NODE: {
1691
459
        op = std::make_shared<DataGenSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1692
459
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1693
459
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1694
459
        break;
1695
459
    }
1696
2.31k
    case TPlanNodeType::SCHEMA_SCAN_NODE: {
1697
2.31k
        op = std::make_shared<SchemaScanOperatorX>(pool, tnode, next_operator_id(), descs);
1698
2.31k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1699
2.31k
        break;
1700
2.31k
    }
1701
6.78k
    case TPlanNodeType::META_SCAN_NODE: {
1702
6.78k
        op = std::make_shared<MetaScanOperatorX>(pool, tnode, next_operator_id(), descs);
1703
6.78k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1704
6.78k
        break;
1705
6.78k
    }
1706
6.78k
    case TPlanNodeType::SELECT_NODE: {
1707
1.67k
        op = std::make_shared<SelectOperatorX>(pool, tnode, next_operator_id(), descs);
1708
1.67k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1709
1.67k
        break;
1710
1.67k
    }
1711
1.67k
    case TPlanNodeType::REC_CTE_NODE: {
1712
151
        op = std::make_shared<RecCTESourceOperatorX>(pool, tnode, next_operator_id(), descs);
1713
151
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1714
1715
151
        const auto downstream_pipeline_id = cur_pipe->id();
1716
151
        if (!_dag.contains(downstream_pipeline_id)) {
1717
148
            _dag.insert({downstream_pipeline_id, {}});
1718
148
        }
1719
1720
151
        PipelinePtr anchor_side_pipe = add_pipeline(cur_pipe);
1721
151
        _dag[downstream_pipeline_id].push_back(anchor_side_pipe->id());
1722
1723
151
        DataSinkOperatorPtr anchor_sink;
1724
151
        anchor_sink = std::make_shared<RecCTEAnchorSinkOperatorX>(next_sink_operator_id(),
1725
151
                                                                  op->operator_id(), tnode, descs);
1726
151
        RETURN_IF_ERROR(anchor_side_pipe->set_sink(anchor_sink));
1727
151
        RETURN_IF_ERROR(anchor_side_pipe->sink()->init(tnode, _runtime_state.get()));
1728
151
        _pipeline_parent_map.push(op->node_id(), anchor_side_pipe);
1729
1730
151
        PipelinePtr rec_side_pipe = add_pipeline(cur_pipe);
1731
151
        _dag[downstream_pipeline_id].push_back(rec_side_pipe->id());
1732
1733
151
        DataSinkOperatorPtr rec_sink;
1734
151
        rec_sink = std::make_shared<RecCTESinkOperatorX>(next_sink_operator_id(), op->operator_id(),
1735
151
                                                         tnode, descs);
1736
151
        RETURN_IF_ERROR(rec_side_pipe->set_sink(rec_sink));
1737
151
        RETURN_IF_ERROR(rec_side_pipe->sink()->init(tnode, _runtime_state.get()));
1738
151
        _pipeline_parent_map.push(op->node_id(), rec_side_pipe);
1739
1740
151
        break;
1741
151
    }
1742
1.95k
    case TPlanNodeType::REC_CTE_SCAN_NODE: {
1743
1.95k
        op = std::make_shared<RecCTEScanOperatorX>(pool, tnode, next_operator_id(), descs);
1744
1.95k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1745
1.95k
        break;
1746
1.95k
    }
1747
1.95k
    default:
1748
0
        return Status::InternalError("Unsupported exec type in pipeline: {}",
1749
0
                                     print_plan_node_type(tnode.node_type));
1750
661k
    }
1751
661k
    if (_params.__isset.parallel_instances && fe_with_old_version) {
1752
0
        cur_pipe->set_num_tasks(_params.parallel_instances);
1753
0
        op->set_serial_operator();
1754
0
    }
1755
1756
661k
    return Status::OK();
1757
661k
}
1758
// NOLINTEND(readability-function-cognitive-complexity)
1759
// NOLINTEND(readability-function-size)
1760
1761
template <bool is_intersect>
1762
Status PipelineFragmentContext::_build_operators_for_set_operation_node(
1763
        ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, OperatorPtr& op,
1764
239
        PipelinePtr& cur_pipe, std::vector<DataSinkOperatorPtr>& sink_ops) {
1765
239
    op.reset(new SetSourceOperatorX<is_intersect>(pool, tnode, next_operator_id(), descs));
1766
239
    RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1767
1768
239
    const auto downstream_pipeline_id = cur_pipe->id();
1769
239
    if (!_dag.contains(downstream_pipeline_id)) {
1770
222
        _dag.insert({downstream_pipeline_id, {}});
1771
222
    }
1772
1773
810
    for (int child_id = 0; child_id < tnode.num_children; child_id++) {
1774
571
        PipelinePtr probe_side_pipe = add_pipeline(cur_pipe);
1775
571
        _dag[downstream_pipeline_id].push_back(probe_side_pipe->id());
1776
1777
571
        if (child_id == 0) {
1778
239
            sink_ops.push_back(std::make_shared<SetSinkOperatorX<is_intersect>>(
1779
239
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1780
332
        } else {
1781
332
            sink_ops.push_back(std::make_shared<SetProbeSinkOperatorX<is_intersect>>(
1782
332
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1783
332
        }
1784
571
        RETURN_IF_ERROR(probe_side_pipe->set_sink(sink_ops.back()));
1785
571
        RETURN_IF_ERROR(probe_side_pipe->sink()->init(tnode, _runtime_state.get()));
1786
        // prepare children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build.
1787
571
        _pipeline_parent_map.push(op->node_id(), probe_side_pipe);
1788
571
    }
1789
1790
239
    return Status::OK();
1791
239
}
_ZN5doris23PipelineFragmentContext39_build_operators_for_set_operation_nodeILb1EEENS_6StatusEPNS_10ObjectPoolERKNS_9TPlanNodeERKNS_13DescriptorTblERSt10shared_ptrINS_13OperatorXBaseEERSB_INS_8PipelineEERSt6vectorISB_INS_21DataSinkOperatorXBaseEESaISK_EE
Line
Count
Source
1764
114
        PipelinePtr& cur_pipe, std::vector<DataSinkOperatorPtr>& sink_ops) {
1765
114
    op.reset(new SetSourceOperatorX<is_intersect>(pool, tnode, next_operator_id(), descs));
1766
114
    RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1767
1768
114
    const auto downstream_pipeline_id = cur_pipe->id();
1769
114
    if (!_dag.contains(downstream_pipeline_id)) {
1770
105
        _dag.insert({downstream_pipeline_id, {}});
1771
105
    }
1772
1773
420
    for (int child_id = 0; child_id < tnode.num_children; child_id++) {
1774
306
        PipelinePtr probe_side_pipe = add_pipeline(cur_pipe);
1775
306
        _dag[downstream_pipeline_id].push_back(probe_side_pipe->id());
1776
1777
306
        if (child_id == 0) {
1778
114
            sink_ops.push_back(std::make_shared<SetSinkOperatorX<is_intersect>>(
1779
114
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1780
192
        } else {
1781
192
            sink_ops.push_back(std::make_shared<SetProbeSinkOperatorX<is_intersect>>(
1782
192
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1783
192
        }
1784
306
        RETURN_IF_ERROR(probe_side_pipe->set_sink(sink_ops.back()));
1785
306
        RETURN_IF_ERROR(probe_side_pipe->sink()->init(tnode, _runtime_state.get()));
1786
        // prepare children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build.
1787
306
        _pipeline_parent_map.push(op->node_id(), probe_side_pipe);
1788
306
    }
1789
1790
114
    return Status::OK();
1791
114
}
_ZN5doris23PipelineFragmentContext39_build_operators_for_set_operation_nodeILb0EEENS_6StatusEPNS_10ObjectPoolERKNS_9TPlanNodeERKNS_13DescriptorTblERSt10shared_ptrINS_13OperatorXBaseEERSB_INS_8PipelineEERSt6vectorISB_INS_21DataSinkOperatorXBaseEESaISK_EE
Line
Count
Source
1764
125
        PipelinePtr& cur_pipe, std::vector<DataSinkOperatorPtr>& sink_ops) {
1765
125
    op.reset(new SetSourceOperatorX<is_intersect>(pool, tnode, next_operator_id(), descs));
1766
125
    RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1767
1768
125
    const auto downstream_pipeline_id = cur_pipe->id();
1769
125
    if (!_dag.contains(downstream_pipeline_id)) {
1770
117
        _dag.insert({downstream_pipeline_id, {}});
1771
117
    }
1772
1773
390
    for (int child_id = 0; child_id < tnode.num_children; child_id++) {
1774
265
        PipelinePtr probe_side_pipe = add_pipeline(cur_pipe);
1775
265
        _dag[downstream_pipeline_id].push_back(probe_side_pipe->id());
1776
1777
265
        if (child_id == 0) {
1778
125
            sink_ops.push_back(std::make_shared<SetSinkOperatorX<is_intersect>>(
1779
125
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1780
140
        } else {
1781
140
            sink_ops.push_back(std::make_shared<SetProbeSinkOperatorX<is_intersect>>(
1782
140
                    child_id, next_sink_operator_id(), op->operator_id(), pool, tnode, descs));
1783
140
        }
1784
265
        RETURN_IF_ERROR(probe_side_pipe->set_sink(sink_ops.back()));
1785
265
        RETURN_IF_ERROR(probe_side_pipe->sink()->init(tnode, _runtime_state.get()));
1786
        // prepare children pipelines. if any pipeline found this as its father, will use the prepared pipeline to build.
1787
265
        _pipeline_parent_map.push(op->node_id(), probe_side_pipe);
1788
265
    }
1789
1790
125
    return Status::OK();
1791
125
}
1792
1793
427k
Status PipelineFragmentContext::submit() {
1794
427k
    if (_submitted) {
1795
0
        return Status::InternalError("submitted");
1796
0
    }
1797
427k
    _submitted = true;
1798
1799
427k
    int submit_tasks = 0;
1800
427k
    Status st;
1801
427k
    auto* scheduler = _query_ctx->get_pipe_exec_scheduler();
1802
1.24M
    for (auto& task : _tasks) {
1803
2.13M
        for (auto& t : task) {
1804
2.13M
            st = scheduler->submit(t.first);
1805
2.13M
            DBUG_EXECUTE_IF("PipelineFragmentContext.submit.failed",
1806
2.13M
                            { st = Status::Aborted("PipelineFragmentContext.submit.failed"); });
1807
2.13M
            if (!st) {
1808
0
                cancel(Status::InternalError("submit context to executor fail"));
1809
0
                std::lock_guard<std::mutex> l(_task_mutex);
1810
0
                _total_tasks = submit_tasks;
1811
0
                break;
1812
0
            }
1813
2.13M
            submit_tasks++;
1814
2.13M
        }
1815
1.24M
    }
1816
427k
    if (!st.ok()) {
1817
0
        bool need_remove = false;
1818
0
        {
1819
0
            std::lock_guard<std::mutex> l(_task_mutex);
1820
0
            if (_closed_tasks >= _total_tasks) {
1821
0
                need_remove = _close_fragment_instance();
1822
0
            }
1823
0
        }
1824
        // Call remove_pipeline_context() outside _task_mutex to avoid ABBA deadlock.
1825
0
        if (need_remove) {
1826
0
            _exec_env->fragment_mgr()->remove_pipeline_context({_query_id, _fragment_id});
1827
0
        }
1828
0
        return Status::InternalError("Submit pipeline failed. err = {}, BE: {}", st.to_string(),
1829
0
                                     BackendOptions::get_localhost());
1830
427k
    } else {
1831
427k
        return st;
1832
427k
    }
1833
427k
}
1834
1835
0
void PipelineFragmentContext::print_profile(const std::string& extra_info) {
1836
0
    if (_runtime_state->enable_profile()) {
1837
0
        std::stringstream ss;
1838
0
        for (auto runtime_profile_ptr : _runtime_state->pipeline_id_to_profile()) {
1839
0
            runtime_profile_ptr->pretty_print(&ss);
1840
0
        }
1841
1842
0
        if (_runtime_state->load_channel_profile()) {
1843
0
            _runtime_state->load_channel_profile()->pretty_print(&ss);
1844
0
        }
1845
1846
0
        auto profile_str =
1847
0
                fmt::format("Query {} fragment {} {}, profile, {}", print_id(this->_query_id),
1848
0
                            this->_fragment_id, extra_info, ss.str());
1849
0
        LOG_LONG_STRING(INFO, profile_str);
1850
0
    }
1851
0
}
1852
// If all pipeline tasks binded to the fragment instance are finished, then we could
1853
// close the fragment instance.
1854
// Returns true if the caller should call remove_pipeline_context() **after** releasing
1855
// _task_mutex. We must not call remove_pipeline_context() here because it acquires
1856
// _pipeline_map's shard lock, and this function is called while _task_mutex is held.
1857
// Acquiring _pipeline_map while holding _task_mutex creates an ABBA deadlock with
1858
// dump_pipeline_tasks(), which acquires _pipeline_map first and then _task_mutex
1859
// (via debug_string()).
1860
428k
bool PipelineFragmentContext::_close_fragment_instance() {
1861
428k
    if (_is_fragment_instance_closed) {
1862
0
        return false;
1863
0
    }
1864
428k
    Defer defer_op {[&]() { _is_fragment_instance_closed = true; }};
1865
428k
    _fragment_level_profile->total_time_counter()->update(_fragment_watcher.elapsed_time());
1866
428k
    if (!_need_notify_close) {
1867
424k
        auto st = send_report(true);
1868
424k
        if (!st) {
1869
0
            LOG(WARNING) << fmt::format("Failed to send report for query {}, fragment {}: {}",
1870
0
                                        print_id(_query_id), _fragment_id, st.to_string());
1871
0
        }
1872
424k
    }
1873
    // Print profile content in info log is a tempoeray solution for stream load and external_connector.
1874
    // Since stream load does not have someting like coordinator on FE, so
1875
    // backend can not report profile to FE, ant its profile can not be shown
1876
    // in the same way with other query. So we print the profile content to info log.
1877
1878
428k
    if (_runtime_state->enable_profile() &&
1879
428k
        (_query_ctx->get_query_source() == QuerySource::STREAM_LOAD ||
1880
2.53k
         _query_ctx->get_query_source() == QuerySource::EXTERNAL_CONNECTOR ||
1881
2.53k
         _query_ctx->get_query_source() == QuerySource::GROUP_COMMIT_LOAD)) {
1882
0
        std::stringstream ss;
1883
        // Compute the _local_time_percent before pretty_print the runtime_profile
1884
        // Before add this operation, the print out like that:
1885
        // UNION_NODE (id=0):(Active: 56.720us, non-child: 00.00%)
1886
        // After add the operation, the print out like that:
1887
        // UNION_NODE (id=0):(Active: 56.720us, non-child: 82.53%)
1888
        // We can easily know the exec node execute time without child time consumed.
1889
0
        for (auto runtime_profile_ptr : _runtime_state->pipeline_id_to_profile()) {
1890
0
            runtime_profile_ptr->pretty_print(&ss);
1891
0
        }
1892
1893
0
        if (_runtime_state->load_channel_profile()) {
1894
0
            _runtime_state->load_channel_profile()->pretty_print(&ss);
1895
0
        }
1896
1897
0
        LOG_INFO("Query {} fragment {} profile:\n {}", print_id(_query_id), _fragment_id, ss.str());
1898
0
    }
1899
1900
428k
    if (_query_ctx->enable_profile()) {
1901
2.53k
        _query_ctx->add_fragment_profile(_fragment_id, collect_realtime_profile(),
1902
2.53k
                                         collect_realtime_load_channel_profile());
1903
2.53k
    }
1904
1905
    // Return whether the caller needs to remove from the pipeline map.
1906
    // The caller must do this after releasing _task_mutex.
1907
428k
    return !_need_notify_close;
1908
428k
}
1909
1910
2.12M
void PipelineFragmentContext::decrement_running_task(PipelineId pipeline_id) {
1911
    // If all tasks of this pipeline has been closed, upstream tasks is never needed, and we just make those runnable here
1912
2.12M
    DCHECK(_pip_id_to_pipeline.contains(pipeline_id));
1913
2.12M
    if (_pip_id_to_pipeline[pipeline_id]->close_task()) {
1914
686k
        if (_dag.contains(pipeline_id)) {
1915
390k
            for (auto dep : _dag[pipeline_id]) {
1916
390k
                _pip_id_to_pipeline[dep]->make_all_runnable(pipeline_id);
1917
390k
            }
1918
300k
        }
1919
686k
    }
1920
2.12M
    bool need_remove = false;
1921
2.12M
    {
1922
2.12M
        std::lock_guard<std::mutex> l(_task_mutex);
1923
2.12M
        ++_closed_tasks;
1924
2.12M
        if (_closed_tasks >= _total_tasks) {
1925
428k
            need_remove = _close_fragment_instance();
1926
428k
        }
1927
2.12M
    }
1928
    // Call remove_pipeline_context() outside _task_mutex to avoid ABBA deadlock.
1929
2.12M
    if (need_remove) {
1930
424k
        _exec_env->fragment_mgr()->remove_pipeline_context({_query_id, _fragment_id});
1931
424k
    }
1932
2.12M
}
1933
1934
52.6k
std::string PipelineFragmentContext::get_load_error_url() {
1935
52.6k
    if (const auto& str = _runtime_state->get_error_log_file_path(); !str.empty()) {
1936
0
        return to_load_error_http_path(str);
1937
0
    }
1938
140k
    for (auto& tasks : _tasks) {
1939
224k
        for (auto& task : tasks) {
1940
224k
            if (const auto& str = task.second->get_error_log_file_path(); !str.empty()) {
1941
165
                return to_load_error_http_path(str);
1942
165
            }
1943
224k
        }
1944
140k
    }
1945
52.5k
    return "";
1946
52.6k
}
1947
1948
52.6k
std::string PipelineFragmentContext::get_first_error_msg() {
1949
52.6k
    if (const auto& str = _runtime_state->get_first_error_msg(); !str.empty()) {
1950
0
        return str;
1951
0
    }
1952
140k
    for (auto& tasks : _tasks) {
1953
224k
        for (auto& task : tasks) {
1954
224k
            if (const auto& str = task.second->get_first_error_msg(); !str.empty()) {
1955
165
                return str;
1956
165
            }
1957
224k
        }
1958
140k
    }
1959
52.5k
    return "";
1960
52.6k
}
1961
1962
429k
Status PipelineFragmentContext::send_report(bool done) {
1963
429k
    Status exec_status = _query_ctx->exec_status();
1964
    // If plan is done successfully, but _is_report_success is false,
1965
    // no need to send report.
1966
    // Load will set _is_report_success to true because load wants to know
1967
    // the process.
1968
429k
    if (!_is_report_success && done && exec_status.ok()) {
1969
382k
        return Status::OK();
1970
382k
    }
1971
1972
    // If both _is_report_success and _is_report_on_cancel are false,
1973
    // which means no matter query is success or failed, no report is needed.
1974
    // This may happen when the query limit reached and
1975
    // a internal cancellation being processed
1976
    // When limit is reached the fragment is also cancelled, but _is_report_on_cancel will
1977
    // be set to false, to avoid sending fault report to FE.
1978
47.0k
    if (!_is_report_success && !_is_report_on_cancel) {
1979
378
        if (done) {
1980
            // if done is true, which means the query is finished successfully, we can safely close the fragment instance without sending report to FE, and just return OK status here.
1981
378
            return Status::OK();
1982
378
        }
1983
0
        return Status::NeedSendAgain("");
1984
378
    }
1985
1986
46.6k
    std::vector<RuntimeState*> runtime_states;
1987
1988
119k
    for (auto& tasks : _tasks) {
1989
174k
        for (auto& task : tasks) {
1990
174k
            runtime_states.push_back(task.second.get());
1991
174k
        }
1992
119k
    }
1993
1994
46.6k
    std::string load_eror_url = _query_ctx->get_load_error_url().empty()
1995
46.6k
                                        ? get_load_error_url()
1996
18.4E
                                        : _query_ctx->get_load_error_url();
1997
46.6k
    std::string first_error_msg = _query_ctx->get_first_error_msg().empty()
1998
46.6k
                                          ? get_first_error_msg()
1999
18.4E
                                          : _query_ctx->get_first_error_msg();
2000
2001
46.6k
    ReportStatusRequest req {.status = exec_status,
2002
46.6k
                             .runtime_states = runtime_states,
2003
46.6k
                             .done = done || !exec_status.ok(),
2004
46.6k
                             .coord_addr = _query_ctx->coord_addr,
2005
46.6k
                             .query_id = _query_id,
2006
46.6k
                             .fragment_id = _fragment_id,
2007
46.6k
                             .fragment_instance_id = TUniqueId(),
2008
46.6k
                             .backend_num = -1,
2009
46.6k
                             .runtime_state = _runtime_state.get(),
2010
46.6k
                             .load_error_url = load_eror_url,
2011
46.6k
                             .first_error_msg = first_error_msg,
2012
46.6k
                             .cancel_fn = [this](const Status& reason) { cancel(reason); }};
2013
2014
46.6k
    return _report_status_cb(
2015
46.6k
            req, std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this()));
2016
47.0k
}
2017
2018
0
size_t PipelineFragmentContext::get_revocable_size(bool* has_running_task) const {
2019
0
    size_t res = 0;
2020
    // _tasks will be cleared during ~PipelineFragmentContext, so that it's safe
2021
    // here to traverse the vector.
2022
0
    for (const auto& task_instances : _tasks) {
2023
0
        for (const auto& task : task_instances) {
2024
0
            if (task.first->is_running()) {
2025
0
                LOG_EVERY_N(INFO, 50) << "Query: " << print_id(_query_id)
2026
0
                                      << " is running, task: " << (void*)task.first.get()
2027
0
                                      << ", is_running: " << task.first->is_running();
2028
0
                *has_running_task = true;
2029
0
                return 0;
2030
0
            }
2031
2032
0
            size_t revocable_size = task.first->get_revocable_size();
2033
0
            if (revocable_size >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) {
2034
0
                res += revocable_size;
2035
0
            }
2036
0
        }
2037
0
    }
2038
0
    return res;
2039
0
}
2040
2041
0
std::vector<PipelineTask*> PipelineFragmentContext::get_revocable_tasks() const {
2042
0
    std::vector<PipelineTask*> revocable_tasks;
2043
0
    for (const auto& task_instances : _tasks) {
2044
0
        for (const auto& task : task_instances) {
2045
0
            size_t revocable_size_ = task.first->get_revocable_size();
2046
2047
0
            if (revocable_size_ >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) {
2048
0
                revocable_tasks.emplace_back(task.first.get());
2049
0
            }
2050
0
        }
2051
0
    }
2052
0
    return revocable_tasks;
2053
0
}
2054
2055
121
std::string PipelineFragmentContext::debug_string() {
2056
121
    std::lock_guard<std::mutex> l(_task_mutex);
2057
121
    fmt::memory_buffer debug_string_buffer;
2058
121
    fmt::format_to(debug_string_buffer,
2059
121
                   "PipelineFragmentContext Info: _closed_tasks={}, _total_tasks={}, "
2060
121
                   "need_notify_close={}, fragment_id={}, _rec_cte_stage={}\n",
2061
121
                   _closed_tasks, _total_tasks, _need_notify_close, _fragment_id, _rec_cte_stage);
2062
811
    for (size_t j = 0; j < _tasks.size(); j++) {
2063
690
        fmt::format_to(debug_string_buffer, "Tasks in instance {}:\n", j);
2064
1.72k
        for (size_t i = 0; i < _tasks[j].size(); i++) {
2065
1.03k
            fmt::format_to(debug_string_buffer, "Task {}: {}\n", i,
2066
1.03k
                           _tasks[j][i].first->debug_string());
2067
1.03k
        }
2068
690
    }
2069
2070
121
    return fmt::to_string(debug_string_buffer);
2071
121
}
2072
2073
std::vector<std::shared_ptr<TRuntimeProfileTree>>
2074
2.53k
PipelineFragmentContext::collect_realtime_profile() const {
2075
2.53k
    std::vector<std::shared_ptr<TRuntimeProfileTree>> res;
2076
2077
    // we do not have mutex to protect pipeline_id_to_profile
2078
    // so we need to make sure this funciton is invoked after fragment context
2079
    // has already been prepared.
2080
2.53k
    if (!_prepared) {
2081
0
        std::string msg =
2082
0
                "Query " + print_id(_query_id) + " collecting profile, but its not prepared";
2083
0
        DCHECK(false) << msg;
2084
0
        LOG_ERROR(msg);
2085
0
        return res;
2086
0
    }
2087
2088
    // Make sure first profile is fragment level profile
2089
2.53k
    auto fragment_profile = std::make_shared<TRuntimeProfileTree>();
2090
2.53k
    _fragment_level_profile->to_thrift(fragment_profile.get(), _runtime_state->profile_level());
2091
2.53k
    res.push_back(fragment_profile);
2092
2093
    // pipeline_id_to_profile is initialized in prepare stage
2094
4.68k
    for (auto pipeline_profile : _runtime_state->pipeline_id_to_profile()) {
2095
4.68k
        auto profile_ptr = std::make_shared<TRuntimeProfileTree>();
2096
4.68k
        pipeline_profile->to_thrift(profile_ptr.get(), _runtime_state->profile_level());
2097
4.68k
        res.push_back(profile_ptr);
2098
4.68k
    }
2099
2100
2.53k
    return res;
2101
2.53k
}
2102
2103
std::shared_ptr<TRuntimeProfileTree>
2104
2.53k
PipelineFragmentContext::collect_realtime_load_channel_profile() const {
2105
    // we do not have mutex to protect pipeline_id_to_profile
2106
    // so we need to make sure this funciton is invoked after fragment context
2107
    // has already been prepared.
2108
2.53k
    if (!_prepared) {
2109
0
        std::string msg =
2110
0
                "Query " + print_id(_query_id) + " collecting profile, but its not prepared";
2111
0
        DCHECK(false) << msg;
2112
0
        LOG_ERROR(msg);
2113
0
        return nullptr;
2114
0
    }
2115
2116
7.48k
    for (const auto& tasks : _tasks) {
2117
14.7k
        for (const auto& task : tasks) {
2118
14.7k
            if (task.second->load_channel_profile() == nullptr) {
2119
0
                continue;
2120
0
            }
2121
2122
14.7k
            auto tmp_load_channel_profile = std::make_shared<TRuntimeProfileTree>();
2123
2124
14.7k
            task.second->load_channel_profile()->to_thrift(tmp_load_channel_profile.get(),
2125
14.7k
                                                           _runtime_state->profile_level());
2126
14.7k
            _runtime_state->load_channel_profile()->update(*tmp_load_channel_profile);
2127
14.7k
        }
2128
7.48k
    }
2129
2130
2.53k
    auto load_channel_profile = std::make_shared<TRuntimeProfileTree>();
2131
2.53k
    _runtime_state->load_channel_profile()->to_thrift(load_channel_profile.get(),
2132
2.53k
                                                      _runtime_state->profile_level());
2133
2.53k
    return load_channel_profile;
2134
2.53k
}
2135
2136
// Collect runtime filter IDs registered by all tasks in this PFC.
2137
// Used during recursive CTE stage transitions to know which filters to deregister
2138
// before creating the new PFC for the next recursion round.
2139
// Called from rerun_fragment(wait_for_destroy) while tasks are still closing.
2140
// Thread safety: safe because _tasks is structurally immutable after prepare() —
2141
// the vector sizes do not change, and individual RuntimeState filter sets are
2142
// written only during open() which has completed by the time we reach rerun.
2143
3.28k
std::set<int> PipelineFragmentContext::get_deregister_runtime_filter() const {
2144
3.28k
    std::set<int> result;
2145
7.66k
    for (const auto& _task : _tasks) {
2146
16.1k
        for (const auto& task : _task) {
2147
16.1k
            auto set = task.first->runtime_state()->get_deregister_runtime_filter();
2148
16.1k
            result.merge(set);
2149
16.1k
        }
2150
7.66k
    }
2151
3.28k
    if (_runtime_state) {
2152
3.28k
        auto set = _runtime_state->get_deregister_runtime_filter();
2153
3.28k
        result.merge(set);
2154
3.28k
    }
2155
3.28k
    return result;
2156
3.28k
}
2157
2158
429k
void PipelineFragmentContext::_release_resource() {
2159
429k
    std::lock_guard<std::mutex> l(_task_mutex);
2160
    // The memory released by the query end is recorded in the query mem tracker.
2161
429k
    SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_ctx->query_mem_tracker());
2162
429k
    auto st = _query_ctx->exec_status();
2163
1.24M
    for (auto& _task : _tasks) {
2164
1.24M
        if (!_task.empty()) {
2165
1.24M
            _call_back(_task.front().first->runtime_state(), &st);
2166
1.24M
        }
2167
1.24M
    }
2168
429k
    _tasks.clear();
2169
429k
    _dag.clear();
2170
429k
    _pip_id_to_pipeline.clear();
2171
429k
    _pipelines.clear();
2172
429k
    _sink.reset();
2173
429k
    _root_op.reset();
2174
429k
    _runtime_filter_mgr_map.clear();
2175
429k
    _op_id_to_shared_state.clear();
2176
429k
}
2177
2178
#include "common/compile_check_end.h"
2179
} // namespace doris