Coverage Report

Created: 2025-11-05 22:50

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