Coverage Report

Created: 2026-01-07 23:46

next uncovered line (L), next uncovered region (R), next uncovered branch (B)
/root/doris/be/src/pipeline/pipeline_task.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 "pipeline_task.h"
19
20
#include <fmt/core.h>
21
#include <fmt/format.h>
22
#include <gen_cpp/Metrics_types.h>
23
#include <glog/logging.h>
24
25
#include <algorithm>
26
#include <memory>
27
#include <ostream>
28
#include <vector>
29
30
#include "common/logging.h"
31
#include "common/status.h"
32
#include "pipeline/dependency.h"
33
#include "pipeline/exec/exchange_source_operator.h"
34
#include "pipeline/exec/operator.h"
35
#include "pipeline/exec/rec_cte_source_operator.h"
36
#include "pipeline/exec/scan_operator.h"
37
#include "pipeline/pipeline.h"
38
#include "pipeline/pipeline_fragment_context.h"
39
#include "pipeline/task_queue.h"
40
#include "pipeline/task_scheduler.h"
41
#include "revokable_task.h"
42
#include "runtime/descriptors.h"
43
#include "runtime/exec_env.h"
44
#include "runtime/query_context.h"
45
#include "runtime/thread_context.h"
46
#include "runtime/workload_group/workload_group_manager.h"
47
#include "util/defer_op.h"
48
#include "util/mem_info.h"
49
#include "util/runtime_profile.h"
50
#include "util/uid_util.h"
51
#include "vec/core/block.h"
52
#include "vec/spill/spill_stream.h"
53
54
namespace doris {
55
class RuntimeState;
56
} // namespace doris
57
58
namespace doris::pipeline {
59
#include "common/compile_check_begin.h"
60
61
PipelineTask::PipelineTask(PipelinePtr& pipeline, uint32_t task_id, RuntimeState* state,
62
                           std::shared_ptr<PipelineFragmentContext> fragment_context,
63
                           RuntimeProfile* parent_profile,
64
                           std::map<int, std::pair<std::shared_ptr<BasicSharedState>,
65
                                                   std::vector<std::shared_ptr<Dependency>>>>
66
                                   shared_state_map,
67
                           int task_idx)
68
        :
69
#ifdef BE_TEST
70
72.1k
          _query_id(fragment_context ? fragment_context->get_query_id() : TUniqueId()),
71
#else
72
          _query_id(fragment_context->get_query_id()),
73
#endif
74
72.1k
          _index(task_id),
75
72.1k
          _pipeline(pipeline),
76
72.1k
          _opened(false),
77
72.1k
          _state(state),
78
72.1k
          _fragment_context(fragment_context),
79
72.1k
          _parent_profile(parent_profile),
80
72.1k
          _operators(pipeline->operators()),
81
72.1k
          _source(_operators.front().get()),
82
72.1k
          _root(_operators.back().get()),
83
72.1k
          _sink(pipeline->sink_shared_pointer()),
84
72.1k
          _shared_state_map(std::move(shared_state_map)),
85
72.1k
          _task_idx(task_idx),
86
72.1k
          _memory_sufficient_dependency(state->get_query_ctx()->get_memory_sufficient_dependency()),
87
72.1k
          _pipeline_name(_pipeline->name()) {
88
#ifndef BE_TEST
89
    _query_mem_tracker = fragment_context->get_query_ctx()->query_mem_tracker();
90
#endif
91
72.1k
    _execution_dependencies.push_back(state->get_query_ctx()->get_execution_dependency());
92
72.1k
    if (!_shared_state_map.contains(_sink->dests_id().front())) {
93
72.1k
        auto shared_state = _sink->create_shared_state();
94
72.1k
        if (shared_state) {
95
29
            _sink_shared_state = shared_state;
96
29
        }
97
72.1k
    }
98
72.1k
}
99
100
72.1k
PipelineTask::~PipelineTask() {
101
72.1k
    auto reset_member = [&]() {
102
72.1k
        _shared_state_map.clear();
103
72.1k
        _sink_shared_state.reset();
104
72.1k
        _op_shared_states.clear();
105
72.1k
        _sink.reset();
106
72.1k
        _operators.clear();
107
72.1k
        _block.reset();
108
72.1k
        _pipeline.reset();
109
72.1k
    };
110
// PipelineTask is also hold by task queue( https://github.com/apache/doris/pull/49753),
111
// so that it maybe the last one to be destructed.
112
// But pipeline task hold some objects, like operators, shared state, etc. So that should release
113
// memory manually.
114
#ifndef BE_TEST
115
    if (_query_mem_tracker) {
116
        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_mem_tracker);
117
        reset_member();
118
        return;
119
    }
120
#endif
121
72.1k
    reset_member();
122
72.1k
}
123
124
Status PipelineTask::prepare(const std::vector<TScanRangeParams>& scan_range, const int sender_id,
125
16
                             const TDataSink& tsink) {
126
16
    DCHECK(_sink);
127
16
    _init_profile();
128
16
    SCOPED_TIMER(_task_profile->total_time_counter());
129
16
    SCOPED_CPU_TIMER(_task_cpu_timer);
130
16
    SCOPED_TIMER(_prepare_timer);
131
16
    DBUG_EXECUTE_IF("fault_inject::PipelineXTask::prepare", {
132
16
        Status status = Status::Error<INTERNAL_ERROR>("fault_inject pipeline_task prepare failed");
133
16
        return status;
134
16
    });
135
16
    {
136
        // set sink local state
137
16
        LocalSinkStateInfo info {_task_idx,         _task_profile.get(),
138
16
                                 sender_id,         get_sink_shared_state().get(),
139
16
                                 _shared_state_map, tsink};
140
16
        RETURN_IF_ERROR(_sink->setup_local_state(_state, info));
141
16
    }
142
143
16
    _scan_ranges = scan_range;
144
16
    auto* parent_profile = _state->get_sink_local_state()->operator_profile();
145
146
34
    for (int op_idx = cast_set<int>(_operators.size() - 1); op_idx >= 0; op_idx--) {
147
18
        auto& op = _operators[op_idx];
148
18
        LocalStateInfo info {parent_profile, _scan_ranges, get_op_shared_state(op->operator_id()),
149
18
                             _shared_state_map, _task_idx};
150
18
        RETURN_IF_ERROR(op->setup_local_state(_state, info));
151
18
        parent_profile = _state->get_local_state(op->operator_id())->operator_profile();
152
18
    }
153
16
    {
154
16
        const auto& deps =
155
16
                _state->get_local_state(_source->operator_id())->execution_dependencies();
156
16
        std::unique_lock<std::mutex> lc(_dependency_lock);
157
16
        std::copy(deps.begin(), deps.end(),
158
16
                  std::inserter(_execution_dependencies, _execution_dependencies.end()));
159
16
    }
160
16
    if (auto fragment = _fragment_context.lock()) {
161
15
        if (fragment->get_query_ctx()->is_cancelled()) {
162
0
            terminate();
163
0
            return fragment->get_query_ctx()->exec_status();
164
0
        }
165
15
    } else {
166
1
        return Status::InternalError("Fragment already finished! Query: {}", print_id(_query_id));
167
1
    }
168
15
    _block = doris::vectorized::Block::create_unique();
169
15
    return _state_transition(State::RUNNABLE);
170
16
}
171
172
13
Status PipelineTask::_extract_dependencies() {
173
13
    std::vector<std::vector<Dependency*>> read_dependencies;
174
13
    std::vector<Dependency*> write_dependencies;
175
13
    std::vector<Dependency*> finish_dependencies;
176
13
    read_dependencies.resize(_operators.size());
177
13
    size_t i = 0;
178
15
    for (auto& op : _operators) {
179
15
        auto* local_state = _state->get_local_state(op->operator_id());
180
15
        DCHECK(local_state);
181
15
        read_dependencies[i] = local_state->dependencies();
182
15
        auto* fin_dep = local_state->finishdependency();
183
15
        if (fin_dep) {
184
8
            finish_dependencies.push_back(fin_dep);
185
8
        }
186
15
        i++;
187
15
    }
188
13
    DBUG_EXECUTE_IF("fault_inject::PipelineXTask::_extract_dependencies", {
189
13
        Status status = Status::Error<INTERNAL_ERROR>(
190
13
                "fault_inject pipeline_task _extract_dependencies failed");
191
13
        return status;
192
13
    });
193
13
    {
194
13
        auto* local_state = _state->get_sink_local_state();
195
13
        write_dependencies = local_state->dependencies();
196
13
        auto* fin_dep = local_state->finishdependency();
197
13
        if (fin_dep) {
198
13
            finish_dependencies.push_back(fin_dep);
199
13
        }
200
13
    }
201
13
    {
202
13
        std::unique_lock<std::mutex> lc(_dependency_lock);
203
13
        read_dependencies.swap(_read_dependencies);
204
13
        write_dependencies.swap(_write_dependencies);
205
13
        finish_dependencies.swap(_finish_dependencies);
206
13
    }
207
13
    return Status::OK();
208
13
}
209
210
6
bool PipelineTask::inject_shared_state(std::shared_ptr<BasicSharedState> shared_state) {
211
6
    if (!shared_state) {
212
1
        return false;
213
1
    }
214
    // Shared state is created by upstream task's sink operator and shared by source operator of
215
    // this task.
216
7
    for (auto& op : _operators) {
217
7
        if (shared_state->related_op_ids.contains(op->operator_id())) {
218
3
            _op_shared_states.insert({op->operator_id(), shared_state});
219
3
            return true;
220
3
        }
221
7
    }
222
    // Shared state is created by the first sink operator and shared by sink operator of this task.
223
    // For example, Set operations.
224
2
    if (shared_state->related_op_ids.contains(_sink->dests_id().front())) {
225
1
        DCHECK_EQ(_sink_shared_state, nullptr)
226
0
                << " Sink: " << _sink->get_name() << " dest id: " << _sink->dests_id().front();
227
1
        _sink_shared_state = shared_state;
228
1
        return true;
229
1
    }
230
1
    return false;
231
2
}
232
233
16
void PipelineTask::_init_profile() {
234
16
    _task_profile = std::make_unique<RuntimeProfile>(fmt::format("PipelineTask(index={})", _index));
235
16
    _parent_profile->add_child(_task_profile.get(), true, nullptr);
236
16
    _task_cpu_timer = ADD_TIMER(_task_profile, "TaskCpuTime");
237
238
16
    static const char* exec_time = "ExecuteTime";
239
16
    _exec_timer = ADD_TIMER(_task_profile, exec_time);
240
16
    _prepare_timer = ADD_CHILD_TIMER(_task_profile, "PrepareTime", exec_time);
241
16
    _open_timer = ADD_CHILD_TIMER(_task_profile, "OpenTime", exec_time);
242
16
    _get_block_timer = ADD_CHILD_TIMER(_task_profile, "GetBlockTime", exec_time);
243
16
    _get_block_counter = ADD_COUNTER(_task_profile, "GetBlockCounter", TUnit::UNIT);
244
16
    _sink_timer = ADD_CHILD_TIMER(_task_profile, "SinkTime", exec_time);
245
16
    _close_timer = ADD_CHILD_TIMER(_task_profile, "CloseTime", exec_time);
246
247
16
    _wait_worker_timer = ADD_TIMER_WITH_LEVEL(_task_profile, "WaitWorkerTime", 1);
248
249
16
    _schedule_counts = ADD_COUNTER(_task_profile, "NumScheduleTimes", TUnit::UNIT);
250
16
    _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT);
