Coverage Report

Created: 2026-04-15 19:01

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