Coverage Report

Created: 2026-04-04 07:17

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