Skip to content

Commit

Permalink
Address review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
zanmato1984 committed Feb 21, 2025
1 parent c6460ac commit 4cba288
Showing 1 changed file with 2 additions and 9 deletions.
11 changes: 2 additions & 9 deletions cpp/src/arrow/acero/hash_join_benchmark.cc
Original file line number Diff line number Diff line change
Expand Up @@ -163,11 +163,7 @@ class JoinBenchmark {
&(schema_mgr_->proj_maps[1]), std::move(key_cmp), settings.residual_filter,
std::move(register_task_group_callback), std::move(start_task_group_callback),
[](int64_t, ExecBatch) { return Status::OK(); },
[&](int64_t) {
std::unique_lock<std::mutex> lk(finished_mutex_);
finished_cv_.notify_one();
return Status::OK();
}));
[&](int64_t) { return Status::OK(); }));

task_group_probe_ = scheduler_->RegisterTaskGroup(
[this](size_t thread_index, int64_t task_id) -> Status {
Expand All @@ -194,15 +190,12 @@ class JoinBenchmark {
l_batches_.batch_count());
}));

std::unique_lock<std::mutex> lk(finished_mutex_);
finished_cv_.wait(lk);
thread_pool_->WaitForIdle();
}

std::unique_ptr<TaskScheduler> scheduler_;
ThreadIndexer thread_indexer_;
arrow::internal::ThreadPool* thread_pool_;
std::condition_variable finished_cv_;
std::mutex finished_mutex_;

AccumulationQueue l_batches_;
AccumulationQueue r_batches_;
Expand Down

0 comments on commit 4cba288

Please sign in to comment.