251
16
    _core_change_times = ADD_COUNTER(_task_profile, "CoreChangeTimes", TUnit::UNIT);
252
16
    _memory_reserve_times = ADD_COUNTER(_task_profile, "MemoryReserveTimes", TUnit::UNIT);
253
16
    _memory_reserve_failed_times =
254
16
            ADD_COUNTER(_task_profile, "MemoryReserveFailedTimes", TUnit::UNIT);
255
16
}
256
257
6
void PipelineTask::_fresh_profile_counter() {
258
6
    COUNTER_SET(_schedule_counts, (int64_t)_schedule_time);
259
6
    COUNTER_SET(_wait_worker_timer, (int64_t)_wait_worker_watcher.elapsed_time());
260
6
}
261
262
13
Status PipelineTask::_open() {
263
13
    SCOPED_TIMER(_task_profile->total_time_counter());
264
13
    SCOPED_CPU_TIMER(_task_cpu_timer);
265
13
    SCOPED_TIMER(_open_timer);
266
13
    _dry_run = _sink->should_dry_run(_state);
267
15
    for (auto& o : _operators) {
268
15
        RETURN_IF_ERROR(_state->get_local_state(o->operator_id())->open(_state));
269
15
    }
270
13
    RETURN_IF_ERROR(_state->get_sink_local_state()->open(_state));
271
13
    RETURN_IF_ERROR(_extract_dependencies());
272
13
    DBUG_EXECUTE_IF("fault_inject::PipelineXTask::open", {
273
13
        Status status = Status::Error<INTERNAL_ERROR>("fault_inject pipeline_task open failed");
274
13
        return status;
275
13
    });
276
13
    _opened = true;
277
13
    return Status::OK();
278
13
}
279
280
66
Status PipelineTask::_prepare() {
281
66
    SCOPED_TIMER(_task_profile->total_time_counter());
282
66
    SCOPED_CPU_TIMER(_task_cpu_timer);
283
78
    for (auto& o : _operators) {
284
78
        RETURN_IF_ERROR(_state->get_local_state(o->operator_id())->prepare(_state));
285
78
    }
286
66
    RETURN_IF_ERROR(_state->get_sink_local_state()->prepare(_state));
287
66
    return Status::OK();
288
66
}
289
290
44
bool PipelineTask::_wait_to_start() {
291
    // Before task starting, we should make sure
292
    // 1. Execution dependency is ready (which is controlled by FE 2-phase commit)
293
    // 2. Runtime filter dependencies are ready
294
    // 3. All tablets are loaded into local storage
295
44
    return std::any_of(
296
44
            _execution_dependencies.begin(), _execution_dependencies.end(),
297
60
            [&](Dependency* dep) -> bool { return dep->is_blocked_by(shared_from_this()); });
298
44
}
299
300
16
bool PipelineTask::_is_pending_finish() {
301
    // Spilling may be in progress if eos is true.
302
21
    return std::ranges::any_of(_finish_dependencies, [&](Dependency* dep) -> bool {
303
21
        return dep->is_blocked_by(shared_from_this());
304
21
    });
305
16
}
306
307
0
bool PipelineTask::is_blockable() const {
308
    // Before task starting, we should make sure
309
    // 1. Execution dependency is ready (which is controlled by FE 2-phase commit)
310
    // 2. Runtime filter dependencies are ready
311
    // 3. All tablets are loaded into local storage
312
313
0
    if (_state->enable_fuzzy_blockable_task()) {
314
0
        if ((_schedule_time + _task_idx) % 2 == 0) {
315
0
            return true;
316
0
        }
317
0
    }
318
319
0
    return std::ranges::any_of(_operators,
320
0
                               [&](OperatorPtr op) -> bool { return op->is_blockable(_state); }) ||
321
0
           _sink->is_blockable(_state);
322
0
}
323
324
439k
bool PipelineTask::_is_blocked() {
325
    // `_dry_run = true` means we do not need data from source operator.
326
439k
    if (!_dry_run) {
327
879k
        for (int i = cast_set<int>(_read_dependencies.size() - 1); i >= 0; i--) {
328
            // `_read_dependencies` is organized according to operators. For each operator, running condition is met iff all dependencies are ready.
329
439k
            for (auto* dep : _read_dependencies[i]) {
330
439k
                if (dep->is_blocked_by(shared_from_this())) {
331
15
                    return true;
332
15
                }
333
439k
            }
334
            // If all dependencies are ready for this operator, we can execute this task if no datum is needed from upstream operators.
335
439k
            if (!_operators[i]->need_more_input_data(_state)) {
336
2
                break;
337
2
            }
338
439k
        }
339
439k
    }
340
439k
    return _memory_sufficient_dependency->is_blocked_by(shared_from_this()) ||
341
439k
           std::ranges::any_of(_write_dependencies, [&](Dependency* dep) -> bool {
342
439k
               return dep->is_blocked_by(shared_from_this());
343
439k
           });
344
439k
}
345
346
4
void PipelineTask::terminate() {
347
    // We use a lock to assure all dependencies are not deconstructed here.
348
4
    std::unique_lock<std::mutex> lc(_dependency_lock);
349
4
    auto fragment = _fragment_context.lock();
350
4
    if (!is_finalized() && fragment) {
351
4
        try {
352
4
            DCHECK(_wake_up_early || fragment->is_canceled());
353
4
            std::ranges::for_each(_write_dependencies,
354
4
                                  [&](Dependency* dep) { dep->set_always_ready(); });
355
4
            std::ranges::for_each(_finish_dependencies,
356
8
                                  [&](Dependency* dep) { dep->set_always_ready(); });
357
4
            std::ranges::for_each(_read_dependencies, [&](std::vector<Dependency*>& deps) {
358
4
                std::ranges::for_each(deps, [&](Dependency* dep) { dep->set_always_ready(); });
359
4
            });
360
            // All `_execution_deps` will never be set blocking from ready. So we just set ready here.
361
4
            std::ranges::for_each(_execution_dependencies,
362
8
                                  [&](Dependency* dep) { dep->set_ready(); });
363
4
            _memory_sufficient_dependency->set_ready();
364
4
        } catch (const doris::Exception& e) {
365
0
            LOG(WARNING) << "Terminate failed: " << e.code() << ", " << e.to_string();
366
0
        }
367
4
    }
368
4
}
369
370
/**
371
 * `_eos` indicates whether the execution phase is done. `done` indicates whether we could close
372
 * this task.
373
 *
374
 * For example,
375
 * 1. if `_eos` is false which means we should continue to get next block so we cannot close (e.g.
376
 *    `done` is false)
377
 * 2. if `_eos` is true which means all blocks from source are exhausted but `_is_pending_finish()`
378
 *    is true which means we should wait for a pending dependency ready (maybe a running rpc), so we
379
 *    cannot close (e.g. `done` is false)
380
 * 3. if `_eos` is true which means all blocks from source are exhausted and `_is_pending_finish()`
381
 *    is false which means we can close immediately (e.g. `done` is true)
382
 * @param done
383
 * @return
384
 */
