Coverage Report

Created: 2026-03-20 12:38

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