Coverage Report

Created: 2026-05-09 09:50

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