Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Reproduce task util issue 45266 #45318

Closed
wants to merge 3 commits into from
Closed
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
25 changes: 18 additions & 7 deletions cpp/src/arrow/acero/task_util.cc
Original file line number Diff line number Diff line change
Expand Up @@ -212,11 +212,16 @@ std::vector<std::pair<int, int64_t>> TaskSchedulerImpl::PickTasks(int num_tasks,

Status TaskSchedulerImpl::ExecuteTask(size_t thread_id, int group_id, int64_t task_id,
bool* task_group_finished) {
Status status;
if (!aborted_) {
RETURN_NOT_OK(task_groups_[group_id].task_impl_(thread_id, task_id));
status = task_groups_[group_id].task_impl_(thread_id, task_id);
}
*task_group_finished = PostExecuteTask(thread_id, group_id);
return Status::OK();
if (*task_group_finished) {
bool all_task_groups_finished = false;
RETURN_NOT_OK(OnTaskGroupFinished(thread_id, group_id, &all_task_groups_finished));
}
return status;
}

bool TaskSchedulerImpl::PostExecuteTask(size_t thread_id, int group_id) {
Expand Down Expand Up @@ -373,11 +378,17 @@ Status TaskSchedulerImpl::ScheduleMore(size_t thread_id, int num_tasks_finished)

bool task_group_finished = false;
RETURN_NOT_OK(ExecuteTask(thread_id, group_id, task_id, &task_group_finished));

if (task_group_finished) {
bool all_task_groups_finished = false;
return OnTaskGroupFinished(thread_id, group_id, &all_task_groups_finished);
}
// if (!status.ok()) {
// if (PostExecuteTask(thread_id, group_id)) {
// bool all_task_groups_finished = false;
// RETURN_NOT_OK(
// OnTaskGroupFinished(thread_id, group_id, &all_task_groups_finished));
// }
// return status;
// } else if (task_group_finished) {
// bool all_task_groups_finished = false;
// return OnTaskGroupFinished(thread_id, group_id, &all_task_groups_finished);
// }

return Status::OK();
}));
Expand Down
73 changes: 73 additions & 0 deletions cpp/src/arrow/acero/task_util_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -231,5 +231,78 @@ TEST(TaskScheduler, StressTwo) {
}
}

TEST(TaskScheduler, AbortContOnTaskErrorSerial) {
constexpr int kNumTasks = 16;

auto scheduler = TaskScheduler::Make();
auto task = [&](std::size_t, int64_t task_id) {
if (task_id == kNumTasks / 2) {
return Status::Invalid("Task failed");
}
return Status::OK();
};

int task_group =
scheduler->RegisterTaskGroup(task, [](std::size_t) { return Status::OK(); });
scheduler->RegisterEnd();

ASSERT_OK(scheduler->StartScheduling(
0, [](TaskScheduler::TaskGroupContinuationImpl) { return Status::OK(); }, 1, true));
ASSERT_RAISES_WITH_MESSAGE(Invalid, "Invalid: Task failed",
scheduler->StartTaskGroup(0, task_group, kNumTasks));

bool abort_cont_called = false;
auto abort_cont = [&]() { abort_cont_called = true; };
scheduler->Abort(abort_cont);
ASSERT_TRUE(abort_cont_called);
}

TEST(TaskScheduler, AbortContOnTaskErrorParallel) {
#ifndef ARROW_ENABLE_THREADING
GTEST_SKIP() << "Test requires threading support";
#endif
constexpr int kNumThreads = 16;

ThreadIndexer thread_indexer;
int num_threads = std::min(static_cast<int>(thread_indexer.Capacity()), kNumThreads);
ASSERT_OK_AND_ASSIGN(std::shared_ptr<ThreadPool> thread_pool,
MakePrimedThreadPool(num_threads));
TaskScheduler::ScheduleImpl schedule =
[&](TaskScheduler::TaskGroupContinuationImpl task) {
return thread_pool->Spawn([&, task] {
std::size_t thread_id = thread_indexer();
auto status = task(thread_id);
ASSERT_TRUE(status.ok() || status.IsInvalid() || status.IsCancelled());
});
};

int num_tasks = num_threads * 2;
auto scheduler = TaskScheduler::Make();
auto task = [&](std::size_t, int64_t task_id) {
if (task_id % 2 == 0) {
return Status::Invalid("Task failed");
}
return Status::OK();
};

int task_group =
scheduler->RegisterTaskGroup(task, [](std::size_t) { return Status::OK(); });
scheduler->RegisterEnd();

ASSERT_OK(scheduler->StartScheduling(0, schedule, num_tasks, false));
ASSERT_OK(scheduler->StartTaskGroup(0, task_group, num_tasks));

thread_pool->WaitForIdle();

bool abort_cont_called = false;
auto abort_cont = [&]() {
ASSERT_FALSE(abort_cont_called);
abort_cont_called = true;
};
scheduler->Abort(abort_cont);

ASSERT_TRUE(abort_cont_called);
}

} // namespace acero
} // namespace arrow
Loading