Coverage Report

Created: 2026-04-17 14:13

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