385
37
Status PipelineTask::execute(bool* done) {
386
37
    if (_exec_state != State::RUNNABLE || _blocked_dep != nullptr) [[unlikely]] {
387
1
#ifdef BE_TEST
388
1
        return Status::InternalError("Pipeline task is not runnable! Task info: {}",
389
1
                                     debug_string());
390
#else
391
        return Status::FatalError("Pipeline task is not runnable! Task info: {}", debug_string());
392
#endif
393
1
    }
394
395
36
    auto fragment_context = _fragment_context.lock();
396
36
    if (!fragment_context) {
397
0
        return Status::InternalError("Fragment already finished! Query: {}", print_id(_query_id));
398
0
    }
399
36
    int64_t time_spent = 0;
400
36
    ThreadCpuStopWatch cpu_time_stop_watch;
401
36
    cpu_time_stop_watch.start();
402
36
    SCOPED_ATTACH_TASK(_state);
403
36
    Defer running_defer {[&]() {
404
36
        int64_t delta_cpu_time = cpu_time_stop_watch.elapsed_time();
405
36
        _task_cpu_timer->update(delta_cpu_time);
406
36
        fragment_context->get_query_ctx()->resource_ctx()->cpu_context()->update_cpu_cost_ms(
407
36
                delta_cpu_time);
408
409
        // If task is woke up early, we should terminate all operators, and this task could be closed immediately.
410
36
        if (_wake_up_early) {
411
3
            terminate();
412
3
            THROW_IF_ERROR(_root->terminate(_state));
413
3
            THROW_IF_ERROR(_sink->terminate(_state));
414
3
            _eos = true;
415
3
            *done = true;
416
33
        } else if (_eos && !_spilling &&
417
33
                   (fragment_context->is_canceled() || !_is_pending_finish())) {
418
10
            *done = true;
419
10
        }
420
36
    }};
421
36
    const auto query_id = _state->query_id();
422
    // If this task is already EOS and block is empty (which means we already output all blocks),
423
    // just return here.
424
36
    if (_eos && !_spilling) {
425
2
        return Status::OK();
426
2
    }
427
    // If this task is blocked by a spilling request and waken up immediately, the spilling
428
    // dependency will not block this task and we should just run here.
429
34
    if (!_block->empty()) {
430
0
        LOG(INFO) << "Query: " << print_id(query_id) << " has pending block, size: "
431
0
                  << PrettyPrinter::print_bytes(_block->allocated_bytes());
432
0
        DCHECK(_spilling);
433
0
    }
434
435
34
    SCOPED_TIMER(_task_profile->total_time_counter());
436
34
    SCOPED_TIMER(_exec_timer);
437
438
34
    if (!_wake_up_early) {
439
34
        RETURN_IF_ERROR(_prepare());
440
34
    }
441
34
    DBUG_EXECUTE_IF("fault_inject::PipelineXTask::execute", {
442
34
        Status status = Status::Error<INTERNAL_ERROR>("fault_inject pipeline_task execute failed");
443
34
        return status;
444
34
    });
445
    // `_wake_up_early` must be after `_wait_to_start()`
446
34
    if (_wait_to_start() || _wake_up_early) {
447
2
        return Status::OK();
448
2
    }
449
32
    RETURN_IF_ERROR(_prepare());
450
451
    // The status must be runnable
452
32
    if (!_opened && !fragment_context->is_canceled()) {
453
12
        DBUG_EXECUTE_IF("PipelineTask::execute.open_sleep", {
454
12
            auto required_pipeline_id =
455
12
                    DebugPoints::instance()->get_debug_param_or_default<int32_t>(
456
12
                            "PipelineTask::execute.open_sleep", "pipeline_id", -1);
457
12
            auto required_task_id = DebugPoints::instance()->get_debug_param_or_default<int32_t>(
458
12
                    "PipelineTask::execute.open_sleep", "task_id", -1);
459
12
            if (required_pipeline_id == pipeline_id() && required_task_id == task_id()) {
460
12
                LOG(WARNING) << "PipelineTask::execute.open_sleep sleep 5s";
461
12
                sleep(5);
462
12
            }
463
12
        });
464
465
12
        SCOPED_RAW_TIMER(&time_spent);
466
12
        RETURN_IF_ERROR(_open());
467
12
    }
468
469
439k
    while (!fragment_context->is_canceled()) {
470
439k
        SCOPED_RAW_TIMER(&time_spent);
471
439k
        Defer defer {[&]() {
472
            // If this run is pended by a spilling request, the block will be output in next run.
473
439k
            if (!_spilling) {
474
437k
                _block->clear_column_data(_root->row_desc().num_materialized_slots());
475
437k
            }
476
439k
        }};
477
        // `_wake_up_early` must be after `_is_blocked()`
478
439k
        if (_is_blocked() || _wake_up_early) {
479
17
            return Status::OK();
480
17
        }
481
482
        /// When a task is cancelled,
483
        /// its blocking state will be cleared and it will transition to a ready state (though it is not truly ready).
484
        /// Here, checking whether it is cancelled to prevent tasks in a blocking state from being re-executed.
485
439k
        if (fragment_context->is_canceled()) {
486
0
            break;
487
0
        }
488
489
439k
        if (time_spent > _exec_time_slice) {
490
4
            COUNTER_UPDATE(_yield_counts, 1);
491
4
            break;
492
4
        }
493
439k
        auto* block = _block.get();
494
495
439k
        DBUG_EXECUTE_IF("fault_inject::PipelineXTask::executing", {
496
439k
            Status status =
497
439k
                    Status::Error<INTERNAL_ERROR>("fault_inject pipeline_task executing failed");
498
439k
            return status;
499
439k
        });
500
501
        // `_sink->is_finished(_state)` means sink operator should be finished
502
439k
        if (_sink->is_finished(_state)) {
503
1
            set_wake_up_early();
504
1
            return Status::OK();
505
1
        }
506
507
        // `_dry_run` means sink operator need no more data
508
439k
        _eos = _dry_run || _eos;
509
439k
        _spilling = false;
510
439k
        auto workload_group = _state->workload_group();
511
        // If last run is pended by a spilling request, `_block` is produced with some rows in last
512
        // run, so we will resume execution using the block.
513
439k
        if (!_eos && _block->empty()) {
514
438k
            SCOPED_TIMER(_get_block_timer);
515
438k
            if (_state->low_memory_mode()) {
516
0
                _sink->set_low_memory_mode(_state);
517
0
                _root->set_low_memory_mode(_state);
518
0
            }
519
438k
            DEFER_RELEASE_RESERVED();
520
438k
            _get_block_counter->update(1);
521
438k
            const auto reserve_size = _root->get_reserve_mem_size(_state);
522
438k
            _root->reset_reserve_mem_size(_state);
523
524
438k
            if (workload_group &&
525
438k
                _state->get_query_ctx()
526
43.1k
                        ->resource_ctx()
527
43.1k
                        ->task_controller()
528
43.1k
                        ->is_enable_reserve_memory() &&
529
438k
                reserve_size > 0) {
530
917
                if (!_try_to_reserve_memory(reserve_size, _root)) {
531
915
                    continue;
532
915
                }
533
917
            }
534
535
437k
            bool eos = false;
536
437k
            RETURN_IF_ERROR(_root->get_block_after_projects(_state, block, &eos));
537
437k
            RETURN_IF_ERROR(block->check_type_and_column());
538
437k
            _eos = eos;
539
437k
        }
540
541
438k
        if (!_block->empty() || _eos) {
542
933
            SCOPED_TIMER(_sink_timer);
543
933
            Status status = Status::OK();
544
933
            DEFER_RELEASE_RESERVED();
545
933
            if (_state->get_query_ctx()
546
933
                        ->resource_ctx()
547
933
                        ->task_controller()
548
933
                        ->is_enable_reserve_memory() &&
549
933
                workload_group && !(_wake_up_early || _dry_run)) {
550
918
                const auto sink_reserve_size = _sink->get_reserve_mem_size(_state, _eos);
551
918
                if (sink_reserve_size > 0 &&
552
918
                    !_try_to_reserve_memory(sink_reserve_size, _sink.get())) {
553
917
                    continue;
554
917
                }
555
918
            }
556
557
16
            if (_eos && !_sink->need_rerun(_state)) {
558
10
                RETURN_IF_ERROR(close(Status::OK(), false));
559
10
            }
560
561
16
            DBUG_EXECUTE_IF("PipelineTask::execute.sink_eos_sleep", {
562
16
                auto required_pipeline_id =
563
16
                        DebugPoints::instance()->get_debug_param_or_default<int32_t>(
564
16
                                "PipelineTask::execute.sink_eos_sleep", "pipeline_id", -1);
565
16
                auto required_task_id =
566
16
                        DebugPoints::instance()->get_debug_param_or_default<int32_t>(
567
16
                                "PipelineTask::execute.sink_eos_sleep", "task_id", -1);
568
16
                if (required_pipeline_id == pipeline_id() && required_task_id == task_id()) {
569
16
                    LOG(WARNING) << "PipelineTask::execute.sink_eos_sleep sleep 10s";
570
16
                    sleep(10);
571
16
                }
572
16
            });
573
574
16
            DBUG_EXECUTE_IF("PipelineTask::execute.terminate", {
575
16
                if (_eos) {
576
16
                    auto required_pipeline_id =
577
16
                            DebugPoints::instance()->get_debug_param_or_default<int32_t>(
578
16
                                    "PipelineTask::execute.terminate", "pipeline_id", -1);
579
16
                    auto required_task_id =
580
16
                            DebugPoints::instance()->get_debug_param_or_default<int32_t>(
581
16
                                    "PipelineTask::execute.terminate", "task_id", -1);
582
16
                    auto required_fragment_id =
583
16
                            DebugPoints::instance()->get_debug_param_or_default<int32_t>(
584
16
                                    "PipelineTask::execute.terminate", "fragment_id", -1);
585
16
                    if (required_pipeline_id == pipeline_id() && required_task_id == task_id() &&
586
16
                        fragment_context->get_fragment_id() == required_fragment_id) {
587
16
                        _wake_up_early = true;
588
16
                        terminate();
589
16
                    } else if (required_pipeline_id == pipeline_id() &&
590
16
                               fragment_context->get_fragment_id() == required_fragment_id) {
591
16
                        LOG(WARNING) << "PipelineTask::execute.terminate sleep 5s";
592
16
                        sleep(5);
593
16
                    }
594
16
                }
595
16
            });
596
16
            RETURN_IF_ERROR(block->check_type_and_column());
597
16
            status = _sink->sink(_state, block, _eos);
598
599
16
            if (_eos) {
600
10
                if (_sink->need_rerun(_state)) {
601
0
                    if (auto* source = dynamic_cast<ExchangeSourceOperatorX*>(_root);
602
0
                        source != nullptr) {
603
0
                        RETURN_IF_ERROR(source->reset(_state));
604
0
                        _eos = false;
605
0
                    } else {
606
0
                        return Status::InternalError(
607
0
                                "Only ExchangeSourceOperatorX can be rerun, real is {}",
608
0
                                _root->get_name());
609
0
                    }
610
0
                }
611
10
            }
612
613
16
            if (status.is<ErrorCode::END_OF_FILE>()) {
614
1
                set_wake_up_early();
615
1
                return Status::OK();
616
15
            } else if (!status) {
617
0
                return status;
618
0
            }
619
620
15
            if (_eos) { // just return, the scheduler will do finish work
621
9
                return Status::OK();
622
9
            }
623
15
        }
624
438k
    }
625
626
4
    RETURN_IF_ERROR(_state->get_query_ctx()->get_pipe_exec_scheduler()->submit(shared_from_this()));
627
4
    return Status::OK();
628
4
}
629
630
0
Status PipelineTask::do_revoke_memory(const std::shared_ptr<SpillContext>& spill_context) {
631
0
    auto fragment_context = _fragment_context.lock();
632
0
    if (!fragment_context) {
633
0
        return Status::InternalError("Fragment already finished! Query: {}", print_id(_query_id));
634
0
    }
635
636
0
    SCOPED_ATTACH_TASK(_state);
637
0
    ThreadCpuStopWatch cpu_time_stop_watch;
638
0
    cpu_time_stop_watch.start();
639
0
    Defer running_defer {[&]() {
640
0
        int64_t delta_cpu_time = cpu_time_stop_watch.elapsed_time();
641
0
        _task_cpu_timer->update(delta_cpu_time);
642
0
        fragment_context->get_query_ctx()->resource_ctx()->cpu_context()->update_cpu_cost_ms(
643
0
                delta_cpu_time);
644
645
        // If task is woke up early, we should terminate all operators, and this task could be closed immediately.
646
0
        if (_wake_up_early) {
647
0
            terminate();
648
0
            THROW_IF_ERROR(_root->terminate(_state));
649
0
            THROW_IF_ERROR(_sink->terminate(_state));
650
0
            _eos = true;
651
0
        }
652
0
    }};
653
654
0
    return _sink->revoke_memory(_state, spill_context);
655
0
}
656
657
1.83k
bool PipelineTask::_try_to_reserve_memory(const size_t reserve_size, OperatorBase* op) {
658
1.83k
    auto st = thread_context()->thread_mem_tracker_mgr->try_reserve(reserve_size);
659
    // If reserve memory failed and the query is not enable spill, just disable reserve memory(this will enable
660
    // memory hard limit check, and will cancel the query if allocate memory failed) and let it run.
661
1.83k
    if (!st.ok() && !_state->enable_spill()) {
662
2
        LOG(INFO) << print_id(_query_id) << " reserve memory failed due to " << st
663
2
                  << ", and it is not enable spill, disable reserve memory and let it run";
664
2
        _state->get_query_ctx()->resource_ctx()->task_controller()->disable_reserve_memory();
665
2
        return true;
666
2
    }
667
1.83k
    COUNTER_UPDATE(_memory_reserve_times, 1);
668
1.83k
    auto sink_revocable_mem_size = _sink->revocable_mem_size(_state);
669
1.83k
    if (st.ok() && _state->enable_force_spill() && _sink->is_spillable() &&
670
1.83k
        sink_revocable_mem_size >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) {
671
0
        st = Status(ErrorCode::QUERY_MEMORY_EXCEEDED, "Force Spill");
672
0
    }
673
1.83k
    if (!st.ok()) {
674
1.83k
        COUNTER_UPDATE(_memory_reserve_failed_times, 1);
675
1.83k
        auto debug_msg = fmt::format(
676
1.83k
                "Query: {} , try to reserve: {}, operator name: {}, operator "
677
1.83k
                "id: {}, task id: {}, root revocable mem size: {}, sink revocable mem"
678
1.83k
                "size: {}, failed: {}",
679
1.83k
                print_id(_query_id), PrettyPrinter::print_bytes(reserve_size), op->get_name(),
680
1.83k
                op->node_id(), _state->task_id(),
681
1.83k
                PrettyPrinter::print_bytes(op->revocable_mem_size(_state)),
682
1.83k
                PrettyPrinter::print_bytes(sink_revocable_mem_size), st.to_string());
683
        // PROCESS_MEMORY_EXCEEDED error msg already contains process_mem_log_str
684
1.83k
        if (!st.is<ErrorCode::PROCESS_MEMORY_EXCEEDED>()) {
685
1.83k
            debug_msg +=
686
1.83k
                    fmt::format(", debug info: {}", GlobalMemoryArbitrator::process_mem_log_str());
687
1.83k
        }
688
        // If sink has enough revocable memory, trigger revoke memory
689
1.83k
        LOG(INFO) << fmt::format(
690
1.83k
                "Query: {} sink: {}, node id: {}, task id: "
691
1.83k
                "{}, revocable mem size: {}",
692
1.83k
                print_id(_query_id), _sink->get_name(), _sink->node_id(), _state->task_id(),
693
1.83k
                PrettyPrinter::print_bytes(sink_revocable_mem_size));
694
1.83k
        ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query(
695
1.83k
                _state->get_query_ctx()->resource_ctx()->shared_from_this(), reserve_size, st);
696
1.83k
        _spilling = true;
697
1.83k
        return false;
698
        // !!! Attention:
699
        // In the past, if reserve failed, not add this query to paused list, because it is very small, will not
700
        // consume a lot of memory. But need set low memory mode to indicate that the system should
701
        // not use too much memory.
702
        // But if we only set _state->get_query_ctx()->set_low_memory_mode() here, and return true, the query will
703
        // continue to run and not blocked, and this reserve maybe the last block of join sink opertorator, and it will
704
        // build hash table directly and will consume a lot of memory. So that should return false directly.
705
        // TODO: we should using a global system buffer management logic to deal with low memory mode.
706
        /**
707
        if (sink_revocable_mem_size >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) {
708
            LOG(INFO) << fmt::format(
709
                    "Query: {} sink: {}, node id: {}, task id: "
710
                    "{}, revocable mem size: {}",
711
                    print_id(_query_id), _sink->get_name(), _sink->node_id(), _state->task_id(),
712
                    PrettyPrinter::print_bytes(sink_revocable_mem_size));
713
            ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query(
714
                    _state->get_query_ctx()->resource_ctx()->shared_from_this(), reserve_size, st);
715
            _spilling = true;
716
            return false;
717
        } else {
718
            _state->get_query_ctx()->set_low_memory_mode();
719
        } */
720
1.83k
    }
721
0
    return true;
722
1.83k
}
723
724
210k
void PipelineTask::stop_if_finished() {
725
210k
    auto fragment = _fragment_context.lock();
726
210k
    if (!fragment) {
727
0
        return;
728
0
    }
729
210k
    SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(fragment->get_query_ctx()->query_mem_tracker());
730
210k
    if (auto sink = _sink) {
731
210k
        if (sink->is_finished(_state)) {
732
0
            set_wake_up_early();
733
0
            terminate();
734
0
        }
735
210k
    }
736
210k
}
737
738
1
Status PipelineTask::finalize() {
739
1
    auto fragment = _fragment_context.lock();
740
1
    if (!fragment) {
741
0
        return Status::OK();
742
0
    }
743
1
    SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(fragment->get_query_ctx()->query_mem_tracker());
744
1
    RETURN_IF_ERROR(_state_transition(State::FINALIZED));
745
1
    std::unique_lock<std::mutex> lc(_dependency_lock);
746
1
    _sink_shared_state.reset();
747
1
    _op_shared_states.clear();
748
1
    _shared_state_map.clear();
749
1
    _block.reset();
750
1
    _operators.clear();
751
1
    _sink.reset();
752
1
    _pipeline.reset();
753
1
    return Status::OK();
754
1
}
755
756
16
Status PipelineTask::close(Status exec_status, bool close_sink) {
757
16
    int64_t close_ns = 0;
758
16
    Status s;
759
16
    {
760
16
        SCOPED_RAW_TIMER(&close_ns);
761
16
        if (close_sink) {
762
6
            s = _sink->close(_state, exec_status);
763
6
        }
764
20
        for (auto& op : _operators) {
765
20
            auto tem = op->close(_state);
766
20
            if (!tem.ok() && s.ok()) {
767
0
                s = std::move(tem);
768
0
            }
769
20
        }
770
16
    }
771
16
    if (_opened) {
772
16
        COUNTER_UPDATE(_close_timer, close_ns);
773
16
        COUNTER_UPDATE(_task_profile->total_time_counter(), close_ns);
774
16
    }
775
776
16
    if (close_sink && _opened) {
777
6
        _task_profile->add_info_string("WakeUpEarly", std::to_string(_wake_up_early.load()));
778
6
        _fresh_profile_counter();
779
6
    }
780
781
16
    if (close_sink) {
782
6
        RETURN_IF_ERROR(_state_transition(State::FINISHED));
783
6
    }
784
16
    return s;
785
16
}
786
787
29.5k
std::string PipelineTask::debug_string() {
788
29.5k
    fmt::memory_buffer debug_string_buffer;
789
790
29.5k
    fmt::format_to(debug_string_buffer, "QueryId: {}\n", print_id(_query_id));
791
29.5k
    fmt::format_to(debug_string_buffer, "InstanceId: {}\n",
792
29.5k
                   print_id(_state->fragment_instance_id()));
793
794
29.5k
    fmt::format_to(debug_string_buffer,
795
29.5k
                   "PipelineTask[id = {}, open = {}, eos = {}, state = {}, dry run = "
796
29.5k
                   "{}, _wake_up_early = {}, _wake_up_by = {}, time elapsed since last state "
797
29.5k
                   "changing = {}s, spilling = {}, is running = {}]",
798
29.5k
                   _index, _opened, _eos, _to_string(_exec_state), _dry_run, _wake_up_early.load(),
799
29.5k
                   _wake_by, _state_change_watcher.elapsed_time() / NANOS_PER_SEC, _spilling,
800
29.5k
                   is_running());
801
29.5k
    std::unique_lock<std::mutex> lc(_dependency_lock);
802
29.5k
    auto* cur_blocked_dep = _blocked_dep;
803
29.5k
    auto fragment = _fragment_context.lock();
804
29.5k
    if (is_finalized() || !fragment) {
805
5
        fmt::format_to(debug_string_buffer, " pipeline name = {}", _pipeline_name);
806
5
        return fmt::to_string(debug_string_buffer);
807
5
    }
808
29.5k
    auto elapsed = fragment->elapsed_time() / NANOS_PER_SEC;
809
29.5k
    fmt::format_to(debug_string_buffer, " elapse time = {}s, block dependency = [{}]\n", elapsed,
810
29.5k
                   cur_blocked_dep && !is_finalized() ? cur_blocked_dep->debug_string() : "NULL");
811
812
29.5k
    if (_state && _state->local_runtime_filter_mgr()) {
813
0
        fmt::format_to(debug_string_buffer, "local_runtime_filter_mgr: [{}]\n",
814
0
                       _state->local_runtime_filter_mgr()->debug_string());
815
0
    }
816
817
29.5k
    fmt::format_to(debug_string_buffer, "operators: ");
818
59.1k
    for (size_t i = 0; i < _operators.size(); i++) {
819
29.5k
        fmt::format_to(debug_string_buffer, "\n{}",
820
29.5k
                       _opened && !is_finalized()
821
29.5k
                               ? _operators[i]->debug_string(_state, cast_set<int>(i))
822
29.5k
                               : _operators[i]->debug_string(cast_set<int>(i)));
823
29.5k
    }
824
29.5k
    fmt::format_to(debug_string_buffer, "\n{}\n",
825
29.5k
                   _opened && !is_finalized()
826
29.5k
                           ? _sink->debug_string(_state, cast_set<int>(_operators.size()))
827
29.5k
                           : _sink->debug_string(cast_set<int>(_operators.size())));
828
829
29.5k
    fmt::format_to(debug_string_buffer, "\nRead Dependency Information: \n");
830
831
29.5k
    size_t i = 0;
832
59.1k
    for (; i < _read_dependencies.size(); i++) {
833
59.1k
        for (size_t j = 0; j < _read_dependencies[i].size(); j++) {
834
29.5k
            fmt::format_to(debug_string_buffer, "{}. {}\n", i,
835
29.5k
                           _read_dependencies[i][j]->debug_string(cast_set<int>(i) + 1));
836
29.5k
        }
837
29.5k
    }
838
839
29.5k
    fmt::format_to(debug_string_buffer, "{}. {}\n", i,
840
29.5k
                   _memory_sufficient_dependency->debug_string(cast_set<int>(i++)));
841
842
29.5k
    fmt::format_to(debug_string_buffer, "\nWrite Dependency Information: \n");
843
59.1k
    for (size_t j = 0; j < _write_dependencies.size(); j++, i++) {
844
29.5k
        fmt::format_to(debug_string_buffer, "{}. {}\n", i,
845
29.5k
                       _write_dependencies[j]->debug_string(cast_set<int>(j) + 1));
846
29.5k
    }
847
848
29.5k
    fmt::format_to(debug_string_buffer, "\nExecution Dependency Information: \n");
849
88.7k
    for (size_t j = 0; j < _execution_dependencies.size(); j++, i++) {
850
59.1k
        fmt::format_to(debug_string_buffer, "{}. {}\n", i,
851
59.1k
                       _execution_dependencies[j]->debug_string(cast_set<int>(i) + 1));
852
59.1k
    }
853
854
29.5k
    fmt::format_to(debug_string_buffer, "Finish Dependency Information: \n");
855
88.7k
    for (size_t j = 0; j < _finish_dependencies.size(); j++, i++) {
856
59.1k
        fmt::format_to(debug_string_buffer, "{}. {}\n", i,
857
59.1k
                       _finish_dependencies[j]->debug_string(cast_set<int>(i) + 1));
858
59.1k
    }
859
29.5k
    return fmt::to_string(debug_string_buffer);
860
29.5k
}
861
862
0
size_t PipelineTask::get_revocable_size() const {
863
0
    if (!_opened || is_finalized() || _running || (_eos && !_spilling)) {
864
0
        return 0;
865
0
    }
866
867
0
    return _sink->revocable_mem_size(_state);
868
0
}
869
870
0
Status PipelineTask::revoke_memory(const std::shared_ptr<SpillContext>& spill_context) {
871
0
    DCHECK(spill_context);
872
0
    if (is_finalized()) {
873
0
        spill_context->on_task_finished();
874
0
        VLOG_DEBUG << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this)
875
0
                   << " finalized";
876
0
        return Status::OK();
877
0
    }
