Coverage Report

Created: 2026-04-09 13:36

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
425k
        : _query_id(std::move(query_id)),
137
425k
          _fragment_id(request.fragment_id),
138
425k
          _exec_env(exec_env),
139
425k
          _query_ctx(std::move(query_ctx)),
140
425k
          _call_back(call_back),
141
425k
          _is_report_on_cancel(true),
142
425k
          _report_status_cb(std::move(report_status_cb)),
143
425k
          _params(request),
144
425k
          _parallel_instances(_params.__isset.parallel_instances ? _params.parallel_instances : 0),
145
425k
          _need_notify_close(request.__isset.need_notify_close ? request.need_notify_close
146
425k
                                                               : false) {
147
425k
    _fragment_watcher.start();
148
425k
}
149
150
426k
PipelineFragmentContext::~PipelineFragmentContext() {
151
426k
    LOG_INFO("PipelineFragmentContext::~PipelineFragmentContext")
152
426k
            .tag("query_id", print_id(_query_id))
153
426k
            .tag("fragment_id", _fragment_id);
154
426k
    _release_resource();
155
426k
    {
156
        // The memory released by the query end is recorded in the query mem tracker.
157
426k
        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_ctx->query_mem_tracker());
158
426k
        _runtime_state.reset();
159
426k
        _query_ctx.reset();
160
426k
    }
161
426k
}
162
163
139
bool PipelineFragmentContext::is_timeout(timespec now) const {
164
139
    if (_timeout <= 0) {
165
0
        return false;
166
0
    }
167
139
    return _fragment_watcher.elapsed_time_seconds(now) > _timeout;
168
139
}
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.03k
bool PipelineFragmentContext::notify_close() {
175
9.03k
    bool all_closed = false;
176
9.03k
    bool need_remove = false;
177
9.03k
    {
178
9.03k
        std::lock_guard<std::mutex> l(_task_mutex);
179
9.03k
        if (_closed_tasks >= _total_tasks) {
180
3.44k
            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.37k
                need_remove = true;
187
3.37k
            }
188
3.44k
            all_closed = true;
189
3.44k
        }
190
        // make fragment release by self after cancel
191
9.03k
        _need_notify_close = false;
192
9.03k
    }
193
9.03k
    if (need_remove) {
194
3.37k
        _exec_env->fragment_mgr()->remove_pipeline_context({_query_id, _fragment_id});
195
3.37k
    }
196
9.03k
    return all_closed;
197
9.03k
}
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
5.55k
void PipelineFragmentContext::cancel(const Status reason) {
204
5.55k
    LOG_INFO("PipelineFragmentContext::cancel")
205
5.55k
            .tag("query_id", print_id(_query_id))
206
5.55k
            .tag("fragment_id", _fragment_id)
207
5.55k
            .tag("reason", reason.to_string());
208
5.55k
    if (notify_close()) {
209
86
        return;
210
86
    }
211
    // Timeout is a special error code, we need print current stack to debug timeout issue.
212
5.46k
    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.46k
    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.47k
    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.46k
    if (auto error_url = get_load_error_url(); !error_url.empty()) {
229
22
        _query_ctx->set_load_error_url(error_url);
230
22
    }
231
232
5.46k
    if (auto first_error_msg = get_first_error_msg(); !first_error_msg.empty()) {
233
22
        _query_ctx->set_first_error_msg(first_error_msg);
234
22
    }
235
236
5.46k
    _query_ctx->cancel(reason, _fragment_id);
237
5.46k
    if (reason.is<ErrorCode::LIMIT_REACH>()) {
238
516
        _is_report_on_cancel = false;
239
4.95k
    } else {
240
16.1k
        for (auto& id : _fragment_instance_ids) {
241
16.1k
            LOG(WARNING) << "PipelineFragmentContext cancel instance: " << print_id(id);
242
16.1k
        }
243
4.95k
    }
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.46k
    auto stream_load_ctx = _exec_env->new_load_stream_mgr()->get(_query_id);
247
5.46k
    if (stream_load_ctx != nullptr) {
248
30
        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
30
        stream_load_ctx->error_url = get_load_error_url();
253
30
        stream_load_ctx->first_error_msg = get_first_error_msg();
254
30
    }
255
256
17.7k
    for (auto& tasks : _tasks) {
257
37.5k
        for (auto& task : tasks) {
258
37.5k
            task.first->unblock_all_dependencies();
259
37.5k
        }
260
17.7k
    }
261
5.46k
}
262
263
666k
PipelinePtr PipelineFragmentContext::add_pipeline(PipelinePtr parent, int idx) {
264
666k
    PipelineId id = _next_pipeline_id++;
265
666k
    auto pipeline = std::make_shared<Pipeline>(
266
666k
            id, parent ? std::min(parent->num_tasks(), _num_instances) : _num_instances,
267
666k
            parent ? parent->num_tasks() : _num_instances);
268
666k
    if (idx >= 0) {
269
115k
        _pipelines.insert(_pipelines.begin() + idx, pipeline);
270
551k
    } else {
271
551k
        _pipelines.emplace_back(pipeline);
272
551k
    }
273
666k
    if (parent) {
274
235k
        parent->set_children(pipeline);
275
235k
    }
276
666k
    return pipeline;
277
666k
}
278
279
425k
Status PipelineFragmentContext::_build_and_prepare_full_pipeline(ThreadPool* thread_pool) {
280
425k
    {
281
425k
        SCOPED_TIMER(_build_pipelines_timer);
282
        // 2. Build pipelines with operators in this fragment.
283
425k
        auto root_pipeline = add_pipeline();
284
425k
        RETURN_IF_ERROR(_build_pipelines(_runtime_state->obj_pool(), *_query_ctx->desc_tbl,
285
425k
                                         &_root_op, root_pipeline));
286
287
        // 3. Create sink operator
288
425k
        if (!_params.fragment.__isset.output_sink) {
289
0
            return Status::InternalError("No output sink in this fragment!");
290
0
        }
291
425k
        RETURN_IF_ERROR(_create_data_sink(_runtime_state->obj_pool(), _params.fragment.output_sink,
292
425k
                                          _params.fragment.output_exprs, _params,
293
425k
                                          root_pipeline->output_row_desc(), _runtime_state.get(),
294
425k
                                          *_desc_tbl, root_pipeline->id()));
295
425k
        RETURN_IF_ERROR(_sink->init(_params.fragment.output_sink));
296
425k
        RETURN_IF_ERROR(root_pipeline->set_sink(_sink));
297
298
550k
        for (PipelinePtr& pipeline : _pipelines) {
299
18.4E
            DCHECK(pipeline->sink() != nullptr) << pipeline->operators().size();
300
550k
            RETURN_IF_ERROR(pipeline->sink()->set_child(pipeline->operators().back()));
301
550k
        }
302
425k
    }
303
    // 4. Build local exchanger
304
425k
    if (_runtime_state->enable_local_shuffle()) {
305
423k
        SCOPED_TIMER(_plan_local_exchanger_timer);
306
423k
        RETURN_IF_ERROR(_plan_local_exchange(_params.num_buckets,
307
423k
                                             _params.bucket_seq_to_instance_idx,
308
423k
                                             _params.shuffle_idx_to_instance_idx));
309
423k
    }
310
311
    // 5. Initialize global states in pipelines.
312
667k
    for (PipelinePtr& pipeline : _pipelines) {
313
667k
        SCOPED_TIMER(_prepare_all_pipelines_timer);
314
667k
        pipeline->children().clear();
315
667k
        RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get()));
316
667k
    }
317
318
424k
    {
319
424k
        SCOPED_TIMER(_build_tasks_timer);
320
        // 6. Build pipeline tasks and initialize local state.
321
424k
        RETURN_IF_ERROR(_build_pipeline_tasks(thread_pool));
322
424k
    }
323
324
424k
    return Status::OK();
