Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
38 changes: 20 additions & 18 deletions be/src/pipeline/pipeline_fragment_context.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@
#include "runtime/thread_context.h"
#include "service/backend_options.h"
#include "util/container_util.hpp"
#include "util/countdown_latch.h"
#include "util/debug_util.h"
#include "util/uid_util.h"
#include "vec/common/sort/heap_sorter.h"
Expand Down Expand Up @@ -513,27 +514,28 @@ Status PipelineFragmentContext::_build_pipeline_tasks(const doris::TPipelineFrag
target_size > _runtime_state->query_options().parallel_prepare_threshold)) {
// If instances parallelism is big enough ( > parallel_prepare_threshold), we will prepare all tasks by multi-threads
std::vector<Status> prepare_status(target_size);
std::mutex m;
std::condition_variable cv;
int prepare_done = 0;
int submitted_tasks = 0;
Status submit_status;
CountDownLatch latch((int)target_size);
for (size_t i = 0; i < target_size; i++) {
RETURN_IF_ERROR(thread_pool->submit_func([&, i]() {
submit_status = thread_pool->submit_func([&, i]() {
SCOPED_ATTACH_TASK(_query_ctx.get());
prepare_status[i] = pre_and_submit(i, this);
std::unique_lock<std::mutex> lock(m);
prepare_done++;
if (prepare_done == target_size) {
cv.notify_one();
}
}));
}
std::unique_lock<std::mutex> lock(m);
if (prepare_done != target_size) {
cv.wait(lock);
for (size_t i = 0; i < target_size; i++) {
if (!prepare_status[i].ok()) {
return prepare_status[i];
}
latch.count_down();
});
if (LIKELY(submit_status.ok())) {
submitted_tasks++;
} else {
break;
}
}
latch.arrive_and_wait(target_size - submitted_tasks);
if (UNLIKELY(!submit_status.ok())) {
return submit_status;
}
for (int i = 0; i < submitted_tasks; i++) {
if (!prepare_status[i].ok()) {
return prepare_status[i];
}
}
} else {
Expand Down
7 changes: 7 additions & 0 deletions be/src/util/countdown_latch.h
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,13 @@ class CountDownLatch {
}
}

// decrements the internal counter by n and blocks the calling thread until the counter reaches zero.
void arrive_and_wait(uint64_t n) {
DCHECK_GE(n, 0);
count_down(n);
wait();
}

uint64_t count() const {
std::lock_guard<std::mutex> lock(_lock);
return _count;
Expand Down
Loading