878
879
0
    const auto revocable_size = _sink->revocable_mem_size(_state);
880
0
    if (revocable_size >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) {
881
0
        auto revokable_task = std::make_shared<RevokableTask>(shared_from_this(), spill_context);
882
0
        RETURN_IF_ERROR(_state->get_query_ctx()->get_pipe_exec_scheduler()->submit(revokable_task));
883
0
    } else {
884
0
        spill_context->on_task_finished();
885
0
        LOG(INFO) << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this)
886
0
                  << " has not enough data to revoke: " << revocable_size;
887
0
    }
888
0
    return Status::OK();
889
0
}
890
891
24
Status PipelineTask::wake_up(Dependency* dep, std::unique_lock<std::mutex>& /* dep_lock */) {
892
    // call by dependency
893
24
    DCHECK_EQ(_blocked_dep, dep) << "dep : " << dep->debug_string(0) << "task: " << debug_string();
894
24
    _blocked_dep = nullptr;
895
24
    auto holder = std::dynamic_pointer_cast<PipelineTask>(shared_from_this());
896
24
    RETURN_IF_ERROR(_state_transition(PipelineTask::State::RUNNABLE));
897
24
    if (auto f = _fragment_context.lock(); f) {
898
24
        RETURN_IF_ERROR(_state->get_query_ctx()->get_pipe_exec_scheduler()->submit(holder));
899
24
    }
900
24
    return Status::OK();
901
24
}
902
903
95
Status PipelineTask::_state_transition(State new_state) {
904
95
    if (_exec_state != new_state) {
905
90
        _state_change_watcher.reset();
906
90
        _state_change_watcher.start();
907
90
    }
908
95
    _task_profile->add_info_string("TaskState", _to_string(new_state));
909
95
    _task_profile->add_info_string("BlockedByDependency", _blocked_dep ? _blocked_dep->name() : "");
910
95
    if (!LEGAL_STATE_TRANSITION[(int)new_state].contains(_exec_state)) {
911
17
        return Status::InternalError(
912
17
                "Task state transition from {} to {} is not allowed! Task info: {}",
913
17
                _to_string(_exec_state), _to_string(new_state), debug_string());
914
17
    }
915
78
    _exec_state = new_state;
916
78
    return Status::OK();
917
95
}
918
919
#include "common/compile_check_end.h"
920
} // namespace doris::pipeline