325
424k
}
326
327
425k
Status PipelineFragmentContext::prepare(ThreadPool* thread_pool) {
328
425k
    if (_prepared) {
329
0
        return Status::InternalError("Already prepared");
330
0
    }
331
425k
    if (_params.__isset.query_options && _params.query_options.__isset.execution_timeout) {
332
425k
        _timeout = _params.query_options.execution_timeout;
333
425k
    }
334
335
425k
    _fragment_level_profile = std::make_unique<RuntimeProfile>("PipelineContext");
336
425k
    _prepare_timer = ADD_TIMER(_fragment_level_profile, "PrepareTime");
337
425k
    SCOPED_TIMER(_prepare_timer);
338
425k
    _build_pipelines_timer = ADD_TIMER(_fragment_level_profile, "BuildPipelinesTime");
339
425k
    _init_context_timer = ADD_TIMER(_fragment_level_profile, "InitContextTime");
340
425k
    _plan_local_exchanger_timer = ADD_TIMER(_fragment_level_profile, "PlanLocalLocalExchangerTime");
341
425k
    _build_tasks_timer = ADD_TIMER(_fragment_level_profile, "BuildTasksTime");
342
425k
    _prepare_all_pipelines_timer = ADD_TIMER(_fragment_level_profile, "PrepareAllPipelinesTime");
343
425k
    {
344
425k
        SCOPED_TIMER(_init_context_timer);
345
425k
        cast_set(_num_instances, _params.local_params.size());
346
425k
        _total_instances =
347
425k
                _params.__isset.total_instances ? _params.total_instances : _num_instances;
348
349
425k
        auto* fragment_context = this;
350
351
425k
        if (_params.query_options.__isset.is_report_success) {
352
424k
            fragment_context->set_is_report_success(_params.query_options.is_report_success);
353
424k
        }
354
355
        // 1. Set up the global runtime state.
356
425k
        _runtime_state = RuntimeState::create_unique(
357
425k
                _params.query_id, _params.fragment_id, _params.query_options,
358
425k
                _query_ctx->query_globals, _exec_env, _query_ctx.get());
359
425k
        _runtime_state->set_task_execution_context(shared_from_this());
360
425k
        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_runtime_state->query_mem_tracker());
361
425k
        if (_params.__isset.backend_id) {
362
423k
            _runtime_state->set_backend_id(_params.backend_id);
363
423k
        }
364
425k
        if (_params.__isset.import_label) {
365
238
            _runtime_state->set_import_label(_params.import_label);
366
238
        }
367
425k
        if (_params.__isset.db_name) {
368
190
            _runtime_state->set_db_name(_params.db_name);
369
190
        }
370
425k
        if (_params.__isset.load_job_id) {
371
0
            _runtime_state->set_load_job_id(_params.load_job_id);
372
0
        }
373
374
425k
        if (_params.is_simplified_param) {
375
142k
            _desc_tbl = _query_ctx->desc_tbl;
376
283k
        } else {
377
283k
            DCHECK(_params.__isset.desc_tbl);
378
283k
            RETURN_IF_ERROR(DescriptorTbl::create(_runtime_state->obj_pool(), _params.desc_tbl,
379
283k
                                                  &_desc_tbl));
380
283k
        }
381
425k
        _runtime_state->set_desc_tbl(_desc_tbl);
382
425k
        _runtime_state->set_num_per_fragment_instances(_params.num_senders);
383
425k
        _runtime_state->set_load_stream_per_node(_params.load_stream_per_node);
384
425k
        _runtime_state->set_total_load_streams(_params.total_load_streams);
385
425k
        _runtime_state->set_num_local_sink(_params.num_local_sink);
386
387
        // init fragment_instance_ids
388
425k
        const auto target_size = _params.local_params.size();
389
425k
        _fragment_instance_ids.resize(target_size);
390
1.60M
        for (size_t i = 0; i < _params.local_params.size(); i++) {
391
1.17M
            auto fragment_instance_id = _params.local_params[i].fragment_instance_id;
392
1.17M
            _fragment_instance_ids[i] = fragment_instance_id;
393
1.17M
        }
394
425k
    }
395
396
425k
    RETURN_IF_ERROR(_build_and_prepare_full_pipeline(thread_pool));
397
398
424k
    _init_next_report_time();
399
400
424k
    _prepared = true;
401
424k
    return Status::OK();
402
425k
}
403
404
Status PipelineFragmentContext::_build_pipeline_tasks_for_instance(
405
        int instance_idx,
406
1.17M
        const std::vector<std::shared_ptr<RuntimeProfile>>& pipeline_id_to_profile) {
407
1.17M
    const auto& local_params = _params.local_params[instance_idx];
408
1.17M
    auto fragment_instance_id = local_params.fragment_instance_id;
409
1.17M
    auto runtime_filter_mgr = std::make_unique<RuntimeFilterMgr>(false);
410
1.17M
    std::map<PipelineId, PipelineTask*> pipeline_id_to_task;
411
1.17M
    auto get_shared_state = [&](PipelinePtr pipeline)
412
1.17M
            -> std::map<int, std::pair<std::shared_ptr<BasicSharedState>,
413
2.08M
                                       std::vector<std::shared_ptr<Dependency>>>> {
414
2.08M
        std::map<int, std::pair<std::shared_ptr<BasicSharedState>,
415
2.08M
                                std::vector<std::shared_ptr<Dependency>>>>
416
2.08M
                shared_state_map;
417
2.64M
        for (auto& op : pipeline->operators()) {
418
2.64M
            auto source_id = op->operator_id();
419
2.64M
            if (auto iter = _op_id_to_shared_state.find(source_id);
420
2.64M
                iter != _op_id_to_shared_state.end()) {
421
850k
                shared_state_map.insert({source_id, iter->second});
422
850k
            }
423
2.64M
        }
424
2.08M
        for (auto sink_to_source_id : pipeline->sink()->dests_id()) {
425
2.08M
            if (auto iter = _op_id_to_shared_state.find(sink_to_source_id);
426
2.08M
                iter != _op_id_to_shared_state.end()) {
427
369k
                shared_state_map.insert({sink_to_source_id, iter->second});
428
369k
            }
429
2.08M
        }
430
2.08M
        return shared_state_map;
431
2.08M
    };
432
433
3.73M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
434
2.55M
        auto& pipeline = _pipelines[pip_idx];
435
2.55M
        if (pipeline->num_tasks() > 1 || instance_idx == 0) {
436
2.07M
            auto task_runtime_state = RuntimeState::create_unique(
437
2.07M
                    local_params.fragment_instance_id, _params.query_id, _params.fragment_id,
438
2.07M
                    _params.query_options, _query_ctx->query_globals, _exec_env, _query_ctx.get());
439
2.07M
            {
440
                // Initialize runtime state for this task
441
2.07M
                task_runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker());
442
443
2.07M
                task_runtime_state->set_task_execution_context(shared_from_this());
444
2.07M
                task_runtime_state->set_be_number(local_params.backend_num);
445
446
2.08M
                if (_params.__isset.backend_id) {
447
2.08M
                    task_runtime_state->set_backend_id(_params.backend_id);
448
2.08M
                }
449
2.07M
                if (_params.__isset.import_label) {
450
239
                    task_runtime_state->set_import_label(_params.import_label);
451
239
                }
452
2.07M
                if (_params.__isset.db_name) {
453
191
                    task_runtime_state->set_db_name(_params.db_name);
454
191
                }
455
2.07M
                if (_params.__isset.load_job_id) {
456
0
                    task_runtime_state->set_load_job_id(_params.load_job_id);
457
0
                }
458
2.07M
                if (_params.__isset.wal_id) {
459
114
                    task_runtime_state->set_wal_id(_params.wal_id);
460
114
                }
461
2.07M
                if (_params.__isset.content_length) {
462
31
                    task_runtime_state->set_content_length(_params.content_length);
463
31
                }
464
465
2.07M
                task_runtime_state->set_desc_tbl(_desc_tbl);
466
2.07M
                task_runtime_state->set_per_fragment_instance_idx(local_params.sender_id);
467
2.07M
                task_runtime_state->set_num_per_fragment_instances(_params.num_senders);
468
2.07M
                task_runtime_state->resize_op_id_to_local_state(max_operator_id());
469
2.07M
                task_runtime_state->set_max_operator_id(max_operator_id());
470
2.07M
                task_runtime_state->set_load_stream_per_node(_params.load_stream_per_node);
471
2.07M
                task_runtime_state->set_total_load_streams(_params.total_load_streams);
472
2.07M
                task_runtime_state->set_num_local_sink(_params.num_local_sink);
473
474
2.07M
                task_runtime_state->set_runtime_filter_mgr(runtime_filter_mgr.get());
475
2.07M
            }
476
2.07M
            auto cur_task_id = _total_tasks++;
477
2.07M
            task_runtime_state->set_task_id(cur_task_id);
478
2.07M
            task_runtime_state->set_task_num(pipeline->num_tasks());
479
2.07M
            auto task = std::make_shared<PipelineTask>(
480
2.07M
                    pipeline, cur_task_id, task_runtime_state.get(),
481
2.07M
                    std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this()),
482
2.07M
                    pipeline_id_to_profile[pip_idx].get(), get_shared_state(pipeline),
483
2.07M
                    instance_idx);
484
2.07M
            pipeline->incr_created_tasks(instance_idx, task.get());
485
2.07M
            pipeline_id_to_task.insert({pipeline->id(), task.get()});
486
2.07M
            _tasks[instance_idx].emplace_back(
487
2.07M
                    std::pair<std::shared_ptr<PipelineTask>, std::unique_ptr<RuntimeState>> {
488
2.07M
                            std::move(task), std::move(task_runtime_state)});
489
2.07M
        }
490
2.55M
    }
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.55M
    for (auto& _pipeline : _pipelines) {
510
2.55M
        if (pipeline_id_to_task.contains(_pipeline->id())) {
511
2.07M
            auto* task = pipeline_id_to_task[_pipeline->id()];
512
2.07M
            DCHECK(task != nullptr);
513
514
            // If this task has upstream dependency, then inject it into this task.
515
2.07M
            if (_dag.contains(_pipeline->id())) {
516
1.37M
                auto& deps = _dag[_pipeline->id()];
517
2.20M
                for (auto& dep : deps) {
518
2.20M
                    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
521k
                            task->inject_shared_state(ss);
522
725k
                        } else {
523
725k
                            pipeline_id_to_task[dep]->inject_shared_state(
524
725k
                                    task->get_source_shared_state());
525
725k
                        }
526
1.24M
                    }
527
2.20M
                }
528
1.37M
            }
529
2.07M
        }
530
2.55M
    }
531
3.73M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
532
2.55M
        if (pipeline_id_to_task.contains(_pipelines[pip_idx]->id())) {
533
2.07M
            auto* task = pipeline_id_to_task[_pipelines[pip_idx]->id()];
534
2.07M
            DCHECK(pipeline_id_to_profile[pip_idx]);
535
2.07M
            std::vector<TScanRangeParams> scan_ranges;
536
2.07M
            auto node_id = _pipelines[pip_idx]->operators().front()->node_id();
537
2.07M
            if (local_params.per_node_scan_ranges.contains(node_id)) {
538
342k
                scan_ranges = local_params.per_node_scan_ranges.find(node_id)->second;
539
342k
            }
540
2.07M
            RETURN_IF_ERROR_OR_CATCH_EXCEPTION(task->prepare(scan_ranges, local_params.sender_id,
541
2.07M
                                                             _params.fragment.output_sink));
542
2.07M
        }
543
2.55M
    }
544
1.18M
    {
545
1.18M
        std::lock_guard<std::mutex> l(_state_map_lock);
546
1.18M
        _runtime_filter_mgr_map[instance_idx] = std::move(runtime_filter_mgr);
547
1.18M
    }
548
1.18M
    return Status::OK();
549
1.17M
}
550
551
425k
Status PipelineFragmentContext::_build_pipeline_tasks(ThreadPool* thread_pool) {
552
425k
    _total_tasks = 0;
553
425k
    _closed_tasks = 0;
554
425k
    const auto target_size = _params.local_params.size();
555
425k
    _tasks.resize(target_size);
556
425k
    _runtime_filter_mgr_map.resize(target_size);
557
1.09M
    for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) {
558
666k
        _pip_id_to_pipeline[_pipelines[pip_idx]->id()] = _pipelines[pip_idx].get();
559
666k
    }
560
425k
    auto pipeline_id_to_profile = _runtime_state->build_pipeline_profile(_pipelines.size());
561
562
425k
    if (target_size > 1 &&
563
425k
        (_runtime_state->query_options().__isset.parallel_prepare_threshold &&
564
133k
         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
10.3k
        std::vector<Status> prepare_status(target_size);
567
10.3k
        int submitted_tasks = 0;
568
10.3k
        Status submit_status;
569
10.3k
        CountDownLatch latch((int)target_size);
570
178k
        for (int i = 0; i < target_size; i++) {
571
167k
            submit_status = thread_pool->submit_func([&, i]() {
572
167k
                SCOPED_ATTACH_TASK(_query_ctx.get());
573
167k
                prepare_status[i] = _build_pipeline_tasks_for_instance(i, pipeline_id_to_profile);
574
167k
                latch.count_down();
575
167k
            });
576
167k
            if (LIKELY(submit_status.ok())) {
577
167k
                submitted_tasks++;
578
167k
            } else {
579
1
                break;
580
1
            }
581
167k
        }
582
10.3k
        latch.arrive_and_wait(target_size - submitted_tasks);
583
10.3k
        if (UNLIKELY(!submit_status.ok())) {
584
0
            return submit_status;
585
0
        }
586
178k
        for (int i = 0; i < submitted_tasks; i++) {
587
167k
            if (!prepare_status[i].ok()) {
588
0
                return prepare_status[i];
589
0
            }
590
167k
        }
591
414k
    } else {
592
1.42M
        for (int i = 0; i < target_size; i++) {
593
1.01M
            RETURN_IF_ERROR(_build_pipeline_tasks_for_instance(i, pipeline_id_to_profile));
594
1.01M
        }
595
414k
    }
596
425k
    _pipeline_parent_map.clear();
597
425k
    _op_id_to_shared_state.clear();
598
599
425k
    return Status::OK();
600
425k
}
601
602
423k
void PipelineFragmentContext::_init_next_report_time() {
603
423k
    auto interval_s = config::pipeline_status_report_interval;
604
423k
    if (_is_report_success && interval_s > 0 && _timeout > interval_s) {
605
40.1k
        VLOG_FILE << "enable period report: fragment id=" << _fragment_id;
606
40.1k
        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.1k
        _previous_report_time =
609
40.1k
                MonotonicNanos() + report_fragment_offset - (uint64_t)(interval_s)*NANOS_PER_SEC;
610
40.1k
        _disable_period_report = false;
611
40.1k
    }
612
423k
}
613
614
4.78k
void PipelineFragmentContext::refresh_next_report_time() {
615
4.78k
    auto disable = _disable_period_report.load(std::memory_order_acquire);
616
4.78k
    DCHECK(disable == true);
617
4.78k
    _previous_report_time.store(MonotonicNanos(), std::memory_order_release);
618
4.78k
    _disable_period_report.compare_exchange_strong(disable, false);
619
4.78k
}
620
621
7.54M
void PipelineFragmentContext::trigger_report_if_necessary() {
622
7.54M
    if (!_is_report_success) {
623
7.02M
        return;
624
7.02M
    }
625
528k
    auto disable = _disable_period_report.load(std::memory_order_acquire);
626
528k
    if (disable) {
627
8.79k
        return;
628
8.79k
    }
629
519k
    int32_t interval_s = config::pipeline_status_report_interval;
630
519k
    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
519k
    uint64_t next_report_time = _previous_report_time.load(std::memory_order_acquire) +
636
519k
                                (uint64_t)(interval_s)*NANOS_PER_SEC;
637
519k
    if (MonotonicNanos() > next_report_time) {
638
4.79k
        if (!_disable_period_report.compare_exchange_strong(disable, true,
639
4.79k
                                                            std::memory_order_acq_rel)) {
640
11
            return;
641
11
        }
642
4.78k
        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.78k
        auto st = send_report(false);
659
4.78k
        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.78k
    }
665
519k
}
666
667
Status PipelineFragmentContext::_build_pipelines(ObjectPool* pool, const DescriptorTbl& descs,
668
421k
                                                 OperatorPtr* root, PipelinePtr cur_pipe) {
669
421k
    if (_params.fragment.plan.nodes.empty()) {
670
0
        throw Exception(ErrorCode::INTERNAL_ERROR, "Invalid plan which has no plan node!");
671
0
    }
672
673
421k
    int node_idx = 0;
674
675
421k
    RETURN_IF_ERROR(_create_tree_helper(pool, _params.fragment.plan.nodes, descs, nullptr,
676
421k
                                        &node_idx, root, cur_pipe, 0, false, false));
677
678
421k
    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
421k
    return Status::OK();
683
421k
}
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
653k
        const bool followed_by_shuffled_operator, const bool require_bucket_distribution) {
689
    // propagate error case
690
653k
    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
653k
    const TPlanNode& tnode = tnodes[*node_idx];
696
697
653k
    int num_children = tnodes[*node_idx].num_children;
698
653k
    bool current_followed_by_shuffled_operator = followed_by_shuffled_operator;
699
653k
    bool current_require_bucket_distribution = require_bucket_distribution;
700
    // TODO: Create CacheOperator is confused now
701
653k
    OperatorPtr op = nullptr;
702
653k
    OperatorPtr cache_op = nullptr;
703
653k
    RETURN_IF_ERROR(_create_operator(pool, tnodes[*node_idx], descs, op, cur_pipe,
704
653k
                                     parent == nullptr ? -1 : parent->node_id(), child_idx,
705
653k
                                     followed_by_shuffled_operator,
706
653k
                                     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
653k
    RETURN_IF_ERROR(op->init(tnode, _runtime_state.get()));
710
    // assert(parent != nullptr || (node_idx == 0 && root_expr != nullptr));
711
653k
    if (parent != nullptr) {
712
        // add to parent's child(s)
713
230k
        RETURN_IF_ERROR(parent->set_child(cache_op ? cache_op : op));
714
423k
    } else {
715
423k
        *root = op;
716
423k
    }
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
653k
    auto required_data_distribution =
729
653k
            cur_pipe->operators().empty()
730
653k
                    ? cur_pipe->sink()->required_data_distribution(_runtime_state.get())
731
653k
                    : op->required_data_distribution(_runtime_state.get());
732
653k
    current_followed_by_shuffled_operator =
733
653k
            ((followed_by_shuffled_operator ||
734
653k
              (cur_pipe->operators().empty() ? cur_pipe->sink()->is_shuffled_operator()
735
596k
                                             : op->is_shuffled_operator())) &&
736
653k
             Pipeline::is_hash_exchange(required_data_distribution.distribution_type)) ||
737
653k
            (followed_by_shuffled_operator &&
738
544k
             required_data_distribution.distribution_type == ExchangeType::NOOP);
739
740
653k
    current_require_bucket_distribution =
741
653k
            ((require_bucket_distribution ||
742
653k
              (cur_pipe->operators().empty() ? cur_pipe->sink()->is_colocated_operator()
743
600k
                                             : op->is_colocated_operator())) &&
744
653k
             Pipeline::is_hash_exchange(required_data_distribution.distribution_type)) ||
745
653k
            (require_bucket_distribution &&
746
550k
             required_data_distribution.distribution_type == ExchangeType::NOOP);
747
748
653k
    if (num_children == 0) {
749
438k
        _use_serial_source = op->is_serial_operator();
750
438k
    }
751
    // rely on that tnodes is preorder of the plan
752
884k
    for (int i = 0; i < num_children; i++) {
753
230k
        ++*node_idx;
754
230k
        RETURN_IF_ERROR(_create_tree_helper(pool, tnodes, descs, op, node_idx, nullptr, cur_pipe, i,
755
230k
                                            current_followed_by_shuffled_operator,
756
230k
                                            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
230k
        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
230k
    }
767
768
653k
    return Status::OK();
769
653k
}
770
771
void PipelineFragmentContext::_inherit_pipeline_properties(
772
        const DataDistribution& data_distribution, PipelinePtr pipe_with_source,
773
115k
        PipelinePtr pipe_with_sink) {
774
115k
    pipe_with_sink->set_num_tasks(pipe_with_source->num_tasks());
775
115k
    pipe_with_source->set_num_tasks(_num_instances);
776
115k
    pipe_with_source->set_data_distribution(data_distribution);
777
115k
}
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
115k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
784
115k
    auto& operators = cur_pipe->operators();
785
115k
    const auto downstream_pipeline_id = cur_pipe->id();
786
115k
    auto local_exchange_id = next_operator_id();
787
    // 1. Create a new pipeline with local exchange sink.
788
115k
    DataSinkOperatorPtr sink;
789
115k
    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
115k
    const bool followed_by_shuffled_operator =
796
115k
            operators.size() > idx ? operators[idx]->followed_by_shuffled_operator()
797
115k
                                   : cur_pipe->sink()->followed_by_shuffled_operator();
798
115k
    const bool use_global_hash_shuffle = bucket_seq_to_instance_idx.empty() &&
799
115k
                                         !shuffle_idx_to_instance_idx.contains(-1) &&
800
115k
                                         followed_by_shuffled_operator && !_use_serial_source;
801
115k
    sink = std::make_shared<LocalExchangeSinkOperatorX>(
802
115k
            sink_id, local_exchange_id, use_global_hash_shuffle ? _total_instances : _num_instances,
803
115k
            data_distribution.partition_exprs, bucket_seq_to_instance_idx);
804
115k
    if (bucket_seq_to_instance_idx.empty() &&
805
115k
        data_distribution.distribution_type == ExchangeType::BUCKET_HASH_SHUFFLE) {
806
2
        data_distribution.distribution_type = ExchangeType::HASH_SHUFFLE;
807
2
    }
808
115k
    RETURN_IF_ERROR(new_pip->set_sink(sink));
809
115k
    RETURN_IF_ERROR(new_pip->sink()->init(_runtime_state.get(), data_distribution.distribution_type,
810
115k
                                          num_buckets, use_global_hash_shuffle,
811
115k
                                          shuffle_idx_to_instance_idx));
812
813
    // 2. Create and initialize LocalExchangeSharedState.
814
115k
    std::shared_ptr<LocalExchangeSharedState> shared_state =
815
115k
            LocalExchangeSharedState::create_shared(_num_instances);
816
115k
    switch (data_distribution.distribution_type) {
817
18.3k
    case ExchangeType::HASH_SHUFFLE:
818
18.3k
        shared_state->exchanger = ShuffleExchanger::create_unique(
819
18.3k
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances,
820
18.3k
                use_global_hash_shuffle ? _total_instances : _num_instances,
821
18.3k
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
822
18.3k
                        ? cast_set<int>(
823
18.3k
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
824
18.3k
                        : 0);
825
18.3k
        break;
826
447
    case ExchangeType::BUCKET_HASH_SHUFFLE:
827
447
        shared_state->exchanger = BucketShuffleExchanger::create_unique(
828
447
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances, num_buckets,
829
447
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
830
447
                        ? cast_set<int>(
831
447
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
832
447
                        : 0);
833
447
        break;
834
92.2k
    case ExchangeType::PASSTHROUGH:
835
92.2k
        shared_state->exchanger = PassthroughExchanger::create_unique(
836
92.2k
                cur_pipe->num_tasks(), _num_instances,
837
92.2k
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
838
92.2k
                        ? cast_set<int>(
839
92.1k
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
840
92.2k
                        : 0);
841
92.2k
        break;
842
453
    case ExchangeType::BROADCAST:
843
453
        shared_state->exchanger = BroadcastExchanger::create_unique(
844
453
                cur_pipe->num_tasks(), _num_instances,
845
453
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
846
453
                        ? cast_set<int>(
847
453
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
848
453
                        : 0);
849
453
        break;
850
2.92k
    case ExchangeType::PASS_TO_ONE:
851
2.92k
        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.72k
            shared_state->exchanger = PassToOneExchanger::create_unique(
854
1.72k
                    cur_pipe->num_tasks(), _num_instances,
855
1.72k
                    _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
856
1.72k
                            ? cast_set<int>(_runtime_state->query_options()
857
1.72k
                                                    .local_exchange_free_blocks_limit)
858
18.4E
                            : 0);
859
1.72k
        } else {
860
1.20k
            shared_state->exchanger = BroadcastExchanger::create_unique(
861
1.20k
                    cur_pipe->num_tasks(), _num_instances,
862
1.20k
                    _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
863
1.20k
                            ? cast_set<int>(_runtime_state->query_options()
864
1.20k
                                                    .local_exchange_free_blocks_limit)
865
1.20k
                            : 0);
866
1.20k
        }
867
2.92k
        break;
868
847
    case ExchangeType::ADAPTIVE_PASSTHROUGH:
869
847
        shared_state->exchanger = AdaptivePassthroughExchanger::create_unique(
870
847
                std::max(cur_pipe->num_tasks(), _num_instances), _num_instances,
871
847
                _runtime_state->query_options().__isset.local_exchange_free_blocks_limit
872
847
                        ? cast_set<int>(
873
847
                                  _runtime_state->query_options().local_exchange_free_blocks_limit)
874
847
                        : 0);
875
847
        break;
876
0
    default:
877
0
        return Status::InternalError("Unsupported local exchange type : " +
878
0
                                     std::to_string((int)data_distribution.distribution_type));
879
115k
    }
880
115k
    shared_state->create_source_dependencies(_num_instances, local_exchange_id, local_exchange_id,
881
115k
                                             "LOCAL_EXCHANGE_OPERATOR");
882
115k
    shared_state->create_sink_dependency(sink_id, local_exchange_id, "LOCAL_EXCHANGE_SINK");
883
115k
    _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
115k
    std::copy(operators.begin(), operators.begin() + idx,
890
115k
              std::inserter(new_pip->operators(), new_pip->operators().end()));
891
892
    // 3.2 Erase unused operators in previous pipeline.
893
115k
    operators.erase(operators.begin(), operators.begin() + idx);
894
895
    // 4. Initialize LocalExchangeSource and insert it into this pipeline.
896
115k
    OperatorPtr source_op;
897
115k
    source_op = std::make_shared<LocalExchangeSourceOperatorX>(pool, local_exchange_id);
898
115k
    RETURN_IF_ERROR(source_op->set_child(new_pip->operators().back()));
899
115k
    RETURN_IF_ERROR(source_op->init(data_distribution.distribution_type));
900
115k
    if (!operators.empty()) {
901
42.5k
        RETURN_IF_ERROR(operators.front()->set_child(nullptr));
902
42.5k
        RETURN_IF_ERROR(operators.front()->set_child(source_op));
903
42.5k
    }
904
115k
    operators.insert(operators.begin(), source_op);
905
906
    // 5. Set children for two pipelines separately.
907
115k
    std::vector<std::shared_ptr<Pipeline>> new_children;
908
115k
    std::vector<PipelineId> edges_with_source;
909
134k
    for (auto child : cur_pipe->children()) {
910
134k
        bool found = false;
911
149k
        for (auto op : new_pip->operators()) {
912
149k
            if (child->sink()->node_id() == op->node_id()) {
913
13.3k
                new_pip->set_children(child);
914
13.3k
                found = true;
915
13.3k
            };
916
149k
        }
917
134k
        if (!found) {
918
120k
            new_children.push_back(child);
919
120k
            edges_with_source.push_back(child->id());
920
120k
        }
921
134k
    }
922
115k
    new_children.push_back(new_pip);
923
115k
    edges_with_source.push_back(new_pip->id());
924
925
    // 6. Set DAG for new pipelines.
926
115k
    if (!new_pip->children().empty()) {
927
7.45k
        std::vector<PipelineId> edges_with_sink;
928
13.3k
        for (auto child : new_pip->children()) {
929
13.3k
            edges_with_sink.push_back(child->id());
930
13.3k
        }
931
7.45k
        _dag.insert({new_pip->id(), edges_with_sink});
932
7.45k
    }
933
115k
    cur_pipe->set_children(new_children);
934
115k
    _dag[downstream_pipeline_id] = edges_with_source;
935
115k
    RETURN_IF_ERROR(new_pip->sink()->set_child(new_pip->operators().back()));
936
115k
    RETURN_IF_ERROR(cur_pipe->sink()->set_child(nullptr));
937
115k
    RETURN_IF_ERROR(cur_pipe->sink()->set_child(cur_pipe->operators().back()));
938
939
    // 7. Inherit properties from current pipeline.
940
115k
    _inherit_pipeline_properties(data_distribution, cur_pipe, new_pip);
941
115k
    return Status::OK();
942
115k
}
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
188k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
949
188k
    if (_num_instances <= 1 || cur_pipe->num_tasks_of_parent() <= 1) {
950
47.4k
        return Status::OK();
951
47.4k
    }
952
953
141k
    if (!cur_pipe->need_to_local_exchange(data_distribution, idx)) {
954
44.6k
        return Status::OK();
955
44.6k
    }
956
96.7k
    *do_local_exchange = true;
957
958
96.7k
    auto& operators = cur_pipe->operators();
959
96.7k
    auto total_op_num = operators.size();
960
96.7k
    auto new_pip = add_pipeline(cur_pipe, pip_idx + 1);
961
96.7k
    RETURN_IF_ERROR(_add_local_exchange_impl(
962
96.7k
            idx, pool, cur_pipe, new_pip, data_distribution, do_local_exchange, num_buckets,
963
96.7k
            bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx));
964
965
18.4E
    CHECK(total_op_num + 1 == cur_pipe->operators().size() + new_pip->operators().size())
966
18.4E
            << "total_op_num: " << total_op_num
967
18.4E
            << " cur_pipe->operators().size(): " << cur_pipe->operators().size()
968
18.4E
            << " 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
96.7k
    if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 &&
976
96.7k
        Pipeline::heavy_operations_on_the_sink(data_distribution.distribution_type)) {
977
18.3k
        RETURN_IF_ERROR(_add_local_exchange_impl(
978
18.3k
                cast_set<int>(new_pip->operators().size()), pool, new_pip,
979
18.3k
                add_pipeline(new_pip, pip_idx + 2), DataDistribution(ExchangeType::PASSTHROUGH),
980
18.3k
                do_local_exchange, num_buckets, bucket_seq_to_instance_idx,
981
18.3k
                shuffle_idx_to_instance_idx));
982
18.3k
    }
983
96.7k
    return Status::OK();
984
96.7k
}
985
986
Status PipelineFragmentContext::_plan_local_exchange(
987
        int num_buckets, const std::map<int, int>& bucket_seq_to_instance_idx,
988
422k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
989
970k
    for (int pip_idx = cast_set<int>(_pipelines.size()) - 1; pip_idx >= 0; pip_idx--) {
990
548k
        _pipelines[pip_idx]->init_data_distribution(_runtime_state.get());
991
        // Set property if child pipeline is not join operator's child.
992
548k
        if (!_pipelines[pip_idx]->children().empty()) {
993
120k
            for (auto& child : _pipelines[pip_idx]->children()) {
994
120k
                if (child->sink()->node_id() ==
995
120k
                    _pipelines[pip_idx]->operators().front()->node_id()) {
996
108k
                    _pipelines[pip_idx]->set_data_distribution(child->data_distribution());
997
108k
                }
998
120k
            }
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
548k
        RETURN_IF_ERROR(_plan_local_exchange(num_buckets, pip_idx, _pipelines[pip_idx],
1005
548k
                                             bucket_seq_to_instance_idx,
1006
548k
                                             shuffle_idx_to_instance_idx));
1007
548k
    }
1008
422k
    return Status::OK();
1009
422k
}
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
547k
        const std::map<int, int>& shuffle_idx_to_instance_idx) {
1015
547k
    int idx = 1;
1016
547k
    bool do_local_exchange = false;
1017
589k
    do {
1018
589k
        auto& ops = pip->operators();
1019
589k
        do_local_exchange = false;
1020
        // Plan local exchange for each operator.
1021
658k
        for (; idx < ops.size();) {
1022
111k
            if (ops[idx]->required_data_distribution(_runtime_state.get()).need_local_exchange()) {
1023
102k
                RETURN_IF_ERROR(_add_local_exchange(
1024
102k
                        pip_idx, idx, ops[idx]->node_id(), _runtime_state->obj_pool(), pip,
1025
102k
                        ops[idx]->required_data_distribution(_runtime_state.get()),
1026
102k
                        &do_local_exchange, num_buckets, bucket_seq_to_instance_idx,
1027
102k
                        shuffle_idx_to_instance_idx));
1028
102k
            }
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
42.6k
                idx = 2;
1035
42.6k
                break;
1036
42.6k
            }
1037
68.4k
            idx++;
1038
68.4k
        }
1039
589k
    } while (do_local_exchange);
1040
547k
    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
547k
    return Status::OK();
1047
547k
}
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
425k
                                                  PipelineId cur_pipeline_id) {
1055
425k
    switch (thrift_sink.type) {
1056
140k
    case TDataSinkType::DATA_STREAM_SINK: {
1057
140k
        if (!thrift_sink.__isset.stream_sink) {
1058
0
            return Status::InternalError("Missing data stream sink.");
1059
0
        }
1060
140k
        _sink = std::make_shared<ExchangeSinkOperatorX>(
1061
140k
                state, row_desc, next_sink_operator_id(), thrift_sink.stream_sink,
1062
140k
                params.destinations, _fragment_instance_ids);
1063
140k
        break;
1064
140k
    }
1065
248k
    case TDataSinkType::RESULT_SINK: {
1066
248k
        if (!thrift_sink.__isset.result_sink) {
1067
0
            return Status::InternalError("Missing data buffer sink.");
1068
0
        }
1069
1070
248k
        _sink = std::make_shared<ResultSinkOperatorX>(next_sink_operator_id(), row_desc,
1071
248k
                                                      output_exprs, thrift_sink.result_sink);
1072
248k
        break;
1073
248k
    }
1074
104
    case TDataSinkType::DICTIONARY_SINK: {
1075
104
        if (!thrift_sink.__isset.dictionary_sink) {
1076
0
            return Status::InternalError("Missing dict sink.");
1077
0
        }
1078
1079
104
        _sink = std::make_shared<DictSinkOperatorX>(next_sink_operator_id(), row_desc, output_exprs,
1080
104
                                                    thrift_sink.dictionary_sink);
1081
104
        break;
1082
104
    }
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.09k
            _sink = std::make_shared<OlapTableSinkV2OperatorX>(pool, next_sink_operator_id(),
1089
2.09k
                                                               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
165
    case TDataSinkType::GROUP_COMMIT_BLOCK_SINK: {
1097
165
        DCHECK(thrift_sink.__isset.olap_table_sink);
1098
165
        DCHECK(state->get_query_ctx() != nullptr);
1099
165
        state->get_query_ctx()->query_mem_tracker()->is_group_commit_load = true;
1100
165
        _sink = std::make_shared<GroupCommitBlockSinkOperatorX>(next_sink_operator_id(), row_desc,
1101
165
                                                                output_exprs);
1102
165
        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.61k
    case TDataSinkType::ICEBERG_TABLE_SINK: {
1113
1.61k
        if (!thrift_sink.__isset.iceberg_table_sink) {
1114
0
            return Status::InternalError("Missing iceberg table sink.");
1115
0
        }
1116
1.61k
        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.61k
        } else {
1120
1.61k
            _sink = std::make_shared<IcebergTableSinkOperatorX>(pool, next_sink_operator_id(),
1121
1.61k
                                                                row_desc, output_exprs);
1122
1.61k
        }
1123
1.61k
        break;
1124
1.61k
    }
1125
12
    case TDataSinkType::ICEBERG_DELETE_SINK: {
1126
12
        if (!thrift_sink.__isset.iceberg_delete_sink) {
1127
0
            return Status::InternalError("Missing iceberg delete sink.");
1128
0
        }
1129
12
        _sink = std::make_shared<IcebergDeleteSinkOperatorX>(pool, next_sink_operator_id(),
1130
12
                                                             row_desc, output_exprs);
1131
12
        break;
1132
12
    }
1133
64
    case TDataSinkType::ICEBERG_MERGE_SINK: {
1134
64
        if (!thrift_sink.__isset.iceberg_merge_sink) {
1135
0
            return Status::InternalError("Missing iceberg merge sink.");
1136
0
        }
1137
64
        _sink = std::make_shared<IcebergMergeSinkOperatorX>(pool, next_sink_operator_id(), row_desc,
1138
64
                                                            output_exprs);
1139
64
        break;
1140
64
    }
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.90k
    case TDataSinkType::MULTI_CAST_DATA_STREAM_SINK: {
1189
1.90k
        DCHECK(thrift_sink.__isset.multi_cast_stream_sink);
1190
1.90k
        DCHECK_GT(thrift_sink.multi_cast_stream_sink.sinks.size(), 0);
1191
1.90k
        auto sink_id = next_sink_operator_id();
1192
1.90k
        const int multi_cast_node_id = sink_id;
1193
1.90k
        auto sender_size = thrift_sink.multi_cast_stream_sink.sinks.size();
1194
        // one sink has multiple sources.
1195
1.90k
        std::vector<int> sources;
1196
7.40k
        for (int i = 0; i < sender_size; ++i) {
1197
5.49k
            auto source_id = next_operator_id();
1198
5.49k
            sources.push_back(source_id);
1199
5.49k
        }
1200
1201
1.90k
        _sink = std::make_shared<MultiCastDataStreamSinkOperatorX>(
1202
1.90k
                sink_id, multi_cast_node_id, sources, pool, thrift_sink.multi_cast_stream_sink);
1203
7.40k
        for (int i = 0; i < sender_size; ++i) {
1204
5.49k
            auto new_pipeline = add_pipeline();
1205
            // use to exchange sink
1206
5.49k
            RowDescriptor* exchange_row_desc = nullptr;
1207
5.49k
            {
1208
5.49k
                const auto& tmp_row_desc =
1209
5.49k
                        !thrift_sink.multi_cast_stream_sink.sinks[i].output_exprs.empty()
1210
5.49k
                                ? RowDescriptor(state->desc_tbl(),
1211
5.49k
                                                {thrift_sink.multi_cast_stream_sink.sinks[i]
1212
5.49k
                                                         .output_tuple_id})
1213
5.49k
                                : row_desc;
1214
5.49k
                exchange_row_desc = pool->add(new RowDescriptor(tmp_row_desc));
1215
5.49k
            }
1216
5.49k
            auto source_id = sources[i];
1217
5.49k
            OperatorPtr source_op;
1218
            // 1. create and set the source operator of multi_cast_data_stream_source for new pipeline
1219
5.49k
            source_op = std::make_shared<MultiCastDataStreamerSourceOperatorX>(
1220
5.49k
                    /*node_id*/ source_id, /*consumer_id*/ i, pool,
1221
5.49k
                    thrift_sink.multi_cast_stream_sink.sinks[i], row_desc,
1222
5.49k
                    /*operator_id=*/source_id);
1223
5.49k
            RETURN_IF_ERROR(new_pipeline->add_operator(
1224
5.49k
                    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.49k
            DataSinkOperatorPtr sink_op;
1228
5.49k
            sink_op = std::make_shared<ExchangeSinkOperatorX>(
1229
5.49k
                    state, *exchange_row_desc, next_sink_operator_id(),
1230
5.49k
                    thrift_sink.multi_cast_stream_sink.sinks[i],
1231
5.49k
                    thrift_sink.multi_cast_stream_sink.destinations[i], _fragment_instance_ids);
1232
1233
5.49k
            RETURN_IF_ERROR(new_pipeline->set_sink(sink_op));
1234
5.49k
            {
1235
5.49k
                TDataSink* t = pool->add(new TDataSink());
1236
5.49k
                t->stream_sink = thrift_sink.multi_cast_stream_sink.sinks[i];
1237
5.49k
                RETURN_IF_ERROR(sink_op->init(*t));
1238
5.49k
            }
1239
1240
            // 3. set dependency dag
1241
5.49k
            _dag[new_pipeline->id()].push_back(cur_pipeline_id);
1242
5.49k
        }
1243
1.90k
        if (sources.empty()) {
1244
0
            return Status::InternalError("size of sources must be greater than 0");
1245
0
        }
1246
1.90k
        break;
1247
1.90k
    }
1248
1.90k
    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
425k
    }
1267
424k
    return Status::OK();
1268
425k
}
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
656k
                                                 OperatorPtr& cache_op) {
1279
656k
    std::vector<DataSinkOperatorPtr> sink_ops;
1280
656k
    Defer defer = Defer([&]() {
1281
656k
        if (op) {
1282
656k
            op->update_operator(tnode, followed_by_shuffled_operator, require_bucket_distribution);
1283
656k
        }
1284
655k
        for (auto& s : sink_ops) {
1285
120k
            s->update_operator(tnode, followed_by_shuffled_operator, require_bucket_distribution);
1286
120k
        }
1287
655k
    });
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
656k
    _pipeline_parent_map.pop(cur_pipe, parent_idx, child_idx);
1291
656k
    std::stringstream error_msg;
1292
656k
    bool enable_query_cache = _params.fragment.__isset.query_cache_param;
1293
1294
656k
    bool fe_with_old_version = false;
1295
656k
    switch (tnode.node_type) {
1296
206k
    case TPlanNodeType::OLAP_SCAN_NODE: {
1297
206k
        op = std::make_shared<OlapScanOperatorX>(
1298
206k
                pool, tnode, next_operator_id(), descs, _num_instances,
1299
206k
                enable_query_cache ? _params.fragment.query_cache_param : TQueryCacheParam {});
1300
206k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1301
206k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1302
206k
        break;
1303
206k
    }
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
22.7k
    case TPlanNodeType::FILE_SCAN_NODE: {
1327
22.7k
        op = std::make_shared<FileScanOperatorX>(pool, tnode, next_operator_id(), descs,
1328
22.7k
                                                 _num_instances);
1329
22.7k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1330
22.7k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1331
22.7k
        break;
1332
22.7k
    }
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
144k
    case TPlanNodeType::EXCHANGE_NODE: {
1342
144k
        int num_senders = _params.per_exch_num_senders.contains(tnode.node_id)
1343
144k
                                  ? _params.per_exch_num_senders.find(tnode.node_id)->second
1344
18.4E
                                  : 0;
1345
144k
        DCHECK_GT(num_senders, 0);
1346
144k
        op = std::make_shared<ExchangeSourceOperatorX>(pool, tnode, next_operator_id(), descs,
1347
144k
                                                       num_senders);
1348
144k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1349
144k
        fe_with_old_version = !tnode.__isset.is_serial_operator;
1350
144k
        break;
1351
144k
    }
1352
154k
    case TPlanNodeType::AGGREGATION_NODE: {
1353
154k
        if (tnode.agg_node.grouping_exprs.empty() &&
1354
154k
            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
154k
        bool need_create_cache_op =
1359
154k
                enable_query_cache && tnode.node_id == _params.fragment.query_cache_param.node_id;
1360
154k
        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
154k
        const bool group_by_limit_opt =
1380
154k
                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
154k
        const bool enable_spill = _runtime_state->enable_spill() &&
1385
154k
                                  !tnode.agg_node.grouping_exprs.empty() && !group_by_limit_opt;
1386
154k
        const bool is_streaming_agg = tnode.agg_node.__isset.use_streaming_preaggregation &&
1387
154k
                                      tnode.agg_node.use_streaming_preaggregation &&
1388
154k
                                      !tnode.agg_node.grouping_exprs.empty();
1389
        // TODO: distinct streaming agg does not support spill.
1390
154k
        const bool can_use_distinct_streaming_agg =
1391
154k
                (!enable_spill || is_streaming_agg) && tnode.agg_node.aggregate_functions.empty() &&
1392
154k
                !tnode.agg_node.__isset.agg_sort_info_by_group_key &&
1393
154k
                _params.query_options.__isset.enable_distinct_streaming_aggregation &&
1394
154k
                _params.query_options.enable_distinct_streaming_aggregation;
1395
1396
154k
        if (can_use_distinct_streaming_agg) {
1397
91.2k
            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.2k
            } else {
1408
91.2k
                op = std::make_shared<DistinctStreamingAggOperatorX>(pool, next_operator_id(),
1409
91.2k
                                                                     tnode, descs);
1410
91.2k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1411
91.2k
            }
1412
91.2k
        } else if (is_streaming_agg) {
1413
3.08k
            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.08k
            } else {
1423
3.08k
                op = std::make_shared<StreamingAggOperatorX>(pool, next_operator_id(), tnode,
1424
3.08k
                                                             descs);
1425
3.08k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1426
3.08k
            }
1427
60.4k
        } else {
1428
            // create new pipeline to add query cache operator
1429
60.4k
            PipelinePtr new_pipe;
1430
60.4k
            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.4k
            if (enable_spill) {
1436
103
                op = std::make_shared<PartitionedAggSourceOperatorX>(pool, tnode,
1437
103
                                                                     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.4k
            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.4k
            } else {
1446
60.4k
                RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1447
60.4k
            }
1448
1449
60.4k
            const auto downstream_pipeline_id = cur_pipe->id();
1450
60.4k
            if (!_dag.contains(downstream_pipeline_id)) {
1451
58.3k
                _dag.insert({downstream_pipeline_id, {}});
1452
58.3k
            }
1453
60.4k
            cur_pipe = add_pipeline(cur_pipe);
1454
60.4k
            _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1455
1456
60.4k
            if (enable_spill) {
1457
103
                sink_ops.push_back(std::make_shared<PartitionedAggSinkOperatorX>(
1458
103
                        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.4k
            RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1464
60.4k
            RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1465
60.4k
        }
1466
154k
        break;
1467
154k
    }
1468
154k
    case TPlanNodeType::HASH_JOIN_NODE: {
1469
7.65k
        const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join &&
1470
7.65k
                                       tnode.hash_join_node.is_broadcast_join;
1471
7.65k
        const auto enable_spill = _runtime_state->enable_spill();
1472
7.65k
        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.65k
        } else {
1514
7.65k
            op = std::make_shared<HashJoinProbeOperatorX>(pool, tnode, next_operator_id(), descs);
1515
7.65k
            RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1516
1517
7.65k
            const auto downstream_pipeline_id = cur_pipe->id();
1518
7.65k
            if (!_dag.contains(downstream_pipeline_id)) {
1519
6.87k
                _dag.insert({downstream_pipeline_id, {}});
1520
6.87k
            }
1521
7.65k
            PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1522
7.65k
            _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1523
1524
7.65k
            sink_ops.push_back(std::make_shared<HashJoinBuildSinkOperatorX>(
1525
7.65k
                    pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1526
7.65k
            RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1527
7.65k
            RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode, _runtime_state.get()));
1528
1529
7.65k
            _pipeline_parent_map.push(op->node_id(), cur_pipe);
1530
7.65k
            _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1531
7.65k
        }
1532
7.65k
        if (is_broadcast_join && _runtime_state->enable_share_hash_table_for_broadcast_join()) {
1533
2.95k
            std::shared_ptr<HashJoinSharedState> shared_state =
1534
2.95k
                    HashJoinSharedState::create_shared(_num_instances);
1535
24.6k
            for (int i = 0; i < _num_instances; i++) {
1536
21.6k
                auto sink_dep = std::make_shared<Dependency>(op->operator_id(), op->node_id(),
1537
21.6k
                                                             "HASH_JOIN_BUILD_DEPENDENCY");
1538
21.6k
                sink_dep->set_shared_state(shared_state.get());
1539
21.6k
                shared_state->sink_deps.push_back(sink_dep);
1540
21.6k
            }
1541
2.95k
            shared_state->create_source_dependencies(_num_instances, op->operator_id(),
1542
2.95k
                                                     op->node_id(), "HASH_JOIN_PROBE");
1543
2.95k
            _op_id_to_shared_state.insert(
1544
2.95k
                    {op->operator_id(), {shared_state, shared_state->sink_deps}});
1545
2.95k
        }
1546
7.65k
        break;
1547
7.65k
    }
1548
4.51k
    case TPlanNodeType::CROSS_JOIN_NODE: {
1549
4.51k
        op = std::make_shared<NestedLoopJoinProbeOperatorX>(pool, tnode, next_operator_id(), descs);
1550
4.51k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1551
1552
4.51k
        const auto downstream_pipeline_id = cur_pipe->id();
1553
4.51k
        if (!_dag.contains(downstream_pipeline_id)) {
1554
4.28k
            _dag.insert({downstream_pipeline_id, {}});
1555
4.28k
        }
1556
4.51k
        PipelinePtr build_side_pipe = add_pipeline(cur_pipe);
1557
4.51k
        _dag[downstream_pipeline_id].push_back(build_side_pipe->id());
1558
1559
4.51k
        sink_ops.push_back(std::make_shared<NestedLoopJoinBuildSinkOperatorX>(
1560
4.51k
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1561
4.51k
        RETURN_IF_ERROR(build_side_pipe->set_sink(sink_ops.back()));
1562
4.51k
        RETURN_IF_ERROR(build_side_pipe->sink()->init(tnode, _runtime_state.get()));
1563
4.51k
        _pipeline_parent_map.push(op->node_id(), cur_pipe);
1564
4.51k
        _pipeline_parent_map.push(op->node_id(), build_side_pipe);
1565
4.51k
        break;
1566
4.51k
    }
1567
52.1k
    case TPlanNodeType::UNION_NODE: {
1568
52.1k
        int child_count = tnode.num_children;
1569
52.1k
        op = std::make_shared<UnionSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1570
52.1k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1571
1572
52.1k
        const auto downstream_pipeline_id = cur_pipe->id();
1573
52.1k
        if (!_dag.contains(downstream_pipeline_id)) {
1574
51.4k
            _dag.insert({downstream_pipeline_id, {}});
1575
51.4k
        }
1576
53.7k
        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.1k
        break;
1587
52.1k
    }
1588
52.1k
    case TPlanNodeType::SORT_NODE: {
1589
43.8k
        const auto should_spill = _runtime_state->enable_spill() &&
1590
43.8k
                                  tnode.sort_node.algorithm == TSortAlgorithm::FULL_SORT;
1591
43.8k
        const bool use_local_merge =
1592
43.8k
                tnode.sort_node.__isset.use_local_merge && tnode.sort_node.use_local_merge;
1593
43.8k
        if (should_spill) {
1594
9
            op = std::make_shared<SpillSortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1595
43.8k
        } else if (use_local_merge) {
1596
41.6k
            op = std::make_shared<LocalMergeSortSourceOperatorX>(pool, tnode, next_operator_id(),
1597
41.6k
                                                                 descs);
1598
41.6k
        } else {
1599
2.25k
            op = std::make_shared<SortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1600
2.25k
        }
1601
43.8k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1602
1603
43.8k
        const auto downstream_pipeline_id = cur_pipe->id();
1604
43.8k
        if (!_dag.contains(downstream_pipeline_id)) {
1605
43.8k
            _dag.insert({downstream_pipeline_id, {}});
1606
43.8k
        }
1607
43.8k
        cur_pipe = add_pipeline(cur_pipe);
1608
43.8k
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1609
1610
43.8k
        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
43.8k
        } else {
1614
43.8k
            sink_ops.push_back(std::make_shared<SortSinkOperatorX>(
1615
43.8k
                    pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1616
43.8k
        }
1617
43.8k
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1618
43.8k
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1619
43.8k
        break;
1620
43.8k
    }
1621
43.8k
    case TPlanNodeType::PARTITION_SORT_NODE: {
1622
62
        op = std::make_shared<PartitionSortSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1623
62
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1624
1625
62
        const auto downstream_pipeline_id = cur_pipe->id();
1626
62
        if (!_dag.contains(downstream_pipeline_id)) {
1627
62
            _dag.insert({downstream_pipeline_id, {}});
1628
62
        }
1629
62
        cur_pipe = add_pipeline(cur_pipe);
1630
62
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1631
1632
62
        sink_ops.push_back(std::make_shared<PartitionSortSinkOperatorX>(
1633
62
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1634
62
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1635
62
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1636
62
        break;
1637
62
    }
1638
1.56k
    case TPlanNodeType::ANALYTIC_EVAL_NODE: {
1639
1.56k
        op = std::make_shared<AnalyticSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1640
1.56k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1641
1642
1.56k
        const auto downstream_pipeline_id = cur_pipe->id();
1643
1.56k
        if (!_dag.contains(downstream_pipeline_id)) {
1644
1.54k
            _dag.insert({downstream_pipeline_id, {}});
1645
1.54k
        }
1646
1.56k
        cur_pipe = add_pipeline(cur_pipe);
1647
1.56k
        _dag[downstream_pipeline_id].push_back(cur_pipe->id());
1648
1649
1.56k
        sink_ops.push_back(std::make_shared<AnalyticSinkOperatorX>(
1650
1.56k
                pool, next_sink_operator_id(), op->operator_id(), tnode, descs));
1651
1.56k
        RETURN_IF_ERROR(cur_pipe->set_sink(sink_ops.back()));
1652
1.56k
        RETURN_IF_ERROR(cur_pipe->sink()->init(tnode, _runtime_state.get()));
1653
1.56k
        break;
1654
1.56k
    }
1655
1.60k
    case TPlanNodeType::MATERIALIZATION_NODE: {
1656
1.60k
        op = std::make_shared<MaterializationOperator>(pool, tnode, next_operator_id(), descs);
1657
1.60k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1658
1.60k
        break;
1659
1.60k
    }
1660
1.60k
    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
296
    case TPlanNodeType::REPEAT_NODE: {
1671
296
        op = std::make_shared<RepeatOperatorX>(pool, tnode, next_operator_id(), descs);
1672
296
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1673
296
        break;
1674
296
    }
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.57k
    case TPlanNodeType::EMPTY_SET_NODE: {
1686
1.57k
        op = std::make_shared<EmptySetSourceOperatorX>(pool, tnode, next_operator_id(), descs);
1687
1.57k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1688
1.57k
        break;
1689
1.57k
    }
1690
1.57k
    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.29k
    case TPlanNodeType::SCHEMA_SCAN_NODE: {
1697
2.29k
        op = std::make_shared<SchemaScanOperatorX>(pool, tnode, next_operator_id(), descs);
1698
2.29k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1699
2.29k
        break;
1700
2.29k
    }
1701
6.77k
    case TPlanNodeType::META_SCAN_NODE: {
1702
6.77k
        op = std::make_shared<MetaScanOperatorX>(pool, tnode, next_operator_id(), descs);
1703
6.77k
        RETURN_IF_ERROR(cur_pipe->add_operator(op, _parallel_instances));
1704
6.77k
        break;
1705
6.77k
    }
1706
6.77k
    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
656k
    }
1751
654k
    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
654k
    return Status::OK();
1757
656k
}
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
423k
Status PipelineFragmentContext::submit() {
1794
423k
    if (_submitted) {
1795
0
        return Status::InternalError("submitted");
1796
0
    }
1797
423k
    _submitted = true;
1798
1799
423k
    int submit_tasks = 0;
1800
423k
    Status st;
1801
423k
    auto* scheduler = _query_ctx->get_pipe_exec_scheduler();
1802
1.17M
    for (auto& task : _tasks) {
1803
2.08M
        for (auto& t : task) {
1804
2.08M
            st = scheduler->submit(t.first);
1805
2.08M
            DBUG_EXECUTE_IF("PipelineFragmentContext.submit.failed",
1806
2.08M
                            { st = Status::Aborted("PipelineFragmentContext.submit.failed"); });
1807
2.08M
            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.08M
            submit_tasks++;
1814
2.08M
        }
1815
1.17M
    }
1816
423k
    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
423k
    } else {
1831
423k
        return st;
1832
423k
    }
1833
423k
}
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
425k
bool PipelineFragmentContext::_close_fragment_instance() {
1861
425k
    if (_is_fragment_instance_closed) {
1862
0
        return false;
1863
0
    }
1864
425k
    Defer defer_op {[&]() { _is_fragment_instance_closed = true; }};
1865
425k
    _fragment_level_profile->total_time_counter()->update(_fragment_watcher.elapsed_time());
1866
425k
    if (!_need_notify_close) {
1867
421k
        auto st = send_report(true);
1868
421k
        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
421k
    }
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
425k
    if (_runtime_state->enable_profile() &&
1879
425k
        (_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
425k
    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
425k
    return !_need_notify_close;
1908
425k
}
1909
1910
2.07M
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.07M
    DCHECK(_pip_id_to_pipeline.contains(pipeline_id));
1913
2.07M
    if (_pip_id_to_pipeline[pipeline_id]->close_task()) {
1914
666k
        if (_dag.contains(pipeline_id)) {
1915
356k
            for (auto dep : _dag[pipeline_id]) {
1916
356k
                _pip_id_to_pipeline[dep]->make_all_runnable(pipeline_id);
1917
356k
            }
1918
283k
        }
1919
666k
    }
1920
2.07M
    bool need_remove = false;
1921
2.07M
    {
1922
2.07M
        std::lock_guard<std::mutex> l(_task_mutex);
1923
2.07M
        ++_closed_tasks;
1924
2.07M
        if (_closed_tasks >= _total_tasks) {
1925
425k
            need_remove = _close_fragment_instance();
1926
425k
        }
1927
2.07M
    }
1928
    // Call remove_pipeline_context() outside _task_mutex to avoid ABBA deadlock.
1929
2.07M
    if (need_remove) {
1930
421k
        _exec_env->fragment_mgr()->remove_pipeline_context({_query_id, _fragment_id});
1931
421k
    }
1932
2.07M
}
1933
1934
51.9k
std::string PipelineFragmentContext::get_load_error_url() {
1935
51.9k
    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
135k
    for (auto& tasks : _tasks) {
1939
210k
        for (auto& task : tasks) {
1940
210k
            if (const auto& str = task.second->get_error_log_file_path(); !str.empty()) {
1941
162
                return to_load_error_http_path(str);
1942
162
            }
1943
210k
        }
1944
135k
    }
1945
51.7k
    return "";
1946
51.9k
}
1947
1948
51.8k
std::string PipelineFragmentContext::get_first_error_msg() {
1949
51.8k
    if (const auto& str = _runtime_state->get_first_error_msg(); !str.empty()) {
1950
0
        return str;
1951
0
    }
1952
135k
    for (auto& tasks : _tasks) {
1953
210k
        for (auto& task : tasks) {
1954
210k
            if (const auto& str = task.second->get_first_error_msg(); !str.empty()) {
1955
162
                return str;
1956
162
            }
1957
210k
        }
1958
135k
    }
1959
51.7k
    return "";
1960
51.8k
}
1961
1962
426k
Status PipelineFragmentContext::send_report(bool done) {
1963
426k
    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
426k
    if (!_is_report_success && done && exec_status.ok()) {
1969
379k
        return Status::OK();
1970
379k
    }
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
46.8k
    if (!_is_report_success && !_is_report_on_cancel) {
1979
403
        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
403
            return Status::OK();
1982
403
        }
1983
0
        return Status::NeedSendAgain("");
1984
403
    }
1985
1986
46.4k
    std::vector<RuntimeState*> runtime_states;
1987
1988
117k
    for (auto& tasks : _tasks) {
1989
173k
        for (auto& task : tasks) {
1990
173k
            runtime_states.push_back(task.second.get());
1991
173k
        }
1992
117k
    }
1993
1994
46.4k
    std::string load_eror_url = _query_ctx->get_load_error_url().empty()
1995
46.4k
                                        ? get_load_error_url()
1996
46.4k
                                        : _query_ctx->get_load_error_url();
1997
46.4k
    std::string first_error_msg = _query_ctx->get_first_error_msg().empty()
1998
46.4k
                                          ? get_first_error_msg()
1999
46.4k
                                          : _query_ctx->get_first_error_msg();
2000
2001
46.4k
    ReportStatusRequest req {.status = exec_status,
2002
46.4k
                             .runtime_states = runtime_states,
2003
46.4k
                             .done = done || !exec_status.ok(),
2004
46.4k
                             .coord_addr = _query_ctx->coord_addr,
2005
46.4k
                             .query_id = _query_id,
2006
46.4k
                             .fragment_id = _fragment_id,
2007
46.4k
                             .fragment_instance_id = TUniqueId(),
2008
46.4k
                             .backend_num = -1,
2009
46.4k
                             .runtime_state = _runtime_state.get(),
2010
46.4k
                             .load_error_url = load_eror_url,
2011
46.4k
                             .first_error_msg = first_error_msg,
2012
46.4k
                             .cancel_fn = [this](const Status& reason) { cancel(reason); }};
2013
2014
46.4k
    return _report_status_cb(
2015
46.4k
            req, std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this()));
2016
46.8k
}
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
147
std::string PipelineFragmentContext::debug_string() {
2056
147
    std::lock_guard<std::mutex> l(_task_mutex);
2057
147
    fmt::memory_buffer debug_string_buffer;
2058
147
    fmt::format_to(debug_string_buffer,
2059
147
                   "PipelineFragmentContext Info: _closed_tasks={}, _total_tasks={}, "
2060
147
                   "need_notify_close={}, fragment_id={}, _rec_cte_stage={}\n",
2061
147
                   _closed_tasks, _total_tasks, _need_notify_close, _fragment_id, _rec_cte_stage);
2062
1.37k
    for (size_t j = 0; j < _tasks.size(); j++) {
2063
1.22k
        fmt::format_to(debug_string_buffer, "Tasks in instance {}:\n", j);
2064
2.72k
        for (size_t i = 0; i < _tasks[j].size(); i++) {
2065
1.49k
            fmt::format_to(debug_string_buffer, "Task {}: {}\n", i,
2066
1.49k
                           _tasks[j][i].first->debug_string());
2067
1.49k
        }
2068
1.22k
    }
2069
2070
147
    return fmt::to_string(debug_string_buffer);
2071
147
}
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.69k
    for (auto pipeline_profile : _runtime_state->pipeline_id_to_profile()) {
2095
4.69k
        auto profile_ptr = std::make_shared<TRuntimeProfileTree>();
2096
4.69k
        pipeline_profile->to_thrift(profile_ptr.get(), _runtime_state->profile_level());
2097
4.69k
        res.push_back(profile_ptr);
2098
4.69k
    }
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.68k
    for (const auto& tasks : _tasks) {
2117
15.2k
        for (const auto& task : tasks) {
2118
15.2k
            if (task.second->load_channel_profile() == nullptr) {
2119
0
                continue;
2120
0
            }
2121
2122
15.2k
            auto tmp_load_channel_profile = std::make_shared<TRuntimeProfileTree>();
2123
2124
15.2k
            task.second->load_channel_profile()->to_thrift(tmp_load_channel_profile.get(),
2125
15.2k
                                                           _runtime_state->profile_level());
2126
15.2k
            _runtime_state->load_channel_profile()->update(*tmp_load_channel_profile);
2127
15.2k
        }
2128
7.68k
    }
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
8.96k
    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
8.96k
    }
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
426k
void PipelineFragmentContext::_release_resource() {
2159
426k
    std::lock_guard<std::mutex> l(_task_mutex);
2160
    // The memory released by the query end is recorded in the query mem tracker.
2161
426k
    SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_ctx->query_mem_tracker());
2162
426k
    auto st = _query_ctx->exec_status();
2163
1.18M
    for (auto& _task : _tasks) {
2164
1.18M
        if (!_task.empty()) {
2165
1.18M
            _call_back(_task.front().first->runtime_state(), &st);
2166
1.18M
        }
2167
1.18M
    }
2168
426k
    _tasks.clear();
2169
426k
    _dag.clear();
2170
426k
    _pip_id_to_pipeline.clear();
2171
426k
    _pipelines.clear();
2172
426k
    _sink.reset();
2173
426k
    _root_op.reset();
2174
426k
    _runtime_filter_mgr_map.clear();
2175
426k
    _op_id_to_shared_state.clear();
2176
426k
}
2177
2178
#include "common/compile_check_end.h"
2179
} // namespace doris