From 7cbafc8b92217c0330c2eedcb86f2d0819acf7cb Mon Sep 17 00:00:00 2001 From: michalursa Date: Mon, 3 Jan 2022 14:46:08 -0800 Subject: [PATCH 01/42] Adding Bloom filter implementation --- cpp/src/arrow/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 77f9959fdbe..ec6cada1cda 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -393,7 +393,6 @@ if(ARROW_COMPUTE) compute/exec/key_encode.cc compute/exec/key_hash.cc compute/exec/key_map.cc - compute/exec/options.cc compute/exec/order_by_impl.cc compute/exec/partition_util.cc compute/exec/options.cc From 012175a568a6c4e24e36504ae8a2b408beece124 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 24 Jan 2022 21:31:13 -0800 Subject: [PATCH 02/42] Add Bloom Filter Pushdown --- cpp/src/arrow/compute/exec/bloom_filter.cc | 4 +- cpp/src/arrow/compute/exec/bloom_filter.h | 10 +- cpp/src/arrow/compute/exec/exec_plan.cc | 4 + cpp/src/arrow/compute/exec/exec_plan.h | 7 + cpp/src/arrow/compute/exec/hash_join.cc | 345 ++++++++++++++++-- cpp/src/arrow/compute/exec/hash_join.h | 10 +- .../arrow/compute/exec/hash_join_benchmark.cc | 44 ++- .../arrow/compute/exec/hash_join_graphs.py | 2 +- cpp/src/arrow/compute/exec/hash_join_node.cc | 94 ++++- .../arrow/compute/exec/hash_join_node_test.cc | 179 ++++++++- cpp/src/arrow/compute/exec/key_encode.h | 48 +++ cpp/src/arrow/compute/exec/options.h | 28 +- cpp/src/arrow/compute/exec/util.h | 2 +- cpp/src/arrow/compute/kernels/row_encoder.h | 1 + 14 files changed, 713 insertions(+), 65 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 61031725457..8863b456bb1 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -301,7 +301,7 @@ int64_t BlockedBloomFilter::NumBitsSet() const { Status BloomFilterBuilder_SingleThreaded::Begin(size_t /*num_threads*/, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t /*num_batches*/, - BlockedBloomFilter* build_target) { + BlockedBloomFilter *build_target) { hardware_flags_ = hardware_flags; build_target_ = build_target; @@ -332,7 +332,7 @@ void BloomFilterBuilder_SingleThreaded::PushNextBatchImp(int num_rows, const T* Status BloomFilterBuilder_Parallel::Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t /*num_batches*/, - BlockedBloomFilter* build_target) { + BlockedBloomFilter *build_target) { hardware_flags_ = hardware_flags; build_target_ = build_target; diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index b89343373ae..87abadbb26d 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -259,7 +259,7 @@ class ARROW_EXPORT BloomFilterBuilder { virtual ~BloomFilterBuilder() = default; virtual Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter* build_target) = 0; + BlockedBloomFilter *build_target) = 0; virtual int64_t num_tasks() const { return 0; } virtual Status PushNextBatch(size_t thread_index, int num_rows, const uint32_t* hashes) = 0; @@ -273,7 +273,7 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter* build_target) override; + BlockedBloomFilter *build_target) override; Status PushNextBatch(size_t /*thread_index*/, int num_rows, const uint32_t* hashes) override; @@ -286,14 +286,14 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { void PushNextBatchImp(int num_rows, const T* hashes); int64_t hardware_flags_; - BlockedBloomFilter* build_target_; + BlockedBloomFilter *build_target_; }; class BloomFilterBuilder_Parallel : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter* build_target) override; + BlockedBloomFilter *build_target) override; Status PushNextBatch(size_t thread_id, int num_rows, const uint32_t* hashes) override; @@ -306,7 +306,7 @@ class BloomFilterBuilder_Parallel : public BloomFilterBuilder { void PushNextBatchImp(size_t thread_id, int num_rows, const T* hashes); int64_t hardware_flags_; - BlockedBloomFilter* build_target_; + BlockedBloomFilter *build_target_; int log_num_prtns_; struct ThreadLocalState { std::vector partitioned_hashes_32; diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index b7a9c7e1bb0..07e73d3e3d0 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -98,6 +98,10 @@ struct ExecPlanImpl : public ExecPlan { // producers precede consumers sorted_nodes_ = TopoSort(); + for(ExecNode *node : sorted_nodes_) + { + RETURN_NOT_OK(node->PrepareToProduce()); + } std::vector> futures; diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index be2f23ad24b..508a48209e5 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -212,6 +212,13 @@ class ARROW_EXPORT ExecNode { // A node with multiple outputs will also need to ensure it is applying backpressure if // any of its outputs is asking to pause + /// \brief Steps performed immediately before StartProducing is called + /// + /// This hook performs any actions in between creation of ExecPlan and the call to + /// StartProducing. An example could be Bloom filter pushdown. The order of ExecNodes + /// that executes this method is undefined, but the calls are made synchronously. + virtual Status PrepareToProduce() { return Status::OK(); } + /// \brief Start producing /// /// This must only be called once. If this fails, then other lifecycle diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 3207bb96984..be0d3a95208 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -26,6 +26,7 @@ #include #include "arrow/compute/exec/hash_join_dict.h" +#include "arrow/compute/exec/key_hash.h" #include "arrow/compute/exec/task_util.h" #include "arrow/compute/kernels/row_encoder.h" @@ -45,11 +46,15 @@ class HashJoinBasicImpl : public HashJoinImpl { } EVENT(span_, "InputReceived"); - if (QueueBatchIfNeeded(side, batch)) { + ARROW_ASSIGN_OR_RAISE(bool queued, QueueBatchIfNeeded(thread_index, side, batch)); + if (queued) + { return Status::OK(); - } else { - ARROW_DCHECK(side == 0); - return ProbeBatch(thread_index, batch); + } + else + { + ARROW_DCHECK(side == 0); + return ProbeBatch(thread_index, batch); } } @@ -62,7 +67,7 @@ class HashJoinBasicImpl : public HashJoinImpl { bool proceed; { std::lock_guard lock(finished_mutex_); - proceed = !left_side_finished_ && left_queue_finished_; + proceed = !left_side_finished_ && left_queue_probe_finished_; left_side_finished_ = true; } if (proceed) { @@ -83,12 +88,18 @@ class HashJoinBasicImpl : public HashJoinImpl { } Status Init(ExecContext* ctx, JoinType join_type, bool use_sync_execution, - size_t num_threads, HashJoinSchema* schema_mgr, + size_t /*num_threads*/, HashJoinSchema* schema_mgr, std::vector key_cmp, Expression filter, OutputBatchCallback output_batch_callback, FinishedCallback finished_callback, - TaskScheduler::ScheduleImpl schedule_task_callback) override { - num_threads = std::max(num_threads, static_cast(1)); + TaskScheduler::ScheduleImpl schedule_task_callback, + HashJoinImpl* pushdown_target, + std::vector column_map) override { + // TODO(ARROW-15732) + // Each side of join might have an IO thread being called from. + // As of right now, we ignore the `num_threads` argument, so later we will have to + // readd `num_threads_ = num_threads;` + num_threads_ = GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1; START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", {{"detail", filter.ToString()}, @@ -97,36 +108,57 @@ class HashJoinBasicImpl : public HashJoinImpl { ctx_ = ctx; join_type_ = join_type; - num_threads_ = num_threads; schema_mgr_ = schema_mgr; key_cmp_ = std::move(key_cmp); filter_ = std::move(filter); output_batch_callback_ = std::move(output_batch_callback); finished_callback_ = std::move(finished_callback); - // TODO(ARROW-15732) - // Each side of join might have an IO thread being called from. local_states_.resize(GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1); for (size_t i = 0; i < local_states_.size(); ++i) { local_states_[i].is_initialized = false; local_states_[i].is_has_match_initialized = false; } - dict_probe_.Init(GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1); + temp_stacks_.resize(num_threads_); + for(size_t i = 0; i < temp_stacks_.size(); i++) + RETURN_NOT_OK(temp_stacks_[i].Init(ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); + + dict_probe_.Init(num_threads_); + + pushdown_target_ = pushdown_target; + column_map_ = std::move(column_map); + if(pushdown_target_) + pushdown_target_->ExpectBloomFilter(); + right_input_row_count_ = 0; has_hash_table_ = false; num_batches_produced_.store(0); cancelled_ = false; right_side_finished_ = false; left_side_finished_ = false; - left_queue_finished_ = false; + bloom_filters_ready_ = pushdown_target_ == nullptr; // If we're not pushing the Bloom filter anywhere, the filter is ready + left_queue_bloom_finished_ = false; + left_queue_probe_finished_ = false; scheduler_ = TaskScheduler::Make(); + if(pushdown_target_) + { + bloom_filter_ = std::make_shared(); + bloom_filter_builder_ = BloomFilterBuilder::Make( + use_sync_execution + ? BloomFilterBuildStrategy::SINGLE_THREADED + : BloomFilterBuildStrategy::PARALLEL); + } + + RegisterBuildBloomFilter(); RegisterBuildHashTable(); + RegisterBloomFilterQueuedBatches(); RegisterProbeQueuedBatches(); RegisterScanHashTable(); scheduler_->RegisterEnd(); + RETURN_NOT_OK(scheduler_->StartScheduling( 0 /*thread index*/, std::move(schedule_task_callback), - static_cast(2 * num_threads) /*concurrent tasks*/, use_sync_execution)); + static_cast(2 * num_threads_) /*concurrent tasks*/, use_sync_execution)); return Status::OK(); } @@ -138,6 +170,37 @@ class HashJoinBasicImpl : public HashJoinImpl { scheduler_->Abort(std::move(pos_abort_callback)); } + Status PushBloomFilter( + size_t thread_index, + std::shared_ptr filter, + std::vector column_map) override + { + bool proceed; + { + std::lock_guard lock_bloom(bloom_filters_mutex_); + pushed_bloom_filters_.emplace_back(std::move(filter)); + bloom_filter_column_maps_.emplace_back(std::move(column_map)); + proceed = pushed_bloom_filters_.size() == num_expected_bloom_filters_; + ARROW_DCHECK(pushed_bloom_filters_.size() <= num_expected_bloom_filters_); + } + if(proceed) + { + size_t num_batches; + { + std::lock_guard lock(left_batches_mutex_); + num_batches = left_batches_.size(); + bloom_filters_ready_ = true; + } + RETURN_NOT_OK(BloomFilterQueuedBatches(thread_index, num_batches)); + } + return Status::OK(); + } + + void ExpectBloomFilter() override + { + num_expected_bloom_filters_ += 1; + } + private: void InitEncoder(int side, HashJoinProjection projection_handle, RowEncoder* encoder) { std::vector data_types; @@ -604,8 +667,98 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::OK(); } + Status ApplyBloomFiltersToBatch(size_t thread_index, ExecBatch &batch) + { + size_t bit_vector_bytes = (batch.length + 7) / 8; + std::vector selected(bit_vector_bytes); + std::vector hashes(batch.length); + std::vector bv(bit_vector_bytes); + + // Start with full selection for the current minibatch + memset(selected.data(), 0xff, bit_vector_bytes); + for(size_t ifilter = 0; ifilter < pushed_bloom_filters_.size(); ifilter++) + { + std::vector keys(bloom_filter_column_maps_[ifilter].size()); + for(size_t i = 0; i < keys.size(); i++) + { + int input_idx = bloom_filter_column_maps_[ifilter][i]; + keys[i] = batch[input_idx]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(keys))); + Hashing32::HashBatch( + key_batch, + hashes.data(), + ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], + 0, + key_batch.length); + + pushed_bloom_filters_[ifilter]->Find( + ctx_->cpu_info()->hardware_flags(), + key_batch.length, + hashes.data(), + bv.data()); + arrow::internal::BitmapAnd(bv.data(), 0, selected.data(), 0, key_batch.length, 0, selected.data()); + } + auto selected_buffer = std::make_shared(selected.data(), bit_vector_bytes); + ArrayData selected_arraydata( + boolean(), + batch.length, + { nullptr, std::move(selected_buffer) }); + Datum selected_datum(selected_arraydata); + FilterOptions options; + size_t first_nonscalar = batch.values.size(); + for(size_t i = 0; i < batch.values.size(); i++) + { + if(!batch.values[i].is_scalar()) + { + ARROW_ASSIGN_OR_RAISE(batch.values[i], Filter(batch.values[i], selected_datum, options, ctx_)); + first_nonscalar = std::min(first_nonscalar, i); + ARROW_DCHECK_EQ(batch.values[i].length(), batch.values[first_nonscalar].length()); + } + } + // If they're all Scalar, then the length of the batch is the number of set bits + if(first_nonscalar == batch.values.size()) + batch.length = arrow::internal::CountSetBits(selected.data(), 0, batch.length); + else + batch.length = batch.values[first_nonscalar].length(); + return Status::OK(); + } + int64_t BuildHashTable_num_tasks() { return 1; } + Status BuildBloomFilter_exec_task(size_t thread_index, int64_t task_id) + { + const ExecBatch &input_batch = right_batches_[task_id]; + SchemaProjectionMap key_to_in = schema_mgr_->proj_maps[1].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + std::vector key_columns(key_to_in.num_cols); + for(size_t i = 0; i < key_columns.size(); i++) + { + int input_idx = key_to_in.get(i); + key_columns[i] = input_batch[input_idx]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); + + bloom_filter_builder_->RunInitTask(task_id); + util::TempVectorHolder hash_holder(&temp_stacks_[thread_index], util::MiniBatch::kMiniBatchLength); + uint32_t *hashes = hash_holder.mutable_data(); + for(int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) + { + int64_t length = std::min( + static_cast(key_batch.length - i), + static_cast(util::MiniBatch::kMiniBatchLength)); + Hashing32::HashBatch( + key_batch, + hashes, + ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], + i, + length); + RETURN_NOT_OK(bloom_filter_builder_->PushNextBatch(thread_index, length, hashes)); + } + return Status::OK(); + } + Status BuildHashTable_exec_task(size_t thread_index, int64_t /*task_id*/) { const std::vector& batches = right_batches_; if (batches.empty()) { @@ -651,23 +804,49 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::OK(); } + Status BuildBloomFilter_on_finished(size_t thread_index) + { + if(cancelled_) + return Status::Cancelled("Hash join cancelled"); + ARROW_DCHECK(pushdown_target_); + RETURN_NOT_OK( + pushdown_target_->PushBloomFilter( + thread_index, + std::move(bloom_filter_), + std::move(column_map_))); + return BuildHashTable(thread_index); + } + Status BuildHashTable_on_finished(size_t thread_index) { if (cancelled_) { return Status::Cancelled("Hash join cancelled"); } + bool proceed; { std::lock_guard lock(left_batches_mutex_); + std::lock_guard lock_finish(finished_mutex_); + left_queue_bloom_finished_ = left_queue_bloom_finished_ || num_expected_bloom_filters_ == 0; + proceed = !has_hash_table_ && left_queue_bloom_finished_; has_hash_table_ = true; } + if(proceed) + RETURN_NOT_OK(ProbeQueuedBatches(thread_index)); right_batches_.clear(); - - RETURN_NOT_OK(ProbeQueuedBatches(thread_index)); - return Status::OK(); } + void RegisterBuildBloomFilter() { + task_group_bloom_ = scheduler_->RegisterTaskGroup( + [this](size_t thread_index, int64_t task_id) -> Status { + return BuildBloomFilter_exec_task(thread_index, task_id); + }, + [this](size_t thread_index) -> Status { + return BuildBloomFilter_on_finished(thread_index); + }); + } + void RegisterBuildHashTable() { task_group_build_ = scheduler_->RegisterTaskGroup( [this](size_t thread_index, int64_t task_id) -> Status { @@ -678,11 +857,73 @@ class HashJoinBasicImpl : public HashJoinImpl { }); } + Status BuildBloomFilter(size_t thread_index) { + RETURN_NOT_OK(bloom_filter_builder_->Begin( + num_threads_, + ctx_->cpu_info()->hardware_flags(), + ctx_->memory_pool(), + right_input_row_count_, + right_batches_.size(), + bloom_filter_.get())); + + return scheduler_->StartTaskGroup( + thread_index, + task_group_bloom_, + right_batches_.size()); + } + Status BuildHashTable(size_t thread_index) { return scheduler_->StartTaskGroup(thread_index, task_group_build_, BuildHashTable_num_tasks()); } + Status BloomFilterQueuedBatches_exec_task(size_t thread_index, int64_t task_id) + { + if(cancelled_) + return Status::Cancelled("Hash join cancelled"); + ExecBatch batch; + { + std::lock_guard lock(left_batches_mutex_); + batch = std::move(left_batches_[task_id]); + ARROW_DCHECK(!batch.values.empty()); + } + RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); + { + std::lock_guard lock(left_batches_mutex_); + left_batches_[task_id] = std::move(batch); + } + return Status::OK(); + } + + Status BloomFilterQueuedBatches_on_finished(size_t thread_index) + { + if(cancelled_) + return Status::Cancelled("Hash join cancelled"); + bool proceed; + { + std::lock_guard lock(finished_mutex_); + proceed = !left_queue_bloom_finished_ && has_hash_table_; + left_queue_bloom_finished_ = true; + } + if(proceed) + return ProbeQueuedBatches(thread_index); + return Status::OK(); + } + + void RegisterBloomFilterQueuedBatches() { + task_group_bloom_filter_queued_ = scheduler_->RegisterTaskGroup( + [this](size_t thread_index, int64_t task_id) -> Status { + return BloomFilterQueuedBatches_exec_task(thread_index, task_id); + }, + [this](size_t thread_index) -> Status { + return BloomFilterQueuedBatches_on_finished(thread_index); + }); + } + + Status BloomFilterQueuedBatches(size_t thread_index, size_t num_batches) { + return scheduler_->StartTaskGroup(thread_index, task_group_bloom_filter_queued_, num_batches); + } + int64_t ProbeQueuedBatches_num_tasks() { return static_cast(left_batches_.size()); } @@ -704,8 +945,9 @@ class HashJoinBasicImpl : public HashJoinImpl { bool proceed; { std::lock_guard lock(finished_mutex_); - proceed = left_side_finished_ && !left_queue_finished_; - left_queue_finished_ = true; + ARROW_DCHECK(left_queue_bloom_finished_); + proceed = left_side_finished_ && !left_queue_probe_finished_; + left_queue_probe_finished_ = true; } if (proceed) { RETURN_NOT_OK(OnLeftSideAndQueueFinished(thread_index)); @@ -717,7 +959,7 @@ class HashJoinBasicImpl : public HashJoinImpl { void RegisterProbeQueuedBatches() { task_group_queued_ = scheduler_->RegisterTaskGroup( [this](size_t thread_index, int64_t task_id) -> Status { - return ProbeQueuedBatches_exec_task(thread_index, task_id); + return ProbeQueuedBatches_exec_task(thread_index, task_id); }, [this](size_t thread_index) -> Status { return ProbeQueuedBatches_on_finished(thread_index); @@ -809,22 +1051,46 @@ class HashJoinBasicImpl : public HashJoinImpl { ScanHashTable_num_tasks()); } - bool QueueBatchIfNeeded(int side, ExecBatch batch) { + Result QueueBatchIfNeeded(size_t thread_index, int side, ExecBatch &batch) { if (side == 0) { - std::lock_guard lock(left_batches_mutex_); - if (has_hash_table_) { - return false; - } - left_batches_.emplace_back(std::move(batch)); - return true; + // We don't want to do the filtering while holding the lock, since that can get + // expensive. + bool needs_filtering; + { + std::lock_guard lock(left_batches_mutex_); + bloom_filters_ready_ = bloom_filters_ready_ || num_expected_bloom_filters_ == 0; + needs_filtering = bloom_filters_ready_ && num_expected_bloom_filters_ != 0; + } + if(needs_filtering) + RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); + + bool queued; + { + std::lock_guard lock(left_batches_mutex_); + queued = !bloom_filters_ready_ || !has_hash_table_; + if(queued) + left_batches_.emplace_back(std::move(batch)); + } + return queued; } else { std::lock_guard lock(right_batches_mutex_); + right_input_row_count_ += batch.length; right_batches_.emplace_back(std::move(batch)); return true; } } - Status OnRightSideFinished(size_t thread_index) { return BuildHashTable(thread_index); } + Status OnRightSideFinished(size_t thread_index) + { + if(pushdown_target_ == nullptr) + { + return BuildHashTable(thread_index); + } + else + { + return BuildBloomFilter(thread_index); + } + } Status OnLeftSideAndQueueFinished(size_t thread_index) { return ScanHashTable(thread_index); @@ -875,7 +1141,9 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector key_cmp_; Expression filter_; std::unique_ptr scheduler_; + int task_group_bloom_; int task_group_build_; + int task_group_bloom_filter_queued_; int task_group_queued_; int task_group_scan_; @@ -898,6 +1166,8 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector has_match; }; std::vector local_states_; + std::vector temp_stacks_; // One per thread, but initialized earlier than ThreadLocalState + // Shared runtime state // @@ -914,17 +1184,34 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector left_batches_; bool has_hash_table_; + // This mutex has to be reentrant because in the case of single-threaded execution, if we push + // a Bloom filter to ourselves, we'll be grabbing this mutex twice. This would be a deadlock with + // a non-reentrant mutex std::mutex left_batches_mutex_; + size_t right_input_row_count_; // Sum of the lengths of ExecBatches in right_batches_ std::vector right_batches_; std::mutex right_batches_mutex_; + // Bloom filter stuff + // + std::unique_ptr bloom_filter_builder_; + std::shared_ptr bloom_filter_; + std::vector column_map_; + std::vector< std::shared_ptr > pushed_bloom_filters_; + std::vector< std::vector > bloom_filter_column_maps_; + std::mutex bloom_filters_mutex_; + size_t num_expected_bloom_filters_; + HashJoinImpl *pushdown_target_; + std::atomic num_batches_produced_; bool cancelled_; + bool bloom_filters_ready_; bool right_side_finished_; bool left_side_finished_; - bool left_queue_finished_; + bool left_queue_bloom_finished_; + bool left_queue_probe_finished_; std::mutex finished_mutex_; }; diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/hash_join.h index 12455f0c6d0..fc3692a8e2f 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/compute/exec/hash_join.h @@ -21,6 +21,7 @@ #include #include +#include "arrow/compute/exec/bloom_filter.h" #include "arrow/compute/exec/options.h" #include "arrow/compute/exec/schema_util.h" #include "arrow/compute/exec/task_util.h" @@ -107,7 +108,14 @@ class HashJoinImpl { std::vector key_cmp, Expression filter, OutputBatchCallback output_batch_callback, FinishedCallback finished_callback, - TaskScheduler::ScheduleImpl schedule_task_callback) = 0; + TaskScheduler::ScheduleImpl schedule_task_callback, + HashJoinImpl *pushdown_target, + std::vector column_map) = 0; + virtual void ExpectBloomFilter() = 0; + virtual Status PushBloomFilter( + size_t thread_index, + std::shared_ptr filter, + std::vector column_map) = 0; virtual Status InputReceived(size_t thread_index, int side, ExecBatch batch) = 0; virtual Status InputFinished(size_t thread_index, int side) = 0; virtual void Abort(TaskScheduler::AbortContinuationImpl pos_abort_callback) = 0; diff --git a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc index 3d4271b6cb9..8a56f6799c9 100644 --- a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc +++ b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc @@ -36,6 +36,7 @@ namespace arrow { namespace compute { struct BenchmarkSettings { + bool bloom_filter = false; int num_threads = 1; JoinType join_type = JoinType::INNER; int batch_size = 1024; @@ -57,6 +58,7 @@ class JoinBenchmark { SchemaBuilder l_schema_builder, r_schema_builder; std::vector left_keys, right_keys; + std::vector key_cmp; for (size_t i = 0; i < settings.key_types.size(); i++) { std::string l_name = "lk" + std::to_string(i); std::string r_name = "rk" + std::to_string(i); @@ -93,6 +95,7 @@ class JoinBenchmark { left_keys.push_back(FieldRef(l_name)); right_keys.push_back(FieldRef(r_name)); + key_cmp.push_back(JoinKeyCmp::EQ); } for (size_t i = 0; i < settings.build_payload_types.size(); i++) { @@ -126,6 +129,22 @@ class JoinBenchmark { join_ = *HashJoinImpl::MakeBasic(); + HashJoinImpl *bloom_filter_pushdown_target = nullptr; + std::vector key_input_map; + + bool bloom_filter_does_not_apply_to_join = + settings.join_type == JoinType::LEFT_ANTI || + settings.join_type == JoinType::LEFT_OUTER || + settings.join_type == JoinType::FULL_OUTER; + if(settings.bloom_filter && !bloom_filter_does_not_apply_to_join) + { + bloom_filter_pushdown_target = join_.get(); + SchemaProjectionMap probe_key_to_input = schema_mgr_->proj_maps[0].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + int num_keys = probe_key_to_input.num_cols; + for(int i = 0; i < num_keys; i++) + key_input_map.push_back(probe_key_to_input.get(i)); + } + omp_set_num_threads(settings.num_threads); auto schedule_callback = [](std::function func) -> Status { #pragma omp task @@ -135,8 +154,8 @@ class JoinBenchmark { DCHECK_OK(join_->Init( ctx_.get(), settings.join_type, !is_parallel, settings.num_threads, - schema_mgr_.get(), {JoinKeyCmp::EQ}, std::move(filter), [](ExecBatch) {}, - [](int64_t x) {}, schedule_callback)); + schema_mgr_.get(), std::move(key_cmp), std::move(filter), [](ExecBatch) {}, + [](int64_t x) {}, schedule_callback, bloom_filter_pushdown_target, std::move(key_input_map))); } void RunJoin() { @@ -268,6 +287,17 @@ static void BM_HashJoinBasic_NullPercentage(benchmark::State& st) { HashJoinBasicBenchmarkImpl(st, settings); } + +static void BM_HashJoinBasic_BloomFilter(benchmark::State &st, bool bloom_filter) +{ + BenchmarkSettings settings; + settings.bloom_filter = bloom_filter; + settings.selectivity = static_cast(st.range(0)) / 100.0; + settings.num_build_batches = static_cast(st.range(1)); + settings.num_probe_batches = settings.num_build_batches; + + HashJoinBasicBenchmarkImpl(st, settings); +} #endif std::vector hashtable_krows = benchmark::CreateRange(1, 4096, 8); @@ -395,6 +425,16 @@ BENCHMARK(BM_HashJoinBasic_BuildParallelism) BENCHMARK(BM_HashJoinBasic_NullPercentage) ->ArgNames({"Null Percentage"}) ->DenseRange(0, 100, 10); + +std::vector bloomfilter_argnames = {"Selectivity", "HashTable krows"}; +std::vector> bloomfilter_args = { + benchmark::CreateDenseRange(0, 100, 10), hashtable_krows}; +BENCHMARK_CAPTURE(BM_HashJoinBasic_BloomFilter, "Bloom Filter", true) + ->ArgNames(bloomfilter_argnames) + ->ArgsProduct(selectivity_args); +BENCHMARK_CAPTURE(BM_HashJoinBasic_BloomFilter, "No Bloom Filter", false) + ->ArgNames(bloomfilter_argnames) + ->ArgsProduct(selectivity_args); #else BENCHMARK_CAPTURE(BM_HashJoinBasic_KeyTypes, "{int32}", {int32()}) diff --git a/cpp/src/arrow/compute/exec/hash_join_graphs.py b/cpp/src/arrow/compute/exec/hash_join_graphs.py index ff7ab63187d..16b4c3e5fd4 100755 --- a/cpp/src/arrow/compute/exec/hash_join_graphs.py +++ b/cpp/src/arrow/compute/exec/hash_join_graphs.py @@ -148,7 +148,7 @@ def plot_3d(test, sorted_argnames): num_cols = int(math.ceil(num_graphs / num_rows)) graphs = set(test.args[sorted_argnames[0]]) - for j, graph in enumerate(sorted(graphs, key=string_numeric_sort_key)): + for j, graph in enumerate(sorted(graphs, key=try_as_numeric)): ax = plt.subplot(num_rows, num_cols, j + 1) filtered_test = Test() indices = range(len(test.times)) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 0282e387c42..2b19d0029d9 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/hash_join.h" @@ -466,8 +467,9 @@ class HashJoinNode : public ExecNode { key_cmp_(join_options.key_cmp), filter_(std::move(filter)), schema_mgr_(std::move(schema_mgr)), - impl_(std::move(impl)) { - complete_.store(false); + impl_(std::move(impl)), + disable_bloom_filter_(join_options.disable_bloom_filter) { + complete_.store(false); } static Result Make(ExecPlan* plan, std::vector inputs, @@ -571,6 +573,82 @@ class HashJoinNode : public ExecNode { } } + std::pair> GetPushdownTarget() + { + ARROW_DCHECK(!disable_bloom_filter_); + // We currently only push Bloom filters on the probe side, and only if that input is also a join. + SchemaProjectionMap probe_key_to_input = schema_mgr_->proj_maps[0].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + int num_keys = probe_key_to_input.num_cols; + + // A mapping such that bloom_to_target[i] is the index of key i in the pushdown target's input + std::vector bloom_to_target(num_keys); + HashJoinNode *pushdown_target = this; + for(int i = 0; i < num_keys; i++) + bloom_to_target[i] = probe_key_to_input.get(i); + + for(ExecNode *candidate = inputs()[0]; + candidate->kind_name() == this->kind_name(); + candidate = candidate->inputs()[0]) + { + auto *candidate_as_join = checked_cast(inputs()[0]); + SchemaProjectionMap candidate_output_to_input = + candidate_as_join->schema_mgr_->proj_maps[0].map( + HashJoinProjection::OUTPUT, + HashJoinProjection::INPUT); + + // Check if any of the keys are missing, if they are, break + bool break_outer = false; + for(int i = 0; i < num_keys; i++) + { + // The output is from the candidate's build side + if(bloom_to_target[i] >= candidate_output_to_input.num_cols) + { + break_outer = true; + break; + } + int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); + // The output column has to have come from somewhere... + ARROW_DCHECK_NE(candidate_input_idx, schema_mgr_->kMissingField()); + } + if(break_outer) + break; + + // All keys are present, we can update the mapping + for(int i = 0; i < num_keys; i++) + { + int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); + bloom_to_target[i] = candidate_input_idx; + } + pushdown_target = candidate_as_join; + } + return std::make_pair(pushdown_target->impl_.get(), std::move(bloom_to_target)); + } + + Status PrepareToProduce() override + { + bool use_sync_execution = !(plan_->exec_context()->executor()); + size_t num_threads = use_sync_execution ? 1 : thread_indexer_.Capacity(); + bool bloom_filter_does_not_apply_to_join = + join_type_ == JoinType::LEFT_ANTI || join_type_ == JoinType::LEFT_OUTER || join_type_ == JoinType::FULL_OUTER; + disable_bloom_filter_ = disable_bloom_filter_ || bloom_filter_does_not_apply_to_join; + + HashJoinImpl *pushdown_target = nullptr; + std::vector column_map; + if(!disable_bloom_filter_) + std::tie(pushdown_target, column_map) = GetPushdownTarget(); + + return impl_->Init( + plan_->exec_context(), join_type_, use_sync_execution, num_threads, + schema_mgr_.get(), key_cmp_, filter_, + [this](ExecBatch batch) { this->OutputBatchCallback(batch); }, + [this](int64_t total_num_batches) { this->FinishedCallback(total_num_batches); }, + [this](std::function func) -> Status { + return this->ScheduleTaskCallback(std::move(func)); + }, + pushdown_target, + std::move(column_map)); + } + Status StartProducing() override { START_COMPUTE_SPAN(span_, std::string(kind_name()) + ":" + label(), {{"node.label", label()}, @@ -578,17 +656,6 @@ class HashJoinNode : public ExecNode { {"node.kind", kind_name()}}); END_SPAN_ON_FUTURE_COMPLETION(span_, finished(), this); - bool use_sync_execution = !(plan_->exec_context()->executor()); - size_t num_threads = use_sync_execution ? 1 : thread_indexer_.Capacity(); - - RETURN_NOT_OK(impl_->Init( - plan_->exec_context(), join_type_, use_sync_execution, num_threads, - schema_mgr_.get(), key_cmp_, filter_, - [this](ExecBatch batch) { this->OutputBatchCallback(batch); }, - [this](int64_t total_num_batches) { this->FinishedCallback(total_num_batches); }, - [this](std::function func) -> Status { - return this->ScheduleTaskCallback(std::move(func)); - })); return Status::OK(); } @@ -662,6 +729,7 @@ class HashJoinNode : public ExecNode { std::unique_ptr schema_mgr_; std::unique_ptr impl_; util::AsyncTaskGroup task_group_; + bool disable_bloom_filter_; }; namespace internal { diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 99a8fc01e0a..e5cb24457a5 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1005,6 +1005,7 @@ TEST(HashJoin, Random) { #endif for (int test_id = 0; test_id < num_tests; ++test_id) { bool parallel = (rng.from_range(0, 1) == 1); + bool disable_bloom_filter = (rng.from_range(0, 1) == 1); auto exec_ctx = arrow::internal::make_unique( default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); @@ -1110,7 +1111,8 @@ TEST(HashJoin, Random) { } ARROW_SCOPED_TRACE(join_type_name, " ", key_cmp_str, - " parallel = ", (parallel ? "true" : "false")); + " parallel = ", (parallel ? "true" : "false"), + " bloom_filter =", (disable_bloom_filter ? "false" : "true")); // Run reference join implementation std::vector null_in_key_vectors[2]; @@ -1157,7 +1159,8 @@ TEST(HashJoin, Random) { // Run tested join implementation HashJoinNodeOptions join_options{ join_type, key_fields[0], key_fields[1], output_fields[0], - output_fields[1], key_cmp, filter}; + output_fields[1], key_cmp, filter }; + join_options.disable_bloom_filter = disable_bloom_filter; std::vector> output_schema_fields; for (int i = 0; i < 2; ++i) { for (size_t col = 0; col < output_fields[i].size(); ++col) { @@ -1901,5 +1904,177 @@ TEST(HashJoin, TrivialResidualFilter) { } } + HashJoinNodeOptions GenerateHashJoinNodeOptions( + Random64Bit &rng, + int num_left_cols, + int num_right_cols) + { + HashJoinNodeOptions opts; + opts.join_type = static_cast(rng.from_range(0, 7)); + bool is_left_join = opts.join_type == JoinType::LEFT_SEMI || + opts.join_type == JoinType::LEFT_ANTI || + opts.join_type == JoinType::LEFT_OUTER; + bool is_right_join = opts.join_type == JoinType::RIGHT_SEMI || + opts.join_type == JoinType::RIGHT_ANTI || + opts.join_type == JoinType::RIGHT_OUTER; + + int num_keys = rng.from_range(1, std::min(num_left_cols, num_right_cols)); + for(int i = 0; i < num_left_cols; i++) + { + bool is_out = rng.from_range(0, 2) != 2; + if(is_out && !is_right_join) + opts.left_output.push_back(FieldRef(i)); + } + for(int i = 0; i < num_right_cols; i++) + { + bool is_out = rng.from_range(0, 2) == 2; + if(is_out && !is_left_join) + opts.right_output.push_back(FieldRef(i)); + } + // We need at least one output + if(opts.right_output.empty() && opts.left_output.empty()) + { + if(is_left_join) + { + int col = rng.from_range(0, num_left_cols - 1); + opts.left_output.push_back(FieldRef(col)); + } + else if(is_right_join) + { + int col = rng.from_range(0, num_right_cols - 1); + opts.right_output.push_back(FieldRef(col)); + } + else + { + if(rng.from_range(0, 1) == 0) + { + int col = rng.from_range(0, num_left_cols - 1); + opts.left_output.push_back(FieldRef(col)); + } + else + { + int col = rng.from_range(0, num_right_cols - 1); + opts.right_output.push_back(FieldRef(col)); + } + } + } + + for(int i = 0; i < num_keys; i++) + { + int left = rng.from_range(0, num_left_cols - 1); + int right = rng.from_range(0, num_right_cols - 1); + bool is_or_eq = rng.from_range(0, 1) == 0; + opts.left_keys.push_back(FieldRef(left)); + opts.right_keys.push_back(FieldRef(right)); + opts.key_cmp.push_back( + is_or_eq ? JoinKeyCmp::IS : JoinKeyCmp::EQ); + } + return opts; + } + + void TestSingleChainOfHashJoins(Random64Bit &rng) + { + int num_joins = rng.from_range(2, 5); + std::vector opts; + int num_left_cols = rng.from_range(1, 8); + int num_right_cols = rng.from_range(1, 8); + HashJoinNodeOptions first_opt = GenerateHashJoinNodeOptions(rng, num_left_cols, num_right_cols); + opts.push_back(std::move(first_opt)); + + + std::unordered_map metadata_map; + metadata_map["min"] = "0"; + metadata_map["max"] = "10"; + auto metadata = key_value_metadata(metadata_map); + std::vector> left_fields; + for(int i = 0; i < num_left_cols; i++) + left_fields.push_back(field(std::string("l") + std::to_string(i), int32(), metadata)); + std::vector> first_right_fields; + for(int i = 0; i < num_right_cols; i++) + first_right_fields.push_back(field(std::string("r_0_") + std::to_string(i), int32(), metadata)); + + BatchesWithSchema input_left = MakeRandomBatches(schema(std::move(left_fields))); + std::vector input_right; + input_right.push_back(MakeRandomBatches(schema(std::move(first_right_fields)))); + + for(int i = 1; i < num_joins; i++) + { + int num_right_cols = rng.from_range(1, 8); + HashJoinNodeOptions opt = GenerateHashJoinNodeOptions( + rng, + opts[i - 1].left_output.size() + opts[i - 1].right_output.size(), + num_right_cols); + opts.push_back(std::move(opt)); + + std::vector> right_fields; + for(int j = 0; j < num_right_cols; j++) + right_fields.push_back(field(std::string("r_") + std::to_string(i) + "_" + std::to_string(j), int32(), metadata)); + BatchesWithSchema input = MakeRandomBatches(schema(std::move(right_fields))); + input_right.push_back(std::move(input)); + } + + std::vector reference; + for(bool bloom_filters : { false, true }) + { + bool parallel = true; + ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); + auto exec_ctx = arrow::internal::make_unique( + default_memory_pool(), + parallel ? arrow::internal::GetCpuThreadPool() : nullptr); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); + + ExecNode *left_source; + ASSERT_OK_AND_ASSIGN( + left_source, + MakeExecNode("source", plan.get(), {}, + SourceNodeOptions{input_left.schema, + input_left.gen(parallel, /*slow=*/false)})); + std::vector joins(num_joins); + for(int i = 0; i < num_joins; i++) + { + opts[i].disable_bloom_filter = !bloom_filters; + ExecNode *right_source; + ASSERT_OK_AND_ASSIGN( + right_source, + MakeExecNode("source", + plan.get(), {}, + SourceNodeOptions{ + input_right[i].schema, + input_right[i].gen(parallel, /*slow=*/false) + })); + + std::vector inputs; + if(i == 0) + inputs = { left_source, right_source }; + else + inputs = { joins[i - 1], right_source }; + ASSERT_OK_AND_ASSIGN(joins[i], + MakeExecNode("hashjoin", plan.get(), inputs, opts[i])); + } + AsyncGenerator> sink_gen; + ASSERT_OK_AND_ASSIGN(std::ignore, + MakeExecNode("sink", + plan.get(), + { joins.back() }, + SinkNodeOptions{ &sink_gen })); + ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + if(!bloom_filters) + reference = std::move(result); + else + AssertExecBatchesEqual(joins.back()->output_schema(), reference, result); + } + } + + TEST(HashJoin, ChainedIntegerHashJoins) + { + Random64Bit rng(42); + int num_tests = 10; + for(int i = 0; i < num_tests; i++) + { + ARROW_SCOPED_TRACE("Test ", std::to_string(i)); + TestSingleChainOfHashJoins(rng); + } + } + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_encode.h b/cpp/src/arrow/compute/exec/key_encode.h index f9de31d9c21..4fa6c5a1070 100644 --- a/cpp/src/arrow/compute/exec/key_encode.h +++ b/cpp/src/arrow/compute/exec/key_encode.h @@ -27,6 +27,8 @@ #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/bit_util.h" +#include "arrow/array/data.h" +#include "arrow/compute/exec.h" namespace arrow { namespace compute { @@ -247,6 +249,52 @@ class KeyEncoder { const std::vector& GetBatchColumns() const { return batch_all_cols_; } + static KeyColumnArray ColumnArrayFromArrayData( + const std::shared_ptr &array_data, + int start_row, + int num_rows) + { + KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( + ColumnMetadataFromDataType(array_data->type), + array_data->offset + start_row + num_rows, + array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, + array_data->buffers[1]->data(), + array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR ? array_data->buffers[2]->data() : nullptr); + return KeyEncoder::KeyColumnArray( + column_array, + array_data->offset + start_row, + num_rows); + } + + static KeyColumnMetadata ColumnMetadataFromDataType(const std::shared_ptr &type) + { + if (type->id() == Type::DICTIONARY) + { + auto bit_width = arrow::internal::checked_cast(*type).bit_width(); + ARROW_DCHECK(bit_width % 8 == 0); + return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); + } + else if (type->id() == Type::BOOL) + { + return KeyEncoder::KeyColumnMetadata(true, 0); + } + else if (is_fixed_width(type->id())) + { + return KeyEncoder::KeyColumnMetadata( + true, + arrow::internal::checked_cast(*type).bit_width() / 8); + } + else if (is_binary_like(type->id())) + { + return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); + } + else + { + ARROW_DCHECK(false); + return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); + } + } + private: /// Prepare column array vectors. /// Output column arrays represent a range of input column arrays diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index 3c901be0d2e..48cbf9d3710 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -281,14 +281,16 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { JoinType in_join_type, std::vector in_left_keys, std::vector in_right_keys, Expression filter = literal(true), std::string output_suffix_for_left = default_output_suffix_for_left, - std::string output_suffix_for_right = default_output_suffix_for_right) + std::string output_suffix_for_right = default_output_suffix_for_right, + bool disable_bloom_filter = false) : join_type(in_join_type), left_keys(std::move(in_left_keys)), right_keys(std::move(in_right_keys)), output_all(true), output_suffix_for_left(std::move(output_suffix_for_left)), output_suffix_for_right(std::move(output_suffix_for_right)), - filter(std::move(filter)) { + filter(std::move(filter)), + disable_bloom_filter(disable_bloom_filter) { this->key_cmp.resize(this->left_keys.size()); for (size_t i = 0; i < this->left_keys.size(); ++i) { this->key_cmp[i] = JoinKeyCmp::EQ; @@ -299,7 +301,8 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { std::vector right_keys, std::vector left_output, std::vector right_output, Expression filter = literal(true), std::string output_suffix_for_left = default_output_suffix_for_left, - std::string output_suffix_for_right = default_output_suffix_for_right) + std::string output_suffix_for_right = default_output_suffix_for_right, + bool disable_bloom_filter = false) : join_type(join_type), left_keys(std::move(left_keys)), right_keys(std::move(right_keys)), @@ -308,7 +311,8 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { right_output(std::move(right_output)), output_suffix_for_left(std::move(output_suffix_for_left)), output_suffix_for_right(std::move(output_suffix_for_right)), - filter(std::move(filter)) { + filter(std::move(filter)), + disable_bloom_filter(disable_bloom_filter) { this->key_cmp.resize(this->left_keys.size()); for (size_t i = 0; i < this->left_keys.size(); ++i) { this->key_cmp[i] = JoinKeyCmp::EQ; @@ -320,7 +324,8 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { std::vector right_output, std::vector key_cmp, Expression filter = literal(true), std::string output_suffix_for_left = default_output_suffix_for_left, - std::string output_suffix_for_right = default_output_suffix_for_right) + std::string output_suffix_for_right = default_output_suffix_for_right, + bool disable_bloom_filter = false) : join_type(join_type), left_keys(std::move(left_keys)), right_keys(std::move(right_keys)), @@ -330,17 +335,20 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { key_cmp(std::move(key_cmp)), output_suffix_for_left(std::move(output_suffix_for_left)), output_suffix_for_right(std::move(output_suffix_for_right)), - filter(std::move(filter)) {} + filter(std::move(filter)), + disable_bloom_filter(disable_bloom_filter) {} + + HashJoinNodeOptions() = default; // type of join (inner, left, semi...) - JoinType join_type; + JoinType join_type = JoinType::INNER; // key fields from left input std::vector left_keys; // key fields from right input std::vector right_keys; // if set all valid fields from both left and right input will be output // (and field ref vectors for output fields will be ignored) - bool output_all; + bool output_all = false; // output fields passed from left input std::vector left_output; // output fields passed from right input @@ -358,7 +366,9 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { // the filter are not included. The filter is applied against the // concatenated input schema (left fields then right fields) and can reference // fields that are not included in the output. - Expression filter; + Expression filter = literal(true); + // whether or not to disable Bloom filters in this join + bool disable_bloom_filter = false; }; /// \brief Make a node which select top_k/bottom_k rows passed through it diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 4e7550582a4..9a70e585e5e 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -92,7 +92,7 @@ class TempVectorStack { Status Init(MemoryPool* pool, int64_t size) { num_vectors_ = 0; top_ = 0; - buffer_size_ = size; + buffer_size_ = PaddedAllocationSize(size) + kPadding + 2 * sizeof(uint64_t); ARROW_ASSIGN_OR_RAISE(auto buffer, AllocateResizableBuffer(size, pool)); // Ensure later operations don't accidentally read uninitialized memory. std::memset(buffer->mutable_data(), 0xFF, size); diff --git a/cpp/src/arrow/compute/kernels/row_encoder.h b/cpp/src/arrow/compute/kernels/row_encoder.h index 4087472b67a..a23d5f05605 100644 --- a/cpp/src/arrow/compute/kernels/row_encoder.h +++ b/cpp/src/arrow/compute/kernels/row_encoder.h @@ -50,6 +50,7 @@ struct KeyEncoder { virtual Result> Decode(uint8_t** encoded_bytes, int32_t length, MemoryPool*) = 0; + // extract the null bitmap from the leading nullity bytes of encoded keys static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* null_bitmap, int32_t* null_count); From e52928060c799d377db1de57dbf11345c439ed72 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 12 Apr 2022 14:51:15 -0700 Subject: [PATCH 03/42] Fix after rebase --- cpp/src/arrow/compute/exec/hash_join.cc | 6 +- cpp/src/arrow/compute/exec/key_hash.cc | 100 ++++++++++++++++++++++++ cpp/src/arrow/compute/exec/key_hash.h | 10 +++ 3 files changed, 112 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index be0d3a95208..97f38879ceb 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -669,6 +669,8 @@ class HashJoinBasicImpl : public HashJoinImpl { Status ApplyBloomFiltersToBatch(size_t thread_index, ExecBatch &batch) { + if(batch.length == 0) + return Status::OK(); size_t bit_vector_bytes = (batch.length + 7) / 8; std::vector selected(bit_vector_bytes); std::vector hashes(batch.length); @@ -739,7 +741,6 @@ class HashJoinBasicImpl : public HashJoinImpl { } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); - bloom_filter_builder_->RunInitTask(task_id); util::TempVectorHolder hash_holder(&temp_stacks_[thread_index], util::MiniBatch::kMiniBatchLength); uint32_t *hashes = hash_holder.mutable_data(); for(int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) @@ -1184,9 +1185,6 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector left_batches_; bool has_hash_table_; - // This mutex has to be reentrant because in the case of single-threaded execution, if we push - // a Bloom filter to ourselves, we'll be grabbing this mutex twice. This would be a deadlock with - // a non-reentrant mutex std::mutex left_batches_mutex_; size_t right_input_row_count_; // Sum of the lengths of ExecBatches in right_batches_ diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index 125fd3912e1..9dee0160263 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -24,10 +24,76 @@ #include "arrow/util/bit_util.h" #include "arrow/util/ubsan.h" +#include "key_encode.h" namespace arrow { namespace compute { + static KeyEncoder::KeyColumnMetadata ColumnMetadataFromDataType(const std::shared_ptr &type) + { + if (type->id() == Type::DICTIONARY) + { + auto bit_width = arrow::internal::checked_cast(*type).bit_width(); + ARROW_DCHECK(bit_width % 8 == 0); + return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); + } + else if (type->id() == Type::BOOL) + { + return KeyEncoder::KeyColumnMetadata(true, 0); + } + else if (is_fixed_width(type->id())) + { + return KeyEncoder::KeyColumnMetadata( + true, + arrow::internal::checked_cast(*type).bit_width() / 8); + } + else if (is_binary_like(type->id())) + { + return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); + } + else + { + ARROW_DCHECK(false); + return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); + } + } + + static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( + const std::shared_ptr &array_data, + int start_row, + int num_rows) + { + KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( + ColumnMetadataFromDataType(array_data->type), + array_data->offset + start_row + num_rows, + array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, + array_data->buffers[1]->data(), + array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR ? array_data->buffers[2]->data() : nullptr); + return KeyEncoder::KeyColumnArray( + column_array, + array_data->offset + start_row, + num_rows); + } + + +static void ColumnArraysFromExecBatch( + const ExecBatch &batch, + int start_row, + int num_rows, + std::vector &column_arrays) +{ + int num_columns = static_cast(batch.values.size()); + column_arrays.resize(num_columns); + for (int i = 0; i < num_columns; ++i) + { + const Datum &data = batch.values[i]; + ARROW_DCHECK(data.is_array()); + const std::shared_ptr &array_data = data.array(); + column_arrays[i] = + ColumnArrayFromArrayData(array_data, start_row, num_rows); + } +} + inline uint32_t Hashing32::Round(uint32_t acc, uint32_t input) { acc += input * PRIME32_2; acc = ROTL(acc, 13); @@ -456,6 +522,23 @@ void Hashing32::HashMultiColumn(const std::vector& cols, } } + void Hashing32::HashBatch( + const ExecBatch &key_batch, + uint32_t *hashes, + int64_t hardware_flags, + util::TempVectorStack *temp_stack, + int64_t offset, + int64_t length) + { + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); + + KeyEncoder::KeyEncoderContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + HashMultiColumn(column_arrays, &ctx, hashes); + } + inline uint64_t Hashing64::Avalanche(uint64_t acc) { acc ^= (acc >> 33); acc *= PRIME64_2; @@ -875,5 +958,22 @@ void Hashing64::HashMultiColumn(const std::vector& cols, } } + void Hashing64::HashBatch( + const ExecBatch &key_batch, + uint64_t *hashes, + int64_t hardware_flags, + util::TempVectorStack *temp_stack, + int64_t offset, + int64_t length) + { + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); + + KeyEncoder::KeyEncoderContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + HashMultiColumn(column_arrays, &ctx, hashes); + } + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_hash.h b/cpp/src/arrow/compute/exec/key_hash.h index 719f3dfd460..0d76e3e5fc0 100644 --- a/cpp/src/arrow/compute/exec/key_hash.h +++ b/cpp/src/arrow/compute/exec/key_hash.h @@ -48,6 +48,11 @@ class ARROW_EXPORT Hashing32 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash); + static void HashBatch( + const ExecBatch &key_batch, uint32_t *hashes, + int64_t hardware_flags, util::TempVectorStack *temp_stack, + int64_t offset, int64_t length); + private: static const uint32_t PRIME32_1 = 0x9E3779B1; static const uint32_t PRIME32_2 = 0x85EBCA77; @@ -156,6 +161,11 @@ class ARROW_EXPORT Hashing64 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes); + static void HashBatch( + const ExecBatch &key_batch, uint64_t *hashes, + int64_t hardware_flags, util::TempVectorStack *temp_stack, + int64_t offset, int64_t length); + private: static const uint64_t PRIME64_1 = 0x9E3779B185EBCA87ULL; static const uint64_t PRIME64_2 = 0xC2B2AE3D27D4EB4FULL; From 3b75380b94787dd31084663282185829609a2668 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 12 Apr 2022 14:52:33 -0700 Subject: [PATCH 04/42] clang-format --- cpp/src/arrow/compute/exec/bloom_filter.cc | 4 +- cpp/src/arrow/compute/exec/bloom_filter.h | 10 +- cpp/src/arrow/compute/exec/exec_plan.cc | 5 +- cpp/src/arrow/compute/exec/hash_join.cc | 392 ++++++++---------- cpp/src/arrow/compute/exec/hash_join.h | 10 +- .../arrow/compute/exec/hash_join_benchmark.cc | 36 +- cpp/src/arrow/compute/exec/hash_join_node.cc | 139 +++---- .../arrow/compute/exec/hash_join_node_test.cc | 305 ++++++-------- cpp/src/arrow/compute/exec/key_encode.h | 79 ++-- cpp/src/arrow/compute/exec/key_hash.cc | 153 +++---- cpp/src/arrow/compute/exec/key_hash.h | 14 +- cpp/src/arrow/compute/kernels/row_encoder.h | 1 - 12 files changed, 508 insertions(+), 640 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 8863b456bb1..61031725457 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -301,7 +301,7 @@ int64_t BlockedBloomFilter::NumBitsSet() const { Status BloomFilterBuilder_SingleThreaded::Begin(size_t /*num_threads*/, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t /*num_batches*/, - BlockedBloomFilter *build_target) { + BlockedBloomFilter* build_target) { hardware_flags_ = hardware_flags; build_target_ = build_target; @@ -332,7 +332,7 @@ void BloomFilterBuilder_SingleThreaded::PushNextBatchImp(int num_rows, const T* Status BloomFilterBuilder_Parallel::Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t /*num_batches*/, - BlockedBloomFilter *build_target) { + BlockedBloomFilter* build_target) { hardware_flags_ = hardware_flags; build_target_ = build_target; diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index 87abadbb26d..b89343373ae 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -259,7 +259,7 @@ class ARROW_EXPORT BloomFilterBuilder { virtual ~BloomFilterBuilder() = default; virtual Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter *build_target) = 0; + BlockedBloomFilter* build_target) = 0; virtual int64_t num_tasks() const { return 0; } virtual Status PushNextBatch(size_t thread_index, int num_rows, const uint32_t* hashes) = 0; @@ -273,7 +273,7 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter *build_target) override; + BlockedBloomFilter* build_target) override; Status PushNextBatch(size_t /*thread_index*/, int num_rows, const uint32_t* hashes) override; @@ -286,14 +286,14 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { void PushNextBatchImp(int num_rows, const T* hashes); int64_t hardware_flags_; - BlockedBloomFilter *build_target_; + BlockedBloomFilter* build_target_; }; class BloomFilterBuilder_Parallel : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, - BlockedBloomFilter *build_target) override; + BlockedBloomFilter* build_target) override; Status PushNextBatch(size_t thread_id, int num_rows, const uint32_t* hashes) override; @@ -306,7 +306,7 @@ class BloomFilterBuilder_Parallel : public BloomFilterBuilder { void PushNextBatchImp(size_t thread_id, int num_rows, const T* hashes); int64_t hardware_flags_; - BlockedBloomFilter *build_target_; + BlockedBloomFilter* build_target_; int log_num_prtns_; struct ThreadLocalState { std::vector partitioned_hashes_32; diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 07e73d3e3d0..bb197f8db84 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -98,9 +98,8 @@ struct ExecPlanImpl : public ExecPlan { // producers precede consumers sorted_nodes_ = TopoSort(); - for(ExecNode *node : sorted_nodes_) - { - RETURN_NOT_OK(node->PrepareToProduce()); + for (ExecNode* node : sorted_nodes_) { + RETURN_NOT_OK(node->PrepareToProduce()); } std::vector> futures; diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 97f38879ceb..8cd25c1ad22 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -47,14 +47,11 @@ class HashJoinBasicImpl : public HashJoinImpl { EVENT(span_, "InputReceived"); ARROW_ASSIGN_OR_RAISE(bool queued, QueueBatchIfNeeded(thread_index, side, batch)); - if (queued) - { + if (queued) { return Status::OK(); - } - else - { - ARROW_DCHECK(side == 0); - return ProbeBatch(thread_index, batch); + } else { + ARROW_DCHECK(side == 0); + return ProbeBatch(thread_index, batch); } } @@ -124,10 +121,10 @@ class HashJoinBasicImpl : public HashJoinImpl { dict_probe_.Init(num_threads_); + pushdown_target_ = pushdown_target; column_map_ = std::move(column_map); - if(pushdown_target_) - pushdown_target_->ExpectBloomFilter(); + if (pushdown_target_) pushdown_target_->ExpectBloomFilter(); right_input_row_count_ = 0; has_hash_table_ = false; @@ -135,18 +132,18 @@ class HashJoinBasicImpl : public HashJoinImpl { cancelled_ = false; right_side_finished_ = false; left_side_finished_ = false; - bloom_filters_ready_ = pushdown_target_ == nullptr; // If we're not pushing the Bloom filter anywhere, the filter is ready + bloom_filters_ready_ = + pushdown_target_ == + nullptr; // If we're not pushing the Bloom filter anywhere, the filter is ready left_queue_bloom_finished_ = false; left_queue_probe_finished_ = false; scheduler_ = TaskScheduler::Make(); - if(pushdown_target_) - { - bloom_filter_ = std::make_shared(); - bloom_filter_builder_ = BloomFilterBuilder::Make( - use_sync_execution - ? BloomFilterBuildStrategy::SINGLE_THREADED - : BloomFilterBuildStrategy::PARALLEL); + if (pushdown_target_) { + bloom_filter_ = std::make_shared(); + bloom_filter_builder_ = BloomFilterBuilder::Make( + use_sync_execution ? BloomFilterBuildStrategy::SINGLE_THREADED + : BloomFilterBuildStrategy::PARALLEL); } RegisterBuildBloomFilter(); @@ -170,36 +167,29 @@ class HashJoinBasicImpl : public HashJoinImpl { scheduler_->Abort(std::move(pos_abort_callback)); } - Status PushBloomFilter( - size_t thread_index, - std::shared_ptr filter, - std::vector column_map) override + Status PushBloomFilter(size_t thread_index, std::shared_ptr filter, + std::vector column_map) override { + bool proceed; { - bool proceed; - { - std::lock_guard lock_bloom(bloom_filters_mutex_); - pushed_bloom_filters_.emplace_back(std::move(filter)); - bloom_filter_column_maps_.emplace_back(std::move(column_map)); - proceed = pushed_bloom_filters_.size() == num_expected_bloom_filters_; - ARROW_DCHECK(pushed_bloom_filters_.size() <= num_expected_bloom_filters_); - } - if(proceed) - { - size_t num_batches; - { - std::lock_guard lock(left_batches_mutex_); - num_batches = left_batches_.size(); - bloom_filters_ready_ = true; - } - RETURN_NOT_OK(BloomFilterQueuedBatches(thread_index, num_batches)); - } - return Status::OK(); + std::lock_guard lock_bloom(bloom_filters_mutex_); + pushed_bloom_filters_.emplace_back(std::move(filter)); + bloom_filter_column_maps_.emplace_back(std::move(column_map)); + proceed = pushed_bloom_filters_.size() == num_expected_bloom_filters_; + ARROW_DCHECK(pushed_bloom_filters_.size() <= num_expected_bloom_filters_); } - - void ExpectBloomFilter() override - { - num_expected_bloom_filters_ += 1; + if (proceed) { + size_t num_batches; + { + std::lock_guard lock(left_batches_mutex_); + num_batches = left_batches_.size(); + bloom_filters_ready_ = true; + } + RETURN_NOT_OK(BloomFilterQueuedBatches(thread_index, num_batches)); } + return Status::OK(); + } + + void ExpectBloomFilter() override { num_expected_bloom_filters_ += 1; } private: void InitEncoder(int side, HashJoinProjection projection_handle, RowEncoder* encoder) { @@ -667,98 +657,77 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::OK(); } - Status ApplyBloomFiltersToBatch(size_t thread_index, ExecBatch &batch) - { - if(batch.length == 0) - return Status::OK(); - size_t bit_vector_bytes = (batch.length + 7) / 8; - std::vector selected(bit_vector_bytes); - std::vector hashes(batch.length); - std::vector bv(bit_vector_bytes); - - // Start with full selection for the current minibatch - memset(selected.data(), 0xff, bit_vector_bytes); - for(size_t ifilter = 0; ifilter < pushed_bloom_filters_.size(); ifilter++) - { - std::vector keys(bloom_filter_column_maps_[ifilter].size()); - for(size_t i = 0; i < keys.size(); i++) - { - int input_idx = bloom_filter_column_maps_[ifilter][i]; - keys[i] = batch[input_idx]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(keys))); - Hashing32::HashBatch( - key_batch, - hashes.data(), - ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], - 0, - key_batch.length); - - pushed_bloom_filters_[ifilter]->Find( - ctx_->cpu_info()->hardware_flags(), - key_batch.length, - hashes.data(), - bv.data()); - arrow::internal::BitmapAnd(bv.data(), 0, selected.data(), 0, key_batch.length, 0, selected.data()); - } - auto selected_buffer = std::make_shared(selected.data(), bit_vector_bytes); - ArrayData selected_arraydata( - boolean(), - batch.length, - { nullptr, std::move(selected_buffer) }); - Datum selected_datum(selected_arraydata); - FilterOptions options; - size_t first_nonscalar = batch.values.size(); - for(size_t i = 0; i < batch.values.size(); i++) - { - if(!batch.values[i].is_scalar()) - { - ARROW_ASSIGN_OR_RAISE(batch.values[i], Filter(batch.values[i], selected_datum, options, ctx_)); - first_nonscalar = std::min(first_nonscalar, i); - ARROW_DCHECK_EQ(batch.values[i].length(), batch.values[first_nonscalar].length()); - } - } - // If they're all Scalar, then the length of the batch is the number of set bits - if(first_nonscalar == batch.values.size()) - batch.length = arrow::internal::CountSetBits(selected.data(), 0, batch.length); - else - batch.length = batch.values[first_nonscalar].length(); - return Status::OK(); + Status ApplyBloomFiltersToBatch(size_t thread_index, ExecBatch& batch) { + if (batch.length == 0) return Status::OK(); + size_t bit_vector_bytes = (batch.length + 7) / 8; + std::vector selected(bit_vector_bytes); + std::vector hashes(batch.length); + std::vector bv(bit_vector_bytes); + + // Start with full selection for the current minibatch + memset(selected.data(), 0xff, bit_vector_bytes); + for (size_t ifilter = 0; ifilter < pushed_bloom_filters_.size(); ifilter++) { + std::vector keys(bloom_filter_column_maps_[ifilter].size()); + for (size_t i = 0; i < keys.size(); i++) { + int input_idx = bloom_filter_column_maps_[ifilter][i]; + keys[i] = batch[input_idx]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(keys))); + Hashing32::HashBatch(key_batch, hashes.data(), ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], 0, key_batch.length); + + pushed_bloom_filters_[ifilter]->Find(ctx_->cpu_info()->hardware_flags(), + key_batch.length, hashes.data(), bv.data()); + arrow::internal::BitmapAnd(bv.data(), 0, selected.data(), 0, key_batch.length, 0, + selected.data()); + } + auto selected_buffer = std::make_shared(selected.data(), bit_vector_bytes); + ArrayData selected_arraydata(boolean(), batch.length, + {nullptr, std::move(selected_buffer)}); + Datum selected_datum(selected_arraydata); + FilterOptions options; + size_t first_nonscalar = batch.values.size(); + for (size_t i = 0; i < batch.values.size(); i++) { + if (!batch.values[i].is_scalar()) { + ARROW_ASSIGN_OR_RAISE(batch.values[i], + Filter(batch.values[i], selected_datum, options, ctx_)); + first_nonscalar = std::min(first_nonscalar, i); + ARROW_DCHECK_EQ(batch.values[i].length(), batch.values[first_nonscalar].length()); + } } + // If they're all Scalar, then the length of the batch is the number of set bits + if (first_nonscalar == batch.values.size()) + batch.length = arrow::internal::CountSetBits(selected.data(), 0, batch.length); + else + batch.length = batch.values[first_nonscalar].length(); + return Status::OK(); + } int64_t BuildHashTable_num_tasks() { return 1; } - Status BuildBloomFilter_exec_task(size_t thread_index, int64_t task_id) - { - const ExecBatch &input_batch = right_batches_[task_id]; - SchemaProjectionMap key_to_in = schema_mgr_->proj_maps[1].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); - std::vector key_columns(key_to_in.num_cols); - for(size_t i = 0; i < key_columns.size(); i++) - { - int input_idx = key_to_in.get(i); - key_columns[i] = input_batch[input_idx]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); - - util::TempVectorHolder hash_holder(&temp_stacks_[thread_index], util::MiniBatch::kMiniBatchLength); - uint32_t *hashes = hash_holder.mutable_data(); - for(int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) - { - int64_t length = std::min( - static_cast(key_batch.length - i), - static_cast(util::MiniBatch::kMiniBatchLength)); - Hashing32::HashBatch( - key_batch, - hashes, - ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], - i, - length); - RETURN_NOT_OK(bloom_filter_builder_->PushNextBatch(thread_index, length, hashes)); - } - return Status::OK(); + Status BuildBloomFilter_exec_task(size_t thread_index, int64_t task_id) { + const ExecBatch& input_batch = right_batches_[task_id]; + SchemaProjectionMap key_to_in = + schema_mgr_->proj_maps[1].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + std::vector key_columns(key_to_in.num_cols); + for (size_t i = 0; i < key_columns.size(); i++) { + int input_idx = key_to_in.get(i); + key_columns[i] = input_batch[input_idx]; } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); + + util::TempVectorHolder hash_holder(&temp_stacks_[thread_index], + util::MiniBatch::kMiniBatchLength); + uint32_t* hashes = hash_holder.mutable_data(); + for (int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) { + int64_t length = std::min(static_cast(key_batch.length - i), + static_cast(util::MiniBatch::kMiniBatchLength)); + Hashing32::HashBatch(key_batch, hashes, ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], i, length); + RETURN_NOT_OK(bloom_filter_builder_->PushNextBatch(thread_index, length, hashes)); + } + return Status::OK(); + } Status BuildHashTable_exec_task(size_t thread_index, int64_t /*task_id*/) { const std::vector& batches = right_batches_; @@ -805,18 +774,13 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::OK(); } - Status BuildBloomFilter_on_finished(size_t thread_index) - { - if(cancelled_) - return Status::Cancelled("Hash join cancelled"); - ARROW_DCHECK(pushdown_target_); - RETURN_NOT_OK( - pushdown_target_->PushBloomFilter( - thread_index, - std::move(bloom_filter_), - std::move(column_map_))); - return BuildHashTable(thread_index); - } + Status BuildBloomFilter_on_finished(size_t thread_index) { + if (cancelled_) return Status::Cancelled("Hash join cancelled"); + ARROW_DCHECK(pushdown_target_); + RETURN_NOT_OK(pushdown_target_->PushBloomFilter( + thread_index, std::move(bloom_filter_), std::move(column_map_))); + return BuildHashTable(thread_index); + } Status BuildHashTable_on_finished(size_t thread_index) { if (cancelled_) { @@ -827,12 +791,12 @@ class HashJoinBasicImpl : public HashJoinImpl { { std::lock_guard lock(left_batches_mutex_); std::lock_guard lock_finish(finished_mutex_); - left_queue_bloom_finished_ = left_queue_bloom_finished_ || num_expected_bloom_filters_ == 0; + left_queue_bloom_finished_ = + left_queue_bloom_finished_ || num_expected_bloom_filters_ == 0; proceed = !has_hash_table_ && left_queue_bloom_finished_; has_hash_table_ = true; } - if(proceed) - RETURN_NOT_OK(ProbeQueuedBatches(thread_index)); + if (proceed) RETURN_NOT_OK(ProbeQueuedBatches(thread_index)); right_batches_.clear(); return Status::OK(); @@ -860,17 +824,11 @@ class HashJoinBasicImpl : public HashJoinImpl { Status BuildBloomFilter(size_t thread_index) { RETURN_NOT_OK(bloom_filter_builder_->Begin( - num_threads_, - ctx_->cpu_info()->hardware_flags(), - ctx_->memory_pool(), - right_input_row_count_, - right_batches_.size(), - bloom_filter_.get())); - - return scheduler_->StartTaskGroup( - thread_index, - task_group_bloom_, - right_batches_.size()); + num_threads_, ctx_->cpu_info()->hardware_flags(), ctx_->memory_pool(), + right_input_row_count_, right_batches_.size(), bloom_filter_.get())); + + return scheduler_->StartTaskGroup(thread_index, task_group_bloom_, + right_batches_.size()); } Status BuildHashTable(size_t thread_index) { @@ -878,38 +836,33 @@ class HashJoinBasicImpl : public HashJoinImpl { BuildHashTable_num_tasks()); } - Status BloomFilterQueuedBatches_exec_task(size_t thread_index, int64_t task_id) + Status BloomFilterQueuedBatches_exec_task(size_t thread_index, int64_t task_id) { + if (cancelled_) return Status::Cancelled("Hash join cancelled"); + ExecBatch batch; { - if(cancelled_) - return Status::Cancelled("Hash join cancelled"); - ExecBatch batch; - { - std::lock_guard lock(left_batches_mutex_); - batch = std::move(left_batches_[task_id]); - ARROW_DCHECK(!batch.values.empty()); - } - RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); - { - std::lock_guard lock(left_batches_mutex_); - left_batches_[task_id] = std::move(batch); - } - return Status::OK(); + std::lock_guard lock(left_batches_mutex_); + batch = std::move(left_batches_[task_id]); + ARROW_DCHECK(!batch.values.empty()); } + RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); + { + std::lock_guard lock(left_batches_mutex_); + left_batches_[task_id] = std::move(batch); + } + return Status::OK(); + } - Status BloomFilterQueuedBatches_on_finished(size_t thread_index) + Status BloomFilterQueuedBatches_on_finished(size_t thread_index) { + if (cancelled_) return Status::Cancelled("Hash join cancelled"); + bool proceed; { - if(cancelled_) - return Status::Cancelled("Hash join cancelled"); - bool proceed; - { - std::lock_guard lock(finished_mutex_); - proceed = !left_queue_bloom_finished_ && has_hash_table_; - left_queue_bloom_finished_ = true; - } - if(proceed) - return ProbeQueuedBatches(thread_index); - return Status::OK(); + std::lock_guard lock(finished_mutex_); + proceed = !left_queue_bloom_finished_ && has_hash_table_; + left_queue_bloom_finished_ = true; } + if (proceed) return ProbeQueuedBatches(thread_index); + return Status::OK(); + } void RegisterBloomFilterQueuedBatches() { task_group_bloom_filter_queued_ = scheduler_->RegisterTaskGroup( @@ -922,7 +875,8 @@ class HashJoinBasicImpl : public HashJoinImpl { } Status BloomFilterQueuedBatches(size_t thread_index, size_t num_batches) { - return scheduler_->StartTaskGroup(thread_index, task_group_bloom_filter_queued_, num_batches); + return scheduler_->StartTaskGroup(thread_index, task_group_bloom_filter_queued_, + num_batches); } int64_t ProbeQueuedBatches_num_tasks() { @@ -960,7 +914,7 @@ class HashJoinBasicImpl : public HashJoinImpl { void RegisterProbeQueuedBatches() { task_group_queued_ = scheduler_->RegisterTaskGroup( [this](size_t thread_index, int64_t task_id) -> Status { - return ProbeQueuedBatches_exec_task(thread_index, task_id); + return ProbeQueuedBatches_exec_task(thread_index, task_id); }, [this](size_t thread_index) -> Status { return ProbeQueuedBatches_on_finished(thread_index); @@ -1052,27 +1006,25 @@ class HashJoinBasicImpl : public HashJoinImpl { ScanHashTable_num_tasks()); } - Result QueueBatchIfNeeded(size_t thread_index, int side, ExecBatch &batch) { + Result QueueBatchIfNeeded(size_t thread_index, int side, ExecBatch& batch) { if (side == 0) { - // We don't want to do the filtering while holding the lock, since that can get - // expensive. - bool needs_filtering; - { - std::lock_guard lock(left_batches_mutex_); - bloom_filters_ready_ = bloom_filters_ready_ || num_expected_bloom_filters_ == 0; - needs_filtering = bloom_filters_ready_ && num_expected_bloom_filters_ != 0; - } - if(needs_filtering) - RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); - - bool queued; - { - std::lock_guard lock(left_batches_mutex_); - queued = !bloom_filters_ready_ || !has_hash_table_; - if(queued) - left_batches_.emplace_back(std::move(batch)); - } - return queued; + // We don't want to do the filtering while holding the lock, since that can get + // expensive. + bool needs_filtering; + { + std::lock_guard lock(left_batches_mutex_); + bloom_filters_ready_ = bloom_filters_ready_ || num_expected_bloom_filters_ == 0; + needs_filtering = bloom_filters_ready_ && num_expected_bloom_filters_ != 0; + } + if (needs_filtering) RETURN_NOT_OK(ApplyBloomFiltersToBatch(thread_index, batch)); + + bool queued; + { + std::lock_guard lock(left_batches_mutex_); + queued = !bloom_filters_ready_ || !has_hash_table_; + if (queued) left_batches_.emplace_back(std::move(batch)); + } + return queued; } else { std::lock_guard lock(right_batches_mutex_); right_input_row_count_ += batch.length; @@ -1081,17 +1033,13 @@ class HashJoinBasicImpl : public HashJoinImpl { } } - Status OnRightSideFinished(size_t thread_index) - { - if(pushdown_target_ == nullptr) - { - return BuildHashTable(thread_index); - } - else - { - return BuildBloomFilter(thread_index); - } + Status OnRightSideFinished(size_t thread_index) { + if (pushdown_target_ == nullptr) { + return BuildHashTable(thread_index); + } else { + return BuildBloomFilter(thread_index); } + } Status OnLeftSideAndQueueFinished(size_t thread_index) { return ScanHashTable(thread_index); @@ -1167,8 +1115,8 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector has_match; }; std::vector local_states_; - std::vector temp_stacks_; // One per thread, but initialized earlier than ThreadLocalState - + std::vector + temp_stacks_; // One per thread, but initialized earlier than ThreadLocalState // Shared runtime state // @@ -1187,7 +1135,7 @@ class HashJoinBasicImpl : public HashJoinImpl { bool has_hash_table_; std::mutex left_batches_mutex_; - size_t right_input_row_count_; // Sum of the lengths of ExecBatches in right_batches_ + size_t right_input_row_count_; // Sum of the lengths of ExecBatches in right_batches_ std::vector right_batches_; std::mutex right_batches_mutex_; @@ -1196,11 +1144,11 @@ class HashJoinBasicImpl : public HashJoinImpl { std::unique_ptr bloom_filter_builder_; std::shared_ptr bloom_filter_; std::vector column_map_; - std::vector< std::shared_ptr > pushed_bloom_filters_; - std::vector< std::vector > bloom_filter_column_maps_; + std::vector> pushed_bloom_filters_; + std::vector> bloom_filter_column_maps_; std::mutex bloom_filters_mutex_; size_t num_expected_bloom_filters_; - HashJoinImpl *pushdown_target_; + HashJoinImpl* pushdown_target_; std::atomic num_batches_produced_; bool cancelled_; diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/hash_join.h index fc3692a8e2f..f8e649b56e5 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/compute/exec/hash_join.h @@ -109,13 +109,11 @@ class HashJoinImpl { OutputBatchCallback output_batch_callback, FinishedCallback finished_callback, TaskScheduler::ScheduleImpl schedule_task_callback, - HashJoinImpl *pushdown_target, - std::vector column_map) = 0; + HashJoinImpl* pushdown_target, std::vector column_map) = 0; virtual void ExpectBloomFilter() = 0; - virtual Status PushBloomFilter( - size_t thread_index, - std::shared_ptr filter, - std::vector column_map) = 0; + virtual Status PushBloomFilter(size_t thread_index, + std::shared_ptr filter, + std::vector column_map) = 0; virtual Status InputReceived(size_t thread_index, int side, ExecBatch batch) = 0; virtual Status InputFinished(size_t thread_index, int side) = 0; virtual void Abort(TaskScheduler::AbortContinuationImpl pos_abort_callback) = 0; diff --git a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc index 8a56f6799c9..8d8be7f904f 100644 --- a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc +++ b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc @@ -129,20 +129,20 @@ class JoinBenchmark { join_ = *HashJoinImpl::MakeBasic(); - HashJoinImpl *bloom_filter_pushdown_target = nullptr; + HashJoinImpl* bloom_filter_pushdown_target = nullptr; std::vector key_input_map; bool bloom_filter_does_not_apply_to_join = settings.join_type == JoinType::LEFT_ANTI || settings.join_type == JoinType::LEFT_OUTER || settings.join_type == JoinType::FULL_OUTER; - if(settings.bloom_filter && !bloom_filter_does_not_apply_to_join) - { - bloom_filter_pushdown_target = join_.get(); - SchemaProjectionMap probe_key_to_input = schema_mgr_->proj_maps[0].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); - int num_keys = probe_key_to_input.num_cols; - for(int i = 0; i < num_keys; i++) - key_input_map.push_back(probe_key_to_input.get(i)); + if (settings.bloom_filter && !bloom_filter_does_not_apply_to_join) { + bloom_filter_pushdown_target = join_.get(); + SchemaProjectionMap probe_key_to_input = schema_mgr_->proj_maps[0].map( + HashJoinProjection::KEY, HashJoinProjection::INPUT); + int num_keys = probe_key_to_input.num_cols; + for (int i = 0; i < num_keys; i++) + key_input_map.push_back(probe_key_to_input.get(i)); } omp_set_num_threads(settings.num_threads); @@ -155,7 +155,8 @@ class JoinBenchmark { DCHECK_OK(join_->Init( ctx_.get(), settings.join_type, !is_parallel, settings.num_threads, schema_mgr_.get(), std::move(key_cmp), std::move(filter), [](ExecBatch) {}, - [](int64_t x) {}, schedule_callback, bloom_filter_pushdown_target, std::move(key_input_map))); + [](int64_t x) {}, schedule_callback, bloom_filter_pushdown_target, + std::move(key_input_map))); } void RunJoin() { @@ -288,15 +289,14 @@ static void BM_HashJoinBasic_NullPercentage(benchmark::State& st) { HashJoinBasicBenchmarkImpl(st, settings); } -static void BM_HashJoinBasic_BloomFilter(benchmark::State &st, bool bloom_filter) -{ - BenchmarkSettings settings; - settings.bloom_filter = bloom_filter; - settings.selectivity = static_cast(st.range(0)) / 100.0; - settings.num_build_batches = static_cast(st.range(1)); - settings.num_probe_batches = settings.num_build_batches; - - HashJoinBasicBenchmarkImpl(st, settings); +static void BM_HashJoinBasic_BloomFilter(benchmark::State& st, bool bloom_filter) { + BenchmarkSettings settings; + settings.bloom_filter = bloom_filter; + settings.selectivity = static_cast(st.range(0)) / 100.0; + settings.num_build_batches = static_cast(st.range(1)); + settings.num_probe_batches = settings.num_build_batches; + + HashJoinBasicBenchmarkImpl(st, settings); } #endif diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 2b19d0029d9..3518a95e7b2 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -469,7 +469,7 @@ class HashJoinNode : public ExecNode { schema_mgr_(std::move(schema_mgr)), impl_(std::move(impl)), disable_bloom_filter_(join_options.disable_bloom_filter) { - complete_.store(false); + complete_.store(false); } static Result Make(ExecPlan* plan, std::vector inputs, @@ -573,81 +573,74 @@ class HashJoinNode : public ExecNode { } } - std::pair> GetPushdownTarget() - { - ARROW_DCHECK(!disable_bloom_filter_); - // We currently only push Bloom filters on the probe side, and only if that input is also a join. - SchemaProjectionMap probe_key_to_input = schema_mgr_->proj_maps[0].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); - int num_keys = probe_key_to_input.num_cols; - - // A mapping such that bloom_to_target[i] is the index of key i in the pushdown target's input - std::vector bloom_to_target(num_keys); - HashJoinNode *pushdown_target = this; - for(int i = 0; i < num_keys; i++) - bloom_to_target[i] = probe_key_to_input.get(i); - - for(ExecNode *candidate = inputs()[0]; - candidate->kind_name() == this->kind_name(); - candidate = candidate->inputs()[0]) - { - auto *candidate_as_join = checked_cast(inputs()[0]); - SchemaProjectionMap candidate_output_to_input = - candidate_as_join->schema_mgr_->proj_maps[0].map( - HashJoinProjection::OUTPUT, - HashJoinProjection::INPUT); - - // Check if any of the keys are missing, if they are, break - bool break_outer = false; - for(int i = 0; i < num_keys; i++) - { - // The output is from the candidate's build side - if(bloom_to_target[i] >= candidate_output_to_input.num_cols) - { - break_outer = true; - break; - } - int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); - // The output column has to have come from somewhere... - ARROW_DCHECK_NE(candidate_input_idx, schema_mgr_->kMissingField()); - } - if(break_outer) - break; - - // All keys are present, we can update the mapping - for(int i = 0; i < num_keys; i++) - { - int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); - bloom_to_target[i] = candidate_input_idx; - } - pushdown_target = candidate_as_join; + std::pair> GetPushdownTarget() { + ARROW_DCHECK(!disable_bloom_filter_); + // We currently only push Bloom filters on the probe side, and only if that input is + // also a join. + SchemaProjectionMap probe_key_to_input = + schema_mgr_->proj_maps[0].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + int num_keys = probe_key_to_input.num_cols; + + // A mapping such that bloom_to_target[i] is the index of key i in the pushdown + // target's input + std::vector bloom_to_target(num_keys); + HashJoinNode* pushdown_target = this; + for (int i = 0; i < num_keys; i++) bloom_to_target[i] = probe_key_to_input.get(i); + + for (ExecNode* candidate = inputs()[0]; candidate->kind_name() == this->kind_name(); + candidate = candidate->inputs()[0]) { + auto* candidate_as_join = checked_cast(inputs()[0]); + SchemaProjectionMap candidate_output_to_input = + candidate_as_join->schema_mgr_->proj_maps[0].map(HashJoinProjection::OUTPUT, + HashJoinProjection::INPUT); + + // Check if any of the keys are missing, if they are, break + bool break_outer = false; + for (int i = 0; i < num_keys; i++) { + // The output is from the candidate's build side + if (bloom_to_target[i] >= candidate_output_to_input.num_cols) { + break_outer = true; + break; } - return std::make_pair(pushdown_target->impl_.get(), std::move(bloom_to_target)); - } + int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); + // The output column has to have come from somewhere... + ARROW_DCHECK_NE(candidate_input_idx, schema_mgr_->kMissingField()); + } + if (break_outer) break; - Status PrepareToProduce() override - { - bool use_sync_execution = !(plan_->exec_context()->executor()); - size_t num_threads = use_sync_execution ? 1 : thread_indexer_.Capacity(); - bool bloom_filter_does_not_apply_to_join = - join_type_ == JoinType::LEFT_ANTI || join_type_ == JoinType::LEFT_OUTER || join_type_ == JoinType::FULL_OUTER; - disable_bloom_filter_ = disable_bloom_filter_ || bloom_filter_does_not_apply_to_join; - - HashJoinImpl *pushdown_target = nullptr; - std::vector column_map; - if(!disable_bloom_filter_) - std::tie(pushdown_target, column_map) = GetPushdownTarget(); - - return impl_->Init( - plan_->exec_context(), join_type_, use_sync_execution, num_threads, - schema_mgr_.get(), key_cmp_, filter_, - [this](ExecBatch batch) { this->OutputBatchCallback(batch); }, - [this](int64_t total_num_batches) { this->FinishedCallback(total_num_batches); }, - [this](std::function func) -> Status { - return this->ScheduleTaskCallback(std::move(func)); - }, - pushdown_target, - std::move(column_map)); + // All keys are present, we can update the mapping + for (int i = 0; i < num_keys; i++) { + int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); + bloom_to_target[i] = candidate_input_idx; + } + pushdown_target = candidate_as_join; } + return std::make_pair(pushdown_target->impl_.get(), std::move(bloom_to_target)); + } + + Status PrepareToProduce() override { + bool use_sync_execution = !(plan_->exec_context()->executor()); + size_t num_threads = use_sync_execution ? 1 : thread_indexer_.Capacity(); + bool bloom_filter_does_not_apply_to_join = join_type_ == JoinType::LEFT_ANTI || + join_type_ == JoinType::LEFT_OUTER || + join_type_ == JoinType::FULL_OUTER; + disable_bloom_filter_ = disable_bloom_filter_ || bloom_filter_does_not_apply_to_join; + + HashJoinImpl* pushdown_target = nullptr; + std::vector column_map; + if (!disable_bloom_filter_) + std::tie(pushdown_target, column_map) = GetPushdownTarget(); + + return impl_->Init( + plan_->exec_context(), join_type_, use_sync_execution, num_threads, + schema_mgr_.get(), key_cmp_, filter_, + [this](ExecBatch batch) { this->OutputBatchCallback(batch); }, + [this](int64_t total_num_batches) { this->FinishedCallback(total_num_batches); }, + [this](std::function func) -> Status { + return this->ScheduleTaskCallback(std::move(func)); + }, + pushdown_target, std::move(column_map)); + } Status StartProducing() override { START_COMPUTE_SPAN(span_, std::string(kind_name()) + ":" + label(), diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index e5cb24457a5..59029f4fda9 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1159,7 +1159,7 @@ TEST(HashJoin, Random) { // Run tested join implementation HashJoinNodeOptions join_options{ join_type, key_fields[0], key_fields[1], output_fields[0], - output_fields[1], key_cmp, filter }; + output_fields[1], key_cmp, filter}; join_options.disable_bloom_filter = disable_bloom_filter; std::vector> output_schema_fields; for (int i = 0; i < 2; ++i) { @@ -1904,177 +1904,148 @@ TEST(HashJoin, TrivialResidualFilter) { } } - HashJoinNodeOptions GenerateHashJoinNodeOptions( - Random64Bit &rng, - int num_left_cols, - int num_right_cols) - { - HashJoinNodeOptions opts; - opts.join_type = static_cast(rng.from_range(0, 7)); - bool is_left_join = opts.join_type == JoinType::LEFT_SEMI || - opts.join_type == JoinType::LEFT_ANTI || - opts.join_type == JoinType::LEFT_OUTER; - bool is_right_join = opts.join_type == JoinType::RIGHT_SEMI || - opts.join_type == JoinType::RIGHT_ANTI || - opts.join_type == JoinType::RIGHT_OUTER; - - int num_keys = rng.from_range(1, std::min(num_left_cols, num_right_cols)); - for(int i = 0; i < num_left_cols; i++) - { - bool is_out = rng.from_range(0, 2) != 2; - if(is_out && !is_right_join) - opts.left_output.push_back(FieldRef(i)); - } - for(int i = 0; i < num_right_cols; i++) - { - bool is_out = rng.from_range(0, 2) == 2; - if(is_out && !is_left_join) - opts.right_output.push_back(FieldRef(i)); - } - // We need at least one output - if(opts.right_output.empty() && opts.left_output.empty()) - { - if(is_left_join) - { - int col = rng.from_range(0, num_left_cols - 1); - opts.left_output.push_back(FieldRef(col)); - } - else if(is_right_join) - { - int col = rng.from_range(0, num_right_cols - 1); - opts.right_output.push_back(FieldRef(col)); - } - else - { - if(rng.from_range(0, 1) == 0) - { - int col = rng.from_range(0, num_left_cols - 1); - opts.left_output.push_back(FieldRef(col)); - } - else - { - int col = rng.from_range(0, num_right_cols - 1); - opts.right_output.push_back(FieldRef(col)); - } - } - } - - for(int i = 0; i < num_keys; i++) - { - int left = rng.from_range(0, num_left_cols - 1); - int right = rng.from_range(0, num_right_cols - 1); - bool is_or_eq = rng.from_range(0, 1) == 0; - opts.left_keys.push_back(FieldRef(left)); - opts.right_keys.push_back(FieldRef(right)); - opts.key_cmp.push_back( - is_or_eq ? JoinKeyCmp::IS : JoinKeyCmp::EQ); - } - return opts; +HashJoinNodeOptions GenerateHashJoinNodeOptions(Random64Bit& rng, int num_left_cols, + int num_right_cols) { + HashJoinNodeOptions opts; + opts.join_type = static_cast(rng.from_range(0, 7)); + bool is_left_join = opts.join_type == JoinType::LEFT_SEMI || + opts.join_type == JoinType::LEFT_ANTI || + opts.join_type == JoinType::LEFT_OUTER; + bool is_right_join = opts.join_type == JoinType::RIGHT_SEMI || + opts.join_type == JoinType::RIGHT_ANTI || + opts.join_type == JoinType::RIGHT_OUTER; + + int num_keys = rng.from_range(1, std::min(num_left_cols, num_right_cols)); + for (int i = 0; i < num_left_cols; i++) { + bool is_out = rng.from_range(0, 2) != 2; + if (is_out && !is_right_join) opts.left_output.push_back(FieldRef(i)); + } + for (int i = 0; i < num_right_cols; i++) { + bool is_out = rng.from_range(0, 2) == 2; + if (is_out && !is_left_join) opts.right_output.push_back(FieldRef(i)); + } + // We need at least one output + if (opts.right_output.empty() && opts.left_output.empty()) { + if (is_left_join) { + int col = rng.from_range(0, num_left_cols - 1); + opts.left_output.push_back(FieldRef(col)); + } else if (is_right_join) { + int col = rng.from_range(0, num_right_cols - 1); + opts.right_output.push_back(FieldRef(col)); + } else { + if (rng.from_range(0, 1) == 0) { + int col = rng.from_range(0, num_left_cols - 1); + opts.left_output.push_back(FieldRef(col)); + } else { + int col = rng.from_range(0, num_right_cols - 1); + opts.right_output.push_back(FieldRef(col)); + } } + } - void TestSingleChainOfHashJoins(Random64Bit &rng) - { - int num_joins = rng.from_range(2, 5); - std::vector opts; - int num_left_cols = rng.from_range(1, 8); - int num_right_cols = rng.from_range(1, 8); - HashJoinNodeOptions first_opt = GenerateHashJoinNodeOptions(rng, num_left_cols, num_right_cols); - opts.push_back(std::move(first_opt)); - - - std::unordered_map metadata_map; - metadata_map["min"] = "0"; - metadata_map["max"] = "10"; - auto metadata = key_value_metadata(metadata_map); - std::vector> left_fields; - for(int i = 0; i < num_left_cols; i++) - left_fields.push_back(field(std::string("l") + std::to_string(i), int32(), metadata)); - std::vector> first_right_fields; - for(int i = 0; i < num_right_cols; i++) - first_right_fields.push_back(field(std::string("r_0_") + std::to_string(i), int32(), metadata)); - - BatchesWithSchema input_left = MakeRandomBatches(schema(std::move(left_fields))); - std::vector input_right; - input_right.push_back(MakeRandomBatches(schema(std::move(first_right_fields)))); - - for(int i = 1; i < num_joins; i++) - { - int num_right_cols = rng.from_range(1, 8); - HashJoinNodeOptions opt = GenerateHashJoinNodeOptions( - rng, - opts[i - 1].left_output.size() + opts[i - 1].right_output.size(), - num_right_cols); - opts.push_back(std::move(opt)); - - std::vector> right_fields; - for(int j = 0; j < num_right_cols; j++) - right_fields.push_back(field(std::string("r_") + std::to_string(i) + "_" + std::to_string(j), int32(), metadata)); - BatchesWithSchema input = MakeRandomBatches(schema(std::move(right_fields))); - input_right.push_back(std::move(input)); - } + for (int i = 0; i < num_keys; i++) { + int left = rng.from_range(0, num_left_cols - 1); + int right = rng.from_range(0, num_right_cols - 1); + bool is_or_eq = rng.from_range(0, 1) == 0; + opts.left_keys.push_back(FieldRef(left)); + opts.right_keys.push_back(FieldRef(right)); + opts.key_cmp.push_back(is_or_eq ? JoinKeyCmp::IS : JoinKeyCmp::EQ); + } + return opts; +} - std::vector reference; - for(bool bloom_filters : { false, true }) - { - bool parallel = true; - ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); - auto exec_ctx = arrow::internal::make_unique( - default_memory_pool(), - parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - - ExecNode *left_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_left.schema, - input_left.gen(parallel, /*slow=*/false)})); - std::vector joins(num_joins); - for(int i = 0; i < num_joins; i++) - { - opts[i].disable_bloom_filter = !bloom_filters; - ExecNode *right_source; - ASSERT_OK_AND_ASSIGN( - right_source, - MakeExecNode("source", - plan.get(), {}, - SourceNodeOptions{ - input_right[i].schema, - input_right[i].gen(parallel, /*slow=*/false) - })); - - std::vector inputs; - if(i == 0) - inputs = { left_source, right_source }; - else - inputs = { joins[i - 1], right_source }; - ASSERT_OK_AND_ASSIGN(joins[i], - MakeExecNode("hashjoin", plan.get(), inputs, opts[i])); - } - AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN(std::ignore, - MakeExecNode("sink", - plan.get(), - { joins.back() }, - SinkNodeOptions{ &sink_gen })); - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); - if(!bloom_filters) - reference = std::move(result); - else - AssertExecBatchesEqual(joins.back()->output_schema(), reference, result); - } - } +void TestSingleChainOfHashJoins(Random64Bit& rng) { + int num_joins = rng.from_range(2, 5); + std::vector opts; + int num_left_cols = rng.from_range(1, 8); + int num_right_cols = rng.from_range(1, 8); + HashJoinNodeOptions first_opt = + GenerateHashJoinNodeOptions(rng, num_left_cols, num_right_cols); + opts.push_back(std::move(first_opt)); + + std::unordered_map metadata_map; + metadata_map["min"] = "0"; + metadata_map["max"] = "10"; + auto metadata = key_value_metadata(metadata_map); + std::vector> left_fields; + for (int i = 0; i < num_left_cols; i++) + left_fields.push_back(field(std::string("l") + std::to_string(i), int32(), metadata)); + std::vector> first_right_fields; + for (int i = 0; i < num_right_cols; i++) + first_right_fields.push_back( + field(std::string("r_0_") + std::to_string(i), int32(), metadata)); + + BatchesWithSchema input_left = MakeRandomBatches(schema(std::move(left_fields))); + std::vector input_right; + input_right.push_back(MakeRandomBatches(schema(std::move(first_right_fields)))); + + for (int i = 1; i < num_joins; i++) { + int num_right_cols = rng.from_range(1, 8); + HashJoinNodeOptions opt = GenerateHashJoinNodeOptions( + rng, opts[i - 1].left_output.size() + opts[i - 1].right_output.size(), + num_right_cols); + opts.push_back(std::move(opt)); + + std::vector> right_fields; + for (int j = 0; j < num_right_cols; j++) + right_fields.push_back( + field(std::string("r_") + std::to_string(i) + "_" + std::to_string(j), int32(), + metadata)); + BatchesWithSchema input = MakeRandomBatches(schema(std::move(right_fields))); + input_right.push_back(std::move(input)); + } + + std::vector reference; + for (bool bloom_filters : {false, true}) { + bool parallel = true; + ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); + auto exec_ctx = arrow::internal::make_unique( + default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - TEST(HashJoin, ChainedIntegerHashJoins) - { - Random64Bit rng(42); - int num_tests = 10; - for(int i = 0; i < num_tests; i++) - { - ARROW_SCOPED_TRACE("Test ", std::to_string(i)); - TestSingleChainOfHashJoins(rng); - } + ExecNode* left_source; + ASSERT_OK_AND_ASSIGN( + left_source, + MakeExecNode("source", plan.get(), {}, + SourceNodeOptions{input_left.schema, + input_left.gen(parallel, /*slow=*/false)})); + std::vector joins(num_joins); + for (int i = 0; i < num_joins; i++) { + opts[i].disable_bloom_filter = !bloom_filters; + ExecNode* right_source; + ASSERT_OK_AND_ASSIGN( + right_source, + MakeExecNode("source", plan.get(), {}, + SourceNodeOptions{input_right[i].schema, + input_right[i].gen(parallel, /*slow=*/false)})); + + std::vector inputs; + if (i == 0) + inputs = {left_source, right_source}; + else + inputs = {joins[i - 1], right_source}; + ASSERT_OK_AND_ASSIGN(joins[i], + MakeExecNode("hashjoin", plan.get(), inputs, opts[i])); } + AsyncGenerator> sink_gen; + ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {joins.back()}, + SinkNodeOptions{&sink_gen})); + ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + if (!bloom_filters) + reference = std::move(result); + else + AssertExecBatchesEqual(joins.back()->output_schema(), reference, result); + } +} + +TEST(HashJoin, ChainedIntegerHashJoins) { + Random64Bit rng(42); + int num_tests = 10; + for (int i = 0; i < num_tests; i++) { + ARROW_SCOPED_TRACE("Test ", std::to_string(i)); + TestSingleChainOfHashJoins(rng); + } +} } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_encode.h b/cpp/src/arrow/compute/exec/key_encode.h index 4fa6c5a1070..c8fb8f36a3e 100644 --- a/cpp/src/arrow/compute/exec/key_encode.h +++ b/cpp/src/arrow/compute/exec/key_encode.h @@ -21,14 +21,14 @@ #include #include +#include "arrow/array/data.h" +#include "arrow/compute/exec.h" #include "arrow/compute/exec/util.h" #include "arrow/compute/light_array.h" #include "arrow/memory_pool.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/bit_util.h" -#include "arrow/array/data.h" -#include "arrow/compute/exec.h" namespace arrow { namespace compute { @@ -249,51 +249,40 @@ class KeyEncoder { const std::vector& GetBatchColumns() const { return batch_all_cols_; } - static KeyColumnArray ColumnArrayFromArrayData( - const std::shared_ptr &array_data, - int start_row, - int num_rows) - { - KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( - ColumnMetadataFromDataType(array_data->type), - array_data->offset + start_row + num_rows, - array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, - array_data->buffers[1]->data(), - array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR ? array_data->buffers[2]->data() : nullptr); - return KeyEncoder::KeyColumnArray( - column_array, - array_data->offset + start_row, - num_rows); - } + static KeyColumnArray ColumnArrayFromArrayData( + const std::shared_ptr& array_data, int start_row, int num_rows) { + KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( + ColumnMetadataFromDataType(array_data->type), + array_data->offset + start_row + num_rows, + array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, + array_data->buffers[1]->data(), + array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR + ? array_data->buffers[2]->data() + : nullptr); + return KeyEncoder::KeyColumnArray(column_array, array_data->offset + start_row, + num_rows); + } - static KeyColumnMetadata ColumnMetadataFromDataType(const std::shared_ptr &type) - { - if (type->id() == Type::DICTIONARY) - { - auto bit_width = arrow::internal::checked_cast(*type).bit_width(); - ARROW_DCHECK(bit_width % 8 == 0); - return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); - } - else if (type->id() == Type::BOOL) - { - return KeyEncoder::KeyColumnMetadata(true, 0); - } - else if (is_fixed_width(type->id())) - { - return KeyEncoder::KeyColumnMetadata( - true, - arrow::internal::checked_cast(*type).bit_width() / 8); - } - else if (is_binary_like(type->id())) - { - return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); - } - else - { - ARROW_DCHECK(false); - return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); - } + static KeyColumnMetadata ColumnMetadataFromDataType( + const std::shared_ptr& type) { + if (type->id() == Type::DICTIONARY) { + auto bit_width = + arrow::internal::checked_cast(*type).bit_width(); + ARROW_DCHECK(bit_width % 8 == 0); + return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); + } else if (type->id() == Type::BOOL) { + return KeyEncoder::KeyColumnMetadata(true, 0); + } else if (is_fixed_width(type->id())) { + return KeyEncoder::KeyColumnMetadata( + true, + arrow::internal::checked_cast(*type).bit_width() / 8); + } else if (is_binary_like(type->id())) { + return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); + } else { + ARROW_DCHECK(false); + return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); } + } private: /// Prepare column array vectors. diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index 9dee0160263..e3a69ddd1c1 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -29,69 +29,52 @@ namespace arrow { namespace compute { - static KeyEncoder::KeyColumnMetadata ColumnMetadataFromDataType(const std::shared_ptr &type) - { - if (type->id() == Type::DICTIONARY) - { - auto bit_width = arrow::internal::checked_cast(*type).bit_width(); - ARROW_DCHECK(bit_width % 8 == 0); - return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); - } - else if (type->id() == Type::BOOL) - { - return KeyEncoder::KeyColumnMetadata(true, 0); - } - else if (is_fixed_width(type->id())) - { - return KeyEncoder::KeyColumnMetadata( - true, - arrow::internal::checked_cast(*type).bit_width() / 8); - } - else if (is_binary_like(type->id())) - { - return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); - } - else - { - ARROW_DCHECK(false); - return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); - } - } - - static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( - const std::shared_ptr &array_data, - int start_row, - int num_rows) - { - KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( - ColumnMetadataFromDataType(array_data->type), - array_data->offset + start_row + num_rows, - array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, - array_data->buffers[1]->data(), - array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR ? array_data->buffers[2]->data() : nullptr); - return KeyEncoder::KeyColumnArray( - column_array, - array_data->offset + start_row, - num_rows); - } +static KeyEncoder::KeyColumnMetadata ColumnMetadataFromDataType( + const std::shared_ptr& type) { + if (type->id() == Type::DICTIONARY) { + auto bit_width = + arrow::internal::checked_cast(*type).bit_width(); + ARROW_DCHECK(bit_width % 8 == 0); + return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); + } else if (type->id() == Type::BOOL) { + return KeyEncoder::KeyColumnMetadata(true, 0); + } else if (is_fixed_width(type->id())) { + return KeyEncoder::KeyColumnMetadata( + true, + arrow::internal::checked_cast(*type).bit_width() / 8); + } else if (is_binary_like(type->id())) { + return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); + } else { + ARROW_DCHECK(false); + return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); + } +} +static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( + const std::shared_ptr& array_data, int start_row, int num_rows) { + KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( + ColumnMetadataFromDataType(array_data->type), + array_data->offset + start_row + num_rows, + array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, + array_data->buffers[1]->data(), + array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR + ? array_data->buffers[2]->data() + : nullptr); + return KeyEncoder::KeyColumnArray(column_array, array_data->offset + start_row, + num_rows); +} static void ColumnArraysFromExecBatch( - const ExecBatch &batch, - int start_row, - int num_rows, - std::vector &column_arrays) -{ - int num_columns = static_cast(batch.values.size()); - column_arrays.resize(num_columns); - for (int i = 0; i < num_columns; ++i) - { - const Datum &data = batch.values[i]; - ARROW_DCHECK(data.is_array()); - const std::shared_ptr &array_data = data.array(); - column_arrays[i] = - ColumnArrayFromArrayData(array_data, start_row, num_rows); - } + const ExecBatch& batch, int start_row, int num_rows, + std::vector& column_arrays) { + int num_columns = static_cast(batch.values.size()); + column_arrays.resize(num_columns); + for (int i = 0; i < num_columns; ++i) { + const Datum& data = batch.values[i]; + ARROW_DCHECK(data.is_array()); + const std::shared_ptr& array_data = data.array(); + column_arrays[i] = ColumnArrayFromArrayData(array_data, start_row, num_rows); + } } inline uint32_t Hashing32::Round(uint32_t acc, uint32_t input) { @@ -522,22 +505,17 @@ void Hashing32::HashMultiColumn(const std::vector& cols, } } - void Hashing32::HashBatch( - const ExecBatch &key_batch, - uint32_t *hashes, - int64_t hardware_flags, - util::TempVectorStack *temp_stack, - int64_t offset, - int64_t length) - { - std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); +void Hashing32::HashBatch(const ExecBatch& key_batch, uint32_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length) { + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); - KeyEncoder::KeyEncoderContext ctx; - ctx.hardware_flags = hardware_flags; - ctx.stack = temp_stack; - HashMultiColumn(column_arrays, &ctx, hashes); - } + KeyEncoder::KeyEncoderContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + HashMultiColumn(column_arrays, &ctx, hashes); +} inline uint64_t Hashing64::Avalanche(uint64_t acc) { acc ^= (acc >> 33); @@ -958,22 +936,17 @@ void Hashing64::HashMultiColumn(const std::vector& cols, } } - void Hashing64::HashBatch( - const ExecBatch &key_batch, - uint64_t *hashes, - int64_t hardware_flags, - util::TempVectorStack *temp_stack, - int64_t offset, - int64_t length) - { - std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); +void Hashing64::HashBatch(const ExecBatch& key_batch, uint64_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length) { + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); - KeyEncoder::KeyEncoderContext ctx; - ctx.hardware_flags = hardware_flags; - ctx.stack = temp_stack; - HashMultiColumn(column_arrays, &ctx, hashes); - } + KeyEncoder::KeyEncoderContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + HashMultiColumn(column_arrays, &ctx, hashes); +} } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_hash.h b/cpp/src/arrow/compute/exec/key_hash.h index 0d76e3e5fc0..0f1032a6660 100644 --- a/cpp/src/arrow/compute/exec/key_hash.h +++ b/cpp/src/arrow/compute/exec/key_hash.h @@ -48,10 +48,9 @@ class ARROW_EXPORT Hashing32 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash); - static void HashBatch( - const ExecBatch &key_batch, uint32_t *hashes, - int64_t hardware_flags, util::TempVectorStack *temp_stack, - int64_t offset, int64_t length); + static void HashBatch(const ExecBatch& key_batch, uint32_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length); private: static const uint32_t PRIME32_1 = 0x9E3779B1; @@ -161,10 +160,9 @@ class ARROW_EXPORT Hashing64 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes); - static void HashBatch( - const ExecBatch &key_batch, uint64_t *hashes, - int64_t hardware_flags, util::TempVectorStack *temp_stack, - int64_t offset, int64_t length); + static void HashBatch(const ExecBatch& key_batch, uint64_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length); private: static const uint64_t PRIME64_1 = 0x9E3779B185EBCA87ULL; diff --git a/cpp/src/arrow/compute/kernels/row_encoder.h b/cpp/src/arrow/compute/kernels/row_encoder.h index a23d5f05605..4087472b67a 100644 --- a/cpp/src/arrow/compute/kernels/row_encoder.h +++ b/cpp/src/arrow/compute/kernels/row_encoder.h @@ -50,7 +50,6 @@ struct KeyEncoder { virtual Result> Decode(uint8_t** encoded_bytes, int32_t length, MemoryPool*) = 0; - // extract the null bitmap from the leading nullity bytes of encoded keys static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* null_bitmap, int32_t* null_count); From 0e8a7700df2c13d7bfdaa3d9ecb2dadaa6424cbd Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 13 Apr 2022 13:09:06 -0700 Subject: [PATCH 05/42] Standardize on int64_t, make rngs thread-local --- cpp/src/arrow/compute/exec/bloom_filter.cc | 17 +++++++++-------- cpp/src/arrow/compute/exec/bloom_filter.h | 18 ++++++++++-------- cpp/src/arrow/compute/exec/key_hash.cc | 4 ++-- cpp/src/arrow/compute/exec/partition_util.cc | 20 +++++++++++--------- cpp/src/arrow/compute/exec/partition_util.h | 11 ++++++----- 5 files changed, 38 insertions(+), 32 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 61031725457..fc5fbbe6b8f 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -311,21 +311,22 @@ Status BloomFilterBuilder_SingleThreaded::Begin(size_t /*num_threads*/, } Status BloomFilterBuilder_SingleThreaded::PushNextBatch(size_t /*thread_index*/, - int num_rows, + int64_t num_rows, const uint32_t* hashes) { PushNextBatchImp(num_rows, hashes); return Status::OK(); } Status BloomFilterBuilder_SingleThreaded::PushNextBatch(size_t /*thread_index*/, - int num_rows, + int64_t num_rows, const uint64_t* hashes) { PushNextBatchImp(num_rows, hashes); return Status::OK(); } template -void BloomFilterBuilder_SingleThreaded::PushNextBatchImp(int num_rows, const T* hashes) { +void BloomFilterBuilder_SingleThreaded::PushNextBatchImp(int64_t num_rows, + const T* hashes) { build_target_->Insert(hardware_flags_, num_rows, hashes); } @@ -340,27 +341,27 @@ Status BloomFilterBuilder_Parallel::Begin(size_t num_threads, int64_t hardware_f log_num_prtns_ = std::min(kMaxLogNumPrtns, bit_util::Log2(num_threads)); thread_local_states_.resize(num_threads); - prtn_locks_.Init(1 << log_num_prtns_); + prtn_locks_.Init(num_threads, 1 << log_num_prtns_); RETURN_NOT_OK(build_target->CreateEmpty(num_rows, pool)); return Status::OK(); } -Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int num_rows, +Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int64_t num_rows, const uint32_t* hashes) { PushNextBatchImp(thread_id, num_rows, hashes); return Status::OK(); } -Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int num_rows, +Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int64_t num_rows, const uint64_t* hashes) { PushNextBatchImp(thread_id, num_rows, hashes); return Status::OK(); } template -void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int num_rows, +void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num_rows, const T* hashes) { int num_prtns = 1 << log_num_prtns_; ThreadLocalState& local_state = thread_local_states_[thread_id]; @@ -393,7 +394,7 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int num_row while (num_unprocessed_partitions > 0) { int locked_prtn_id; int locked_prtn_id_pos; - prtn_locks_.AcquirePartitionLock(num_unprocessed_partitions, + prtn_locks_.AcquirePartitionLock(thread_id, num_unprocessed_partitions, unprocessed_partition_ids, /*limit_retries=*/false, /*max_retries=*/-1, &locked_prtn_id, &locked_prtn_id_pos); diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index b89343373ae..771b3c1f951 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -261,9 +261,9 @@ class ARROW_EXPORT BloomFilterBuilder { int64_t num_rows, int64_t num_batches, BlockedBloomFilter* build_target) = 0; virtual int64_t num_tasks() const { return 0; } - virtual Status PushNextBatch(size_t thread_index, int num_rows, + virtual Status PushNextBatch(size_t thread_index, int64_t num_rows, const uint32_t* hashes) = 0; - virtual Status PushNextBatch(size_t thread_index, int num_rows, + virtual Status PushNextBatch(size_t thread_index, int64_t num_rows, const uint64_t* hashes) = 0; virtual void CleanUp() {} static std::unique_ptr Make(BloomFilterBuildStrategy strategy); @@ -275,15 +275,15 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { int64_t num_rows, int64_t num_batches, BlockedBloomFilter* build_target) override; - Status PushNextBatch(size_t /*thread_index*/, int num_rows, + Status PushNextBatch(size_t /*thread_index*/, int64_t num_rows, const uint32_t* hashes) override; - Status PushNextBatch(size_t /*thread_index*/, int num_rows, + Status PushNextBatch(size_t /*thread_index*/, int64_t num_rows, const uint64_t* hashes) override; private: template - void PushNextBatchImp(int num_rows, const T* hashes); + void PushNextBatchImp(int64_t num_rows, const T* hashes); int64_t hardware_flags_; BlockedBloomFilter* build_target_; @@ -295,15 +295,17 @@ class BloomFilterBuilder_Parallel : public BloomFilterBuilder { int64_t num_rows, int64_t num_batches, BlockedBloomFilter* build_target) override; - Status PushNextBatch(size_t thread_id, int num_rows, const uint32_t* hashes) override; + Status PushNextBatch(size_t thread_id, int64_t num_rows, + const uint32_t* hashes) override; - Status PushNextBatch(size_t thread_id, int num_rows, const uint64_t* hashes) override; + Status PushNextBatch(size_t thread_id, int64_t num_rows, + const uint64_t* hashes) override; void CleanUp() override; private: template - void PushNextBatchImp(size_t thread_id, int num_rows, const T* hashes); + void PushNextBatchImp(size_t thread_id, int64_t num_rows, const T* hashes); int64_t hardware_flags_; BlockedBloomFilter* build_target_; diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index e3a69ddd1c1..45378b53f74 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -51,7 +51,7 @@ static KeyEncoder::KeyColumnMetadata ColumnMetadataFromDataType( } static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( - const std::shared_ptr& array_data, int start_row, int num_rows) { + const std::shared_ptr& array_data, int64_t start_row, int64_t num_rows) { KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( ColumnMetadataFromDataType(array_data->type), array_data->offset + start_row + num_rows, @@ -65,7 +65,7 @@ static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( } static void ColumnArraysFromExecBatch( - const ExecBatch& batch, int start_row, int num_rows, + const ExecBatch& batch, int64_t start_row, int64_t num_rows, std::vector& column_arrays) { int num_columns = static_cast(batch.values.size()); column_arrays.resize(num_columns); diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/compute/exec/partition_util.cc index ed5e37edca3..cd287c39082 100644 --- a/cpp/src/arrow/compute/exec/partition_util.cc +++ b/cpp/src/arrow/compute/exec/partition_util.cc @@ -22,23 +22,25 @@ namespace arrow { namespace compute { PartitionLocks::PartitionLocks() - : num_prtns_(0), - locks_(nullptr), - rand_seed_{0, 0, 0, 0, 0, 0, 0, 0}, - rand_engine_(rand_seed_) {} + : num_prtns_(0), locks_(nullptr), rand_engines_(nullptr) {} PartitionLocks::~PartitionLocks() { CleanUp(); } -void PartitionLocks::Init(int num_prtns) { +void PartitionLocks::Init(size_t num_threads, int num_prtns) { num_prtns_ = num_prtns; locks_.reset(new PartitionLock[num_prtns]); + rand_engines_.reset(new arrow::random::pcg32_fast[num_threads]); for (int i = 0; i < num_prtns; ++i) { locks_[i].lock.store(false); } + arrow::random::pcg32_fast seed_gen(0); + std::uniform_int_distribution seed_dist; + for (size_t i = 0; i < num_threads; i++) rand_engines_[i].seed(seed_dist(seed_gen)); } void PartitionLocks::CleanUp() { locks_.reset(); + rand_engines_.reset(); num_prtns_ = 0; } @@ -48,17 +50,17 @@ std::atomic* PartitionLocks::lock_ptr(int prtn_id) { return &(locks_[prtn_id].lock); } -int PartitionLocks::random_int(int num_values) { - return rand_distribution_(rand_engine_) % num_values; +int PartitionLocks::random_int(size_t thread_id, int num_values) { + return rand_distribution_(rand_engines_[thread_id]) % num_values; } -bool PartitionLocks::AcquirePartitionLock(int num_prtns_to_try, +bool PartitionLocks::AcquirePartitionLock(size_t thread_id, int num_prtns_to_try, const int* prtn_ids_to_try, bool limit_retries, int max_retries, int* locked_prtn_id, int* locked_prtn_id_pos) { int trial = 0; while (!limit_retries || trial <= max_retries) { - int prtn_id_pos = random_int(num_prtns_to_try); + int prtn_id_pos = random_int(thread_id, num_prtns_to_try); int prtn_id = prtn_ids_to_try[prtn_id_pos]; std::atomic* lock = lock_ptr(prtn_id); diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h index 6efda4aeeb0..32354f3d764 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -24,6 +24,7 @@ #include #include "arrow/buffer.h" #include "arrow/compute/exec/util.h" +#include "arrow/util/pcg_random.h" namespace arrow { namespace compute { @@ -94,7 +95,7 @@ class PartitionLocks { /// \brief Initializes the control, must be called before use /// /// \param num_prtns Number of partitions to synchronize - void Init(int num_prtns); + void Init(size_t num_threads, int num_prtns); /// \brief Cleans up the control, it should not be used after this call void CleanUp(); /// \brief Acquire a partition to work on one @@ -109,15 +110,15 @@ class PartitionLocks { /// without successfully acquiring a lock /// /// This method is thread safe - bool AcquirePartitionLock(int num_prtns, const int* prtns_to_try, bool limit_retries, - int max_retries, int* locked_prtn_id, + bool AcquirePartitionLock(size_t thread_id, int num_prtns, const int* prtns_to_try, + bool limit_retries, int max_retries, int* locked_prtn_id, int* locked_prtn_id_pos); /// \brief Release a partition so that other threads can work on it void ReleasePartitionLock(int prtn_id); private: std::atomic* lock_ptr(int prtn_id); - int random_int(int num_values); + int random_int(size_t thread_id, int num_values); struct PartitionLock { static constexpr int kCacheLineBytes = 64; @@ -126,9 +127,9 @@ class PartitionLocks { }; int num_prtns_; std::unique_ptr locks_; + std::unique_ptr rand_engines_; std::seed_seq rand_seed_; - std::mt19937 rand_engine_; std::uniform_int_distribution rand_distribution_; }; From 244aaf17362b324e00fd572ea1c2b76921a57ad4 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 14 Apr 2022 13:01:23 -0700 Subject: [PATCH 06/42] Delete some random unnecessary code, fix rand_int --- cpp/src/arrow/compute/exec/key_encode.h | 35 -------------------- cpp/src/arrow/compute/exec/key_hash.cc | 2 +- cpp/src/arrow/compute/exec/partition_util.cc | 11 +++--- cpp/src/arrow/compute/exec/partition_util.h | 5 +-- 4 files changed, 7 insertions(+), 46 deletions(-) diff --git a/cpp/src/arrow/compute/exec/key_encode.h b/cpp/src/arrow/compute/exec/key_encode.h index c8fb8f36a3e..58d8fb233f0 100644 --- a/cpp/src/arrow/compute/exec/key_encode.h +++ b/cpp/src/arrow/compute/exec/key_encode.h @@ -249,41 +249,6 @@ class KeyEncoder { const std::vector& GetBatchColumns() const { return batch_all_cols_; } - static KeyColumnArray ColumnArrayFromArrayData( - const std::shared_ptr& array_data, int start_row, int num_rows) { - KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( - ColumnMetadataFromDataType(array_data->type), - array_data->offset + start_row + num_rows, - array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, - array_data->buffers[1]->data(), - array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR - ? array_data->buffers[2]->data() - : nullptr); - return KeyEncoder::KeyColumnArray(column_array, array_data->offset + start_row, - num_rows); - } - - static KeyColumnMetadata ColumnMetadataFromDataType( - const std::shared_ptr& type) { - if (type->id() == Type::DICTIONARY) { - auto bit_width = - arrow::internal::checked_cast(*type).bit_width(); - ARROW_DCHECK(bit_width % 8 == 0); - return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); - } else if (type->id() == Type::BOOL) { - return KeyEncoder::KeyColumnMetadata(true, 0); - } else if (is_fixed_width(type->id())) { - return KeyEncoder::KeyColumnMetadata( - true, - arrow::internal::checked_cast(*type).bit_width() / 8); - } else if (is_binary_like(type->id())) { - return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); - } else { - ARROW_DCHECK(false); - return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); - } - } - private: /// Prepare column array vectors. /// Output column arrays represent a range of input column arrays diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index 45378b53f74..1ad4e1262c8 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -22,9 +22,9 @@ #include #include +#include "arrow/compute/exec/key_encode.h" #include "arrow/util/bit_util.h" #include "arrow/util/ubsan.h" -#include "key_encode.h" namespace arrow { namespace compute { diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/compute/exec/partition_util.cc index cd287c39082..b31ac2616d3 100644 --- a/cpp/src/arrow/compute/exec/partition_util.cc +++ b/cpp/src/arrow/compute/exec/partition_util.cc @@ -21,26 +21,25 @@ namespace arrow { namespace compute { -PartitionLocks::PartitionLocks() - : num_prtns_(0), locks_(nullptr), rand_engines_(nullptr) {} +PartitionLocks::PartitionLocks() : num_prtns_(0), locks_(nullptr), rngs_(nullptr) {} PartitionLocks::~PartitionLocks() { CleanUp(); } void PartitionLocks::Init(size_t num_threads, int num_prtns) { num_prtns_ = num_prtns; locks_.reset(new PartitionLock[num_prtns]); - rand_engines_.reset(new arrow::random::pcg32_fast[num_threads]); + rngs_.reset(new arrow::random::pcg32_fast[num_threads]); for (int i = 0; i < num_prtns; ++i) { locks_[i].lock.store(false); } arrow::random::pcg32_fast seed_gen(0); std::uniform_int_distribution seed_dist; - for (size_t i = 0; i < num_threads; i++) rand_engines_[i].seed(seed_dist(seed_gen)); + for (size_t i = 0; i < num_threads; i++) rngs_[i].seed(seed_dist(seed_gen)); } void PartitionLocks::CleanUp() { locks_.reset(); - rand_engines_.reset(); + rngs_.reset(); num_prtns_ = 0; } @@ -51,7 +50,7 @@ std::atomic* PartitionLocks::lock_ptr(int prtn_id) { } int PartitionLocks::random_int(size_t thread_id, int num_values) { - return rand_distribution_(rand_engines_[thread_id]) % num_values; + return std::uniform_int_distribution{0, num_values - 1}(rngs_[thread_id]); } bool PartitionLocks::AcquirePartitionLock(size_t thread_id, int num_prtns_to_try, diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h index 32354f3d764..36c9eda53c1 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -127,10 +127,7 @@ class PartitionLocks { }; int num_prtns_; std::unique_ptr locks_; - std::unique_ptr rand_engines_; - - std::seed_seq rand_seed_; - std::uniform_int_distribution rand_distribution_; + std::unique_ptr rngs_; }; } // namespace compute From a3dfc3bc64b277c2bf466f1925f9a9c63dc1122a Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 14 Apr 2022 15:26:14 -0700 Subject: [PATCH 07/42] Make thread sanitizer happy --- cpp/src/arrow/compute/exec/bloom_filter.cc | 4 ++-- cpp/src/arrow/compute/exec/bloom_filter.h | 2 ++ cpp/src/arrow/compute/exec/bloom_filter_avx2.cc | 2 ++ cpp/src/arrow/compute/exec/hash_join.cc | 2 +- cpp/src/arrow/compute/exec/partition_util.h | 6 +++--- 5 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index fc5fbbe6b8f..ac19db9aa63 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -374,13 +374,13 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num PartitionSort::Eval( num_rows, num_prtns, partition_ranges, - [hashes, num_prtns](int row_id) { + [hashes, num_prtns](int64_t row_id) { constexpr int kLogBlocksKeptTogether = 7; constexpr int kPrtnIdBitOffset = BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); }, - [hashes, partitioned_hashes](int row_id, int output_pos) { + [hashes, partitioned_hashes](int64_t row_id, int output_pos) { partitioned_hashes[output_pos] = hashes[row_id]; }); diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index 771b3c1f951..1e7c4fbe05f 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -163,8 +163,10 @@ class ARROW_EXPORT BlockedBloomFilter { inline void Insert(uint64_t hash) { uint64_t m = mask(hash); + std::atomic_thread_fence(std::memory_order_acquire); uint64_t& b = blocks_[block_id(hash)]; b |= m; + std::atomic_thread_fence(std::memory_order_release); } void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes); diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc index 40069618023..d51f69671b1 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -115,10 +115,12 @@ int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { } __m256i mask = mask_avx2(hash); __m256i block_id = block_id_avx2(hash); + std::atomic_thread_fence(std::memory_order_acquire); blocks_[_mm256_extract_epi64(block_id, 0)] |= _mm256_extract_epi64(mask, 0); blocks_[_mm256_extract_epi64(block_id, 1)] |= _mm256_extract_epi64(mask, 1); blocks_[_mm256_extract_epi64(block_id, 2)] |= _mm256_extract_epi64(mask, 2); blocks_[_mm256_extract_epi64(block_id, 3)] |= _mm256_extract_epi64(mask, 3); + std::atomic_thread_fence(std::memory_order_release); } return num_rows - (num_rows % unroll); diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 8cd25c1ad22..aca11370cf5 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -666,7 +666,7 @@ class HashJoinBasicImpl : public HashJoinImpl { // Start with full selection for the current minibatch memset(selected.data(), 0xff, bit_vector_bytes); - for (size_t ifilter = 0; ifilter < pushed_bloom_filters_.size(); ifilter++) { + for (size_t ifilter = 0; ifilter < num_expected_bloom_filters_; ifilter++) { std::vector keys(bloom_filter_column_maps_[ifilter].size()); for (size_t i = 0; i < keys.size(); i++) { int input_idx = bloom_filter_column_maps_[ifilter][i]; diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h index 36c9eda53c1..702e1b8f78b 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -60,14 +60,14 @@ class PartitionSort { /// out_arr: [2, 5, 3, 5, 4, 7] /// prtn_ranges: [0, 1, 5, 6] template - static void Eval(int num_rows, int num_prtns, uint16_t* prtn_ranges, + static void Eval(int64_t num_rows, int num_prtns, uint16_t* prtn_ranges, INPUT_PRTN_ID_FN prtn_id_impl, OUTPUT_POS_FN output_pos_impl) { ARROW_DCHECK(num_rows > 0 && num_rows <= (1 << 15)); ARROW_DCHECK(num_prtns >= 1 && num_prtns <= (1 << 15)); memset(prtn_ranges, 0, (num_prtns + 1) * sizeof(uint16_t)); - for (int i = 0; i < num_rows; ++i) { + for (int64_t i = 0; i < num_rows; ++i) { int prtn_id = static_cast(prtn_id_impl(i)); ++prtn_ranges[prtn_id + 1]; } @@ -79,7 +79,7 @@ class PartitionSort { sum = sum_next; } - for (int i = 0; i < num_rows; ++i) { + for (int64_t i = 0; i < num_rows; ++i) { int prtn_id = static_cast(prtn_id_impl(i)); int pos = prtn_ranges[prtn_id + 1]++; output_pos_impl(i, pos); From df5f8cd7ff68d881578dfb94a52db3d3e8f573c5 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 14 Apr 2022 16:28:58 -0700 Subject: [PATCH 08/42] Make Bloom filter test multithreaded. This upsets tsan. --- .../arrow/compute/exec/bloom_filter_test.cc | 113 ++++++++++++++---- 1 file changed, 89 insertions(+), 24 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index a3b5cded153..62b3554a6cc 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -24,6 +24,7 @@ #include #include "arrow/compute/exec/bloom_filter.h" #include "arrow/compute/exec/key_hash.h" +#include "arrow/compute/exec/task_util.h" #include "arrow/compute/exec/test_util.h" #include "arrow/compute/exec/util.h" #include "arrow/util/bitmap_ops.h" @@ -32,39 +33,103 @@ namespace arrow { namespace compute { -Status BuildBloomFilter(BloomFilterBuildStrategy strategy, int64_t hardware_flags, - MemoryPool* pool, int64_t num_rows, - std::function get_hash32_impl, - std::function get_hash64_impl, - BlockedBloomFilter* target) { - constexpr int batch_size_max = 32 * 1024; - int64_t num_batches = bit_util::CeilDiv(num_rows, batch_size_max); - - auto builder = BloomFilterBuilder::Make(strategy); - - std::vector thread_local_hashes32; - std::vector thread_local_hashes64; - thread_local_hashes32.resize(batch_size_max); - thread_local_hashes64.resize(batch_size_max); - - RETURN_NOT_OK(builder->Begin(/*num_threads=*/1, hardware_flags, pool, num_rows, - bit_util::CeilDiv(num_rows, batch_size_max), target)); - - for (int64_t i = 0; i < num_batches; ++i) { +constexpr int kBatchSizeMax = 32 * 1024; +Status BuildBloomFilter_Serial( + std::unique_ptr& builder, int64_t num_rows, int64_t num_batches, + std::function get_hash32_impl, + std::function get_hash64_impl, + BlockedBloomFilter* target) { + std::vector hashes32(kBatchSizeMax); + std::vector hashes64(kBatchSizeMax); + for (int64_t i = 0; i < num_batches; i++) { size_t thread_index = 0; int batch_size = static_cast( - std::min(num_rows - i * batch_size_max, static_cast(batch_size_max))); + std::min(num_rows - i * kBatchSizeMax, static_cast(kBatchSizeMax))); if (target->NumHashBitsUsed() > 32) { - uint64_t* hashes = thread_local_hashes64.data(); - get_hash64_impl(i * batch_size_max, batch_size, hashes); + uint64_t* hashes = hashes64.data(); + get_hash64_impl(i * kBatchSizeMax, batch_size, hashes); RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); } else { - uint32_t* hashes = thread_local_hashes32.data(); - get_hash32_impl(i * batch_size_max, batch_size, hashes); + uint32_t* hashes = hashes32.data(); + get_hash32_impl(i * kBatchSizeMax, batch_size, hashes); RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); } } + return Status::OK(); +} + +Status BuildBloomFilter_Parallel( + std::unique_ptr& builder, size_t num_threads, int64_t num_rows, + int64_t num_batches, std::function get_hash32_impl, + std::function get_hash64_impl, + BlockedBloomFilter* target) { + std::mutex mutex; + ThreadIndexer thread_indexer; + std::unique_ptr scheduler = TaskScheduler::Make(); + std::vector> thread_local_hashes32(num_threads); + std::vector> thread_local_hashes64(num_threads); + for (std::vector& h : thread_local_hashes32) h.resize(kBatchSizeMax); + for (std::vector& h : thread_local_hashes64) h.resize(kBatchSizeMax); + + auto group = scheduler->RegisterTaskGroup( + [&](size_t thread_index, int64_t task_id) -> Status { + int batch_size = static_cast(std::min(num_rows - task_id * kBatchSizeMax, + static_cast(kBatchSizeMax))); + if (target->NumHashBitsUsed() > 32) { + uint64_t* hashes = thread_local_hashes64[thread_index].data(); + get_hash64_impl(task_id * kBatchSizeMax, batch_size, hashes); + RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); + } else { + uint32_t* hashes = thread_local_hashes32[thread_index].data(); + get_hash32_impl(task_id * kBatchSizeMax, batch_size, hashes); + RETURN_NOT_OK(builder->PushNextBatch(thread_index, batch_size, hashes)); + } + return Status::OK(); + }, + [&](size_t thread_index) -> Status { + mutex.unlock(); + return Status::OK(); + }); + scheduler->RegisterEnd(); + auto tp = arrow::internal::GetCpuThreadPool(); + RETURN_NOT_OK(scheduler->StartScheduling( + 0, + [&](std::function func) -> Status { + return tp->Spawn([&, func] { + size_t tid = thread_indexer(); + std::ignore = func(tid); + }); + }, + 2 * num_threads, false)); + mutex.lock(); + RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); + mutex.lock(); + mutex.unlock(); + return Status::OK(); +} + +Status BuildBloomFilter(BloomFilterBuildStrategy strategy, int64_t hardware_flags, + MemoryPool* pool, int64_t num_rows, + std::function get_hash32_impl, + std::function get_hash64_impl, + BlockedBloomFilter* target) { + int64_t num_batches = bit_util::CeilDiv(num_rows, kBatchSizeMax); + + bool is_serial = strategy == BloomFilterBuildStrategy::SINGLE_THREADED; + auto builder = BloomFilterBuilder::Make(strategy); + size_t num_threads = is_serial ? 1 : arrow::GetCpuThreadPoolCapacity(); + RETURN_NOT_OK(builder->Begin(num_threads, hardware_flags, pool, num_rows, + bit_util::CeilDiv(num_rows, kBatchSizeMax), target)); + + if (is_serial) + RETURN_NOT_OK(BuildBloomFilter_Serial(builder, num_rows, num_batches, + std::move(get_hash32_impl), + std::move(get_hash64_impl), target)); + else + RETURN_NOT_OK(BuildBloomFilter_Parallel(builder, num_threads, num_rows, num_batches, + std::move(get_hash32_impl), + std::move(get_hash64_impl), target)); builder->CleanUp(); return Status::OK(); From eb8117ecc0089f632cea044eea251f8a8f7357cc Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 20 Apr 2022 15:35:22 -0700 Subject: [PATCH 09/42] Dumb bug --- cpp/src/arrow/compute/exec/hash_join.cc | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index aca11370cf5..1dbc781a79a 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -132,9 +132,7 @@ class HashJoinBasicImpl : public HashJoinImpl { cancelled_ = false; right_side_finished_ = false; left_side_finished_ = false; - bloom_filters_ready_ = - pushdown_target_ == - nullptr; // If we're not pushing the Bloom filter anywhere, the filter is ready + bloom_filters_ready_ = false; left_queue_bloom_finished_ = false; left_queue_probe_finished_ = false; From 48204fd645612084587196509afcde468dca8efe Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 20 Apr 2022 15:38:45 -0700 Subject: [PATCH 10/42] Remove my thread fences --- cpp/src/arrow/compute/exec/bloom_filter.h | 2 -- cpp/src/arrow/compute/exec/bloom_filter_avx2.cc | 2 -- 2 files changed, 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index 1e7c4fbe05f..771b3c1f951 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -163,10 +163,8 @@ class ARROW_EXPORT BlockedBloomFilter { inline void Insert(uint64_t hash) { uint64_t m = mask(hash); - std::atomic_thread_fence(std::memory_order_acquire); uint64_t& b = blocks_[block_id(hash)]; b |= m; - std::atomic_thread_fence(std::memory_order_release); } void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes); diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc index d51f69671b1..40069618023 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -115,12 +115,10 @@ int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { } __m256i mask = mask_avx2(hash); __m256i block_id = block_id_avx2(hash); - std::atomic_thread_fence(std::memory_order_acquire); blocks_[_mm256_extract_epi64(block_id, 0)] |= _mm256_extract_epi64(mask, 0); blocks_[_mm256_extract_epi64(block_id, 1)] |= _mm256_extract_epi64(mask, 1); blocks_[_mm256_extract_epi64(block_id, 2)] |= _mm256_extract_epi64(mask, 2); blocks_[_mm256_extract_epi64(block_id, 3)] |= _mm256_extract_epi64(mask, 3); - std::atomic_thread_fence(std::memory_order_release); } return num_rows - (num_rows % unroll); From 68398291b65b8a330af5522f7a759b6784b553b6 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 21 Apr 2022 12:30:17 -0700 Subject: [PATCH 11/42] Rebase --- cpp/src/arrow/compute/exec/key_hash.cc | 56 ++------------------------ cpp/src/arrow/compute/light_array.cc | 5 ++- cpp/src/arrow/compute/light_array.h | 6 +-- 3 files changed, 10 insertions(+), 57 deletions(-) diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index 1ad4e1262c8..a1d56022da8 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -29,54 +29,6 @@ namespace arrow { namespace compute { -static KeyEncoder::KeyColumnMetadata ColumnMetadataFromDataType( - const std::shared_ptr& type) { - if (type->id() == Type::DICTIONARY) { - auto bit_width = - arrow::internal::checked_cast(*type).bit_width(); - ARROW_DCHECK(bit_width % 8 == 0); - return KeyEncoder::KeyColumnMetadata(true, bit_width / 8); - } else if (type->id() == Type::BOOL) { - return KeyEncoder::KeyColumnMetadata(true, 0); - } else if (is_fixed_width(type->id())) { - return KeyEncoder::KeyColumnMetadata( - true, - arrow::internal::checked_cast(*type).bit_width() / 8); - } else if (is_binary_like(type->id())) { - return KeyEncoder::KeyColumnMetadata(false, sizeof(uint32_t)); - } else { - ARROW_DCHECK(false); - return KeyEncoder::KeyColumnMetadata(true, sizeof(int)); - } -} - -static KeyEncoder::KeyColumnArray ColumnArrayFromArrayData( - const std::shared_ptr& array_data, int64_t start_row, int64_t num_rows) { - KeyEncoder::KeyColumnArray column_array = KeyEncoder::KeyColumnArray( - ColumnMetadataFromDataType(array_data->type), - array_data->offset + start_row + num_rows, - array_data->buffers[0] != NULLPTR ? array_data->buffers[0]->data() : nullptr, - array_data->buffers[1]->data(), - array_data->buffers.size() >= 3 && array_data->buffers[2] != NULLPTR - ? array_data->buffers[2]->data() - : nullptr); - return KeyEncoder::KeyColumnArray(column_array, array_data->offset + start_row, - num_rows); -} - -static void ColumnArraysFromExecBatch( - const ExecBatch& batch, int64_t start_row, int64_t num_rows, - std::vector& column_arrays) { - int num_columns = static_cast(batch.values.size()); - column_arrays.resize(num_columns); - for (int i = 0; i < num_columns; ++i) { - const Datum& data = batch.values[i]; - ARROW_DCHECK(data.is_array()); - const std::shared_ptr& array_data = data.array(); - column_arrays[i] = ColumnArrayFromArrayData(array_data, start_row, num_rows); - } -} - inline uint32_t Hashing32::Round(uint32_t acc, uint32_t input) { acc += input * PRIME32_2; acc = ROTL(acc, 13); @@ -508,8 +460,8 @@ void Hashing32::HashMultiColumn(const std::vector& cols, void Hashing32::HashBatch(const ExecBatch& key_batch, uint32_t* hashes, int64_t hardware_flags, util::TempVectorStack* temp_stack, int64_t offset, int64_t length) { - std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays); KeyEncoder::KeyEncoderContext ctx; ctx.hardware_flags = hardware_flags; @@ -939,8 +891,8 @@ void Hashing64::HashMultiColumn(const std::vector& cols, void Hashing64::HashBatch(const ExecBatch& key_batch, uint64_t* hashes, int64_t hardware_flags, util::TempVectorStack* temp_stack, int64_t offset, int64_t length) { - std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, column_arrays); + std::vector column_arrays; + ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays); KeyEncoder::KeyEncoderContext ctx; ctx.hardware_flags = hardware_flags; diff --git a/cpp/src/arrow/compute/light_array.cc b/cpp/src/arrow/compute/light_array.cc index 390dcc6cbf1..4fae01eb6bf 100644 --- a/cpp/src/arrow/compute/light_array.cc +++ b/cpp/src/arrow/compute/light_array.cc @@ -138,7 +138,7 @@ Result ColumnMetadataFromDataType( } Result ColumnArrayFromArrayData( - const std::shared_ptr& array_data, int start_row, int num_rows) { + const std::shared_ptr& array_data, int64_t start_row, int64_t num_rows) { ARROW_ASSIGN_OR_RAISE(KeyColumnMetadata metadata, ColumnMetadataFromDataType(array_data->type)); KeyColumnArray column_array = KeyColumnArray( @@ -165,7 +165,8 @@ Status ColumnMetadatasFromExecBatch(const ExecBatch& batch, return Status::OK(); } -Status ColumnArraysFromExecBatch(const ExecBatch& batch, int start_row, int num_rows, +Status ColumnArraysFromExecBatch(const ExecBatch& batch, int64_t start_row, + int64_t num_rows, std::vector* column_arrays) { int num_columns = static_cast(batch.values.size()); column_arrays->resize(num_columns); diff --git a/cpp/src/arrow/compute/light_array.h b/cpp/src/arrow/compute/light_array.h index 0856e3e8aa5..dd13aa0647f 100644 --- a/cpp/src/arrow/compute/light_array.h +++ b/cpp/src/arrow/compute/light_array.h @@ -171,7 +171,7 @@ ARROW_EXPORT Result ColumnMetadataFromDataType( /// The caller should ensure this is only called on "key" columns. /// \see ColumnMetadataFromDataType for details ARROW_EXPORT Result ColumnArrayFromArrayData( - const std::shared_ptr& array_data, int start_row, int num_rows); + const std::shared_ptr& array_data, int64_t start_row, int64_t num_rows); /// \brief Create KeyColumnMetadata instances from an ExecBatch /// @@ -188,8 +188,8 @@ ARROW_EXPORT Status ColumnMetadatasFromExecBatch( /// /// All columns in `batch` must be eligible "key" columns and have an array shape /// \see ColumnArrayFromArrayData for more details -ARROW_EXPORT Status ColumnArraysFromExecBatch(const ExecBatch& batch, int start_row, - int num_rows, +ARROW_EXPORT Status ColumnArraysFromExecBatch(const ExecBatch& batch, int64_t start_row, + int64_t num_rows, std::vector* column_arrays); /// \brief Create KeyColumnArray instances from an ExecBatch From 05b935c85ac4b35d9bb2327b0804b903bb6093a0 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 21 Apr 2022 13:10:46 -0700 Subject: [PATCH 12/42] Switch to int64_t again --- cpp/src/arrow/compute/exec/hash_join.cc | 11 +++++++---- cpp/src/arrow/compute/exec/key_hash.cc | 18 ++++++++++-------- cpp/src/arrow/compute/exec/key_hash.h | 12 ++++++------ 3 files changed, 23 insertions(+), 18 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 1dbc781a79a..f24c645a004 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -97,6 +97,7 @@ class HashJoinBasicImpl : public HashJoinImpl { // As of right now, we ignore the `num_threads` argument, so later we will have to // readd `num_threads_ = num_threads;` num_threads_ = GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1; + // num_threads_ = GetCpuThreadPoolCapacity() + 1; START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", {{"detail", filter.ToString()}, @@ -671,8 +672,9 @@ class HashJoinBasicImpl : public HashJoinImpl { keys[i] = batch[input_idx]; } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(keys))); - Hashing32::HashBatch(key_batch, hashes.data(), ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], 0, key_batch.length); + RETURN_NOT_OK(Hashing32::HashBatch( + key_batch, hashes.data(), ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], 0, key_batch.length)); pushed_bloom_filters_[ifilter]->Find(ctx_->cpu_info()->hardware_flags(), key_batch.length, hashes.data(), bv.data()); @@ -720,8 +722,9 @@ class HashJoinBasicImpl : public HashJoinImpl { for (int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) { int64_t length = std::min(static_cast(key_batch.length - i), static_cast(util::MiniBatch::kMiniBatchLength)); - Hashing32::HashBatch(key_batch, hashes, ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], i, length); + RETURN_NOT_OK(Hashing32::HashBatch(key_batch, hashes, + ctx_->cpu_info()->hardware_flags(), + &temp_stacks_[thread_index], i, length)); RETURN_NOT_OK(bloom_filter_builder_->PushNextBatch(thread_index, length, hashes)); } return Status::OK(); diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index a1d56022da8..e81ed64b6f2 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -457,16 +457,17 @@ void Hashing32::HashMultiColumn(const std::vector& cols, } } -void Hashing32::HashBatch(const ExecBatch& key_batch, uint32_t* hashes, - int64_t hardware_flags, util::TempVectorStack* temp_stack, - int64_t offset, int64_t length) { +Status Hashing32::HashBatch(const ExecBatch& key_batch, uint32_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length) { std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays); + RETURN_NOT_OK(ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays)); KeyEncoder::KeyEncoderContext ctx; ctx.hardware_flags = hardware_flags; ctx.stack = temp_stack; HashMultiColumn(column_arrays, &ctx, hashes); + return Status::OK(); } inline uint64_t Hashing64::Avalanche(uint64_t acc) { @@ -888,16 +889,17 @@ void Hashing64::HashMultiColumn(const std::vector& cols, } } -void Hashing64::HashBatch(const ExecBatch& key_batch, uint64_t* hashes, - int64_t hardware_flags, util::TempVectorStack* temp_stack, - int64_t offset, int64_t length) { +Status Hashing64::HashBatch(const ExecBatch& key_batch, uint64_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length) { std::vector column_arrays; - ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays); + RETURN_NOT_OK(ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays)); KeyEncoder::KeyEncoderContext ctx; ctx.hardware_flags = hardware_flags; ctx.stack = temp_stack; HashMultiColumn(column_arrays, &ctx, hashes); + return Status::OK(); } } // namespace compute diff --git a/cpp/src/arrow/compute/exec/key_hash.h b/cpp/src/arrow/compute/exec/key_hash.h index 0f1032a6660..05f39bb729a 100644 --- a/cpp/src/arrow/compute/exec/key_hash.h +++ b/cpp/src/arrow/compute/exec/key_hash.h @@ -48,9 +48,9 @@ class ARROW_EXPORT Hashing32 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash); - static void HashBatch(const ExecBatch& key_batch, uint32_t* hashes, - int64_t hardware_flags, util::TempVectorStack* temp_stack, - int64_t offset, int64_t length); + static Status HashBatch(const ExecBatch& key_batch, uint32_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length); private: static const uint32_t PRIME32_1 = 0x9E3779B1; @@ -160,9 +160,9 @@ class ARROW_EXPORT Hashing64 { static void HashMultiColumn(const std::vector& cols, KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes); - static void HashBatch(const ExecBatch& key_batch, uint64_t* hashes, - int64_t hardware_flags, util::TempVectorStack* temp_stack, - int64_t offset, int64_t length); + static Status HashBatch(const ExecBatch& key_batch, uint64_t* hashes, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t offset, int64_t length); private: static const uint64_t PRIME64_1 = 0x9E3779B185EBCA87ULL; From 7e594b25b82be663a19f6d11880a63ebe43353f0 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 21 Apr 2022 13:48:18 -0700 Subject: [PATCH 13/42] Switch to unique_ptr --- cpp/src/arrow/compute/exec/hash_join.cc | 10 +++++----- cpp/src/arrow/compute/exec/hash_join.h | 2 +- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index f24c645a004..7ed560685cc 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -139,7 +139,7 @@ class HashJoinBasicImpl : public HashJoinImpl { scheduler_ = TaskScheduler::Make(); if (pushdown_target_) { - bloom_filter_ = std::make_shared(); + bloom_filter_ = arrow::internal::make_unique(); bloom_filter_builder_ = BloomFilterBuilder::Make( use_sync_execution ? BloomFilterBuildStrategy::SINGLE_THREADED : BloomFilterBuildStrategy::PARALLEL); @@ -166,7 +166,7 @@ class HashJoinBasicImpl : public HashJoinImpl { scheduler_->Abort(std::move(pos_abort_callback)); } - Status PushBloomFilter(size_t thread_index, std::shared_ptr filter, + Status PushBloomFilter(size_t thread_index, std::unique_ptr filter, std::vector column_map) override { bool proceed; { @@ -681,7 +681,7 @@ class HashJoinBasicImpl : public HashJoinImpl { arrow::internal::BitmapAnd(bv.data(), 0, selected.data(), 0, key_batch.length, 0, selected.data()); } - auto selected_buffer = std::make_shared(selected.data(), bit_vector_bytes); + auto selected_buffer = arrow::internal::make_unique(selected.data(), bit_vector_bytes); ArrayData selected_arraydata(boolean(), batch.length, {nullptr, std::move(selected_buffer)}); Datum selected_datum(selected_arraydata); @@ -1143,9 +1143,9 @@ class HashJoinBasicImpl : public HashJoinImpl { // Bloom filter stuff // std::unique_ptr bloom_filter_builder_; - std::shared_ptr bloom_filter_; + std::unique_ptr bloom_filter_; std::vector column_map_; - std::vector> pushed_bloom_filters_; + std::vector> pushed_bloom_filters_; std::vector> bloom_filter_column_maps_; std::mutex bloom_filters_mutex_; size_t num_expected_bloom_filters_; diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/hash_join.h index f8e649b56e5..9739cbc6436 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/compute/exec/hash_join.h @@ -112,7 +112,7 @@ class HashJoinImpl { HashJoinImpl* pushdown_target, std::vector column_map) = 0; virtual void ExpectBloomFilter() = 0; virtual Status PushBloomFilter(size_t thread_index, - std::shared_ptr filter, + std::unique_ptr filter, std::vector column_map) = 0; virtual Status InputReceived(size_t thread_index, int side, ExecBatch batch) = 0; virtual Status InputFinished(size_t thread_index, int side) = 0; diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 59029f4fda9..4f80bfe4972 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1112,7 +1112,7 @@ TEST(HashJoin, Random) { ARROW_SCOPED_TRACE(join_type_name, " ", key_cmp_str, " parallel = ", (parallel ? "true" : "false"), - " bloom_filter =", (disable_bloom_filter ? "false" : "true")); + " bloom_filter = ", (disable_bloom_filter ? "false" : "true")); // Run reference join implementation std::vector null_in_key_vectors[2]; From 5ec4e0c07125fab260b199bbea385354830b6f83 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 21 Apr 2022 15:37:51 -0700 Subject: [PATCH 14/42] Make bloom filter blocks atomics --- cpp/src/arrow/compute/exec/bloom_filter.cc | 14 ++++++++------ cpp/src/arrow/compute/exec/bloom_filter.h | 9 +++++---- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index ac19db9aa63..aae6d0f0d90 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -105,10 +105,10 @@ Status BlockedBloomFilter::CreateEmpty(int64_t num_rows_to_insert, MemoryPool* p // Allocate and zero out bit vector // - int64_t buffer_size = num_blocks_ * sizeof(uint64_t); + int64_t buffer_size = num_blocks_ * sizeof(std::atomic); ARROW_ASSIGN_OR_RAISE(buf_, AllocateBuffer(buffer_size, pool)); - blocks_ = reinterpret_cast(buf_->mutable_data()); - memset(blocks_, 0, buffer_size); + blocks_ = reinterpret_cast*>(buf_->mutable_data()); + memset(reinterpret_cast(blocks_), 0, buffer_size); return Status::OK(); } @@ -262,14 +262,16 @@ void BlockedBloomFilter::SingleFold(int num_folds) { // int64_t num_slices = 1LL << num_folds; int64_t num_slice_blocks = (num_blocks_ >> num_folds); - uint64_t* target_slice = blocks_; + std::atomic* target_slice = blocks_; // OR bits of all the slices and store result in the first slice // for (int64_t slice = 1; slice < num_slices; ++slice) { - const uint64_t* source_slice = blocks_ + slice * num_slice_blocks; + const std::atomic* source_slice = blocks_ + slice * num_slice_blocks; for (int i = 0; i < num_slice_blocks; ++i) { - target_slice[i] |= source_slice[i]; + uint64_t target = target_slice[i].load(std::memory_order_relaxed); + uint64_t source = source_slice[i].load(std::memory_order_relaxed); + target_slice[i].store(target | source, std::memory_order_relaxed); } } diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index 771b3c1f951..c29180a0f56 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -113,7 +113,7 @@ class ARROW_EXPORT BlockedBloomFilter { inline bool Find(uint64_t hash) const { uint64_t m = mask(hash); - uint64_t b = blocks_[block_id(hash)]; + uint64_t b = blocks_[block_id(hash)].load(std::memory_order_relaxed); return (b & m) == m; } @@ -163,8 +163,9 @@ class ARROW_EXPORT BlockedBloomFilter { inline void Insert(uint64_t hash) { uint64_t m = mask(hash); - uint64_t& b = blocks_[block_id(hash)]; - b |= m; + std::atomic& b = blocks_[block_id(hash)]; + uint64_t b_old = b.load(std::memory_order_relaxed); + b.store(b_old | m, std::memory_order_relaxed); } void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes); @@ -237,7 +238,7 @@ class ARROW_EXPORT BlockedBloomFilter { std::shared_ptr buf_; // Pointer to mutable data owned by Buffer // - uint64_t* blocks_; + std::atomic* blocks_; }; // We have two separate implementations of building a Bloom filter, multi-threaded and From 4f5dbcdc11224cebf956e95be76857f77aa0d294 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 22 Apr 2022 11:14:57 -0700 Subject: [PATCH 15/42] Switch back to non-atomics, but silence tsan for these functions --- cpp/src/arrow/compute/exec/bloom_filter.cc | 16 ++++++++-------- cpp/src/arrow/compute/exec/bloom_filter.h | 17 ++++++++++++----- cpp/src/arrow/compute/exec/bloom_filter_avx2.cc | 1 + 3 files changed, 21 insertions(+), 13 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index aae6d0f0d90..82a7a454039 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -105,15 +105,16 @@ Status BlockedBloomFilter::CreateEmpty(int64_t num_rows_to_insert, MemoryPool* p // Allocate and zero out bit vector // - int64_t buffer_size = num_blocks_ * sizeof(std::atomic); + int64_t buffer_size = num_blocks_ * sizeof(uint64_t); ARROW_ASSIGN_OR_RAISE(buf_, AllocateBuffer(buffer_size, pool)); - blocks_ = reinterpret_cast*>(buf_->mutable_data()); - memset(reinterpret_cast(blocks_), 0, buffer_size); + blocks_ = reinterpret_cast(buf_->mutable_data()); + memset(blocks_, 0, buffer_size); return Status::OK(); } template +NO_TSAN void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { for (int64_t i = 0; i < num_rows; ++i) { Insert(hashes[i]); @@ -143,6 +144,7 @@ void BlockedBloomFilter::Insert(int64_t hardware_flags, int64_t num_rows, } template +NO_TSAN void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, uint8_t* result_bit_vector, bool enable_prefetch) const { int64_t num_processed = 0; @@ -262,16 +264,14 @@ void BlockedBloomFilter::SingleFold(int num_folds) { // int64_t num_slices = 1LL << num_folds; int64_t num_slice_blocks = (num_blocks_ >> num_folds); - std::atomic* target_slice = blocks_; + uint64_t* target_slice = blocks_; // OR bits of all the slices and store result in the first slice // for (int64_t slice = 1; slice < num_slices; ++slice) { - const std::atomic* source_slice = blocks_ + slice * num_slice_blocks; + const uint64_t* source_slice = blocks_ + slice * num_slice_blocks; for (int i = 0; i < num_slice_blocks; ++i) { - uint64_t target = target_slice[i].load(std::memory_order_relaxed); - uint64_t source = source_slice[i].load(std::memory_order_relaxed); - target_slice[i].store(target | source, std::memory_order_relaxed); + target_slice[i] |= source_slice[i]; } } diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index c29180a0f56..a7a5f3f1449 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -33,6 +33,12 @@ namespace arrow { namespace compute { +#if defined(__clang__) || defined(__GNUC__) +#define NO_TSAN __attribute__((no_sanitize_thread)) +#else +#define NO_TSAN +#endif + // A set of pre-generated bit masks from a 64-bit word. // // It is used to map selected bits of hash to a bit mask that will be used in @@ -111,9 +117,10 @@ class ARROW_EXPORT BlockedBloomFilter { public: BlockedBloomFilter() : log_num_blocks_(0), num_blocks_(0), blocks_(NULLPTR) {} + NO_TSAN inline bool Find(uint64_t hash) const { uint64_t m = mask(hash); - uint64_t b = blocks_[block_id(hash)].load(std::memory_order_relaxed); + uint64_t b = blocks_[block_id(hash)]; return (b & m) == m; } @@ -161,11 +168,11 @@ class ARROW_EXPORT BlockedBloomFilter { private: Status CreateEmpty(int64_t num_rows_to_insert, MemoryPool* pool); + NO_TSAN inline void Insert(uint64_t hash) { uint64_t m = mask(hash); - std::atomic& b = blocks_[block_id(hash)]; - uint64_t b_old = b.load(std::memory_order_relaxed); - b.store(b_old | m, std::memory_order_relaxed); + uint64_t& b = blocks_[block_id(hash)]; + b |= m; } void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes); @@ -238,7 +245,7 @@ class ARROW_EXPORT BlockedBloomFilter { std::shared_ptr buf_; // Pointer to mutable data owned by Buffer // - std::atomic* blocks_; + uint64_t* blocks_; }; // We have two separate implementations of building a Bloom filter, multi-threaded and diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc index 40069618023..b4a5ba3969d 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -102,6 +102,7 @@ int64_t BlockedBloomFilter::Find_avx2(int64_t num_rows, const uint64_t* hashes, } template +NO_TSAN int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { constexpr int unroll = 4; From c7e250bedeaaf9f77fd9ec13d91dd5a40b67f264 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 22 Apr 2022 11:15:39 -0700 Subject: [PATCH 16/42] clang-format --- cpp/src/arrow/compute/exec/bloom_filter.cc | 9 ++++----- cpp/src/arrow/compute/exec/bloom_filter_avx2.cc | 3 +-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 82a7a454039..999875532e6 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -114,8 +114,7 @@ Status BlockedBloomFilter::CreateEmpty(int64_t num_rows_to_insert, MemoryPool* p } template -NO_TSAN -void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { +NO_TSAN void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { for (int64_t i = 0; i < num_rows; ++i) { Insert(hashes[i]); } @@ -144,9 +143,9 @@ void BlockedBloomFilter::Insert(int64_t hardware_flags, int64_t num_rows, } template -NO_TSAN -void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, - uint8_t* result_bit_vector, bool enable_prefetch) const { +NO_TSAN void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, + uint8_t* result_bit_vector, + bool enable_prefetch) const { int64_t num_processed = 0; uint64_t bits = 0ULL; diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc index b4a5ba3969d..3050fa1b5b7 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -102,8 +102,7 @@ int64_t BlockedBloomFilter::Find_avx2(int64_t num_rows, const uint64_t* hashes, } template -NO_TSAN -int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { +NO_TSAN int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { constexpr int unroll = 4; for (int64_t i = 0; i < num_rows / unroll; ++i) { From 3af8ce4f10aa3647b806f1d52423da5054514c6b Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 22 Apr 2022 13:47:09 -0700 Subject: [PATCH 17/42] clang-format --- cpp/src/arrow/compute/exec/hash_join.cc | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 7ed560685cc..a14adb838ed 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -90,8 +90,7 @@ class HashJoinBasicImpl : public HashJoinImpl { OutputBatchCallback output_batch_callback, FinishedCallback finished_callback, TaskScheduler::ScheduleImpl schedule_task_callback, - HashJoinImpl* pushdown_target, - std::vector column_map) override { + HashJoinImpl* pushdown_target, std::vector column_map) override { // TODO(ARROW-15732) // Each side of join might have an IO thread being called from. // As of right now, we ignore the `num_threads` argument, so later we will have to @@ -117,12 +116,12 @@ class HashJoinBasicImpl : public HashJoinImpl { local_states_[i].is_has_match_initialized = false; } temp_stacks_.resize(num_threads_); - for(size_t i = 0; i < temp_stacks_.size(); i++) - RETURN_NOT_OK(temp_stacks_[i].Init(ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); + for (size_t i = 0; i < temp_stacks_.size(); i++) + RETURN_NOT_OK(temp_stacks_[i].Init( + ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); dict_probe_.Init(num_threads_); - pushdown_target_ = pushdown_target; column_map_ = std::move(column_map); if (pushdown_target_) pushdown_target_->ExpectBloomFilter(); @@ -681,7 +680,8 @@ class HashJoinBasicImpl : public HashJoinImpl { arrow::internal::BitmapAnd(bv.data(), 0, selected.data(), 0, key_batch.length, 0, selected.data()); } - auto selected_buffer = arrow::internal::make_unique(selected.data(), bit_vector_bytes); + auto selected_buffer = + arrow::internal::make_unique(selected.data(), bit_vector_bytes); ArrayData selected_arraydata(boolean(), batch.length, {nullptr, std::move(selected_buffer)}); Datum selected_datum(selected_arraydata); From 21eb53a225645a64f70d5f44c97ef6fd9a6e54f2 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 22 Apr 2022 17:04:17 -0700 Subject: [PATCH 18/42] static_cast --- cpp/src/arrow/compute/exec/hash_join.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index a14adb838ed..1f9157d6b4b 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -711,7 +711,7 @@ class HashJoinBasicImpl : public HashJoinImpl { schema_mgr_->proj_maps[1].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); std::vector key_columns(key_to_in.num_cols); for (size_t i = 0; i < key_columns.size(); i++) { - int input_idx = key_to_in.get(i); + int input_idx = key_to_in.get(static_cast(i)); key_columns[i] = input_batch[input_idx]; } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); From 7c58dd114139bc010b0b730f72e2a259111991a4 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 22 Apr 2022 20:21:12 -0700 Subject: [PATCH 19/42] Use condition variable in test --- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index 62b3554a6cc..5fe340ae354 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -71,6 +71,8 @@ Status BuildBloomFilter_Parallel( for (std::vector& h : thread_local_hashes32) h.resize(kBatchSizeMax); for (std::vector& h : thread_local_hashes64) h.resize(kBatchSizeMax); + std::condition_variable cv; + std::unique_lock lk(mutex, std::defer_lock); auto group = scheduler->RegisterTaskGroup( [&](size_t thread_index, int64_t task_id) -> Status { int batch_size = static_cast(std::min(num_rows - task_id * kBatchSizeMax, @@ -87,7 +89,8 @@ Status BuildBloomFilter_Parallel( return Status::OK(); }, [&](size_t thread_index) -> Status { - mutex.unlock(); + lk.unlock(); + cv.notify_one(); return Status::OK(); }); scheduler->RegisterEnd(); @@ -101,10 +104,11 @@ Status BuildBloomFilter_Parallel( }); }, 2 * num_threads, false)); - mutex.lock(); - RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); - mutex.lock(); - mutex.unlock(); + { + lk.lock(); + RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); + cv.wait(lk); + } return Status::OK(); } From dbb773f930b4590d3400c6384604ea50ba681953 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 25 Apr 2022 12:39:39 -0700 Subject: [PATCH 20/42] Make it green --- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 8 ++++---- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 8 +++++--- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index 5fe340ae354..59ac0a05835 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -103,11 +103,11 @@ Status BuildBloomFilter_Parallel( std::ignore = func(tid); }); }, - 2 * num_threads, false)); + static_cast(2 * num_threads), false)); { - lk.lock(); - RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); - cv.wait(lk); + lk.lock(); + RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); + cv.wait(lk); } return Status::OK(); } diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 4f80bfe4972..37de1da8037 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1981,9 +1981,11 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { for (int i = 1; i < num_joins; i++) { int num_right_cols = rng.from_range(1, 8); - HashJoinNodeOptions opt = GenerateHashJoinNodeOptions( - rng, opts[i - 1].left_output.size() + opts[i - 1].right_output.size(), - num_right_cols); + HashJoinNodeOptions opt = + GenerateHashJoinNodeOptions(rng, + static_cast(opts[i - 1].left_output.size() + + opts[i - 1].right_output.size()), + num_right_cols); opts.push_back(std::move(opt)); std::vector> right_fields; From 80301a41844e14ba4209ac91a463cd537be2cb8d Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 26 Apr 2022 12:47:12 -0700 Subject: [PATCH 21/42] Respond to Weston comments --- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 7 +++---- cpp/src/arrow/compute/exec/hash_join.cc | 4 ++-- cpp/src/arrow/compute/exec/hash_join_node.cc | 7 +++++++ cpp/src/arrow/compute/exec/hash_join_node_test.cc | 12 ++++++------ 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index 59ac0a05835..e7d10f9ad5e 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -89,7 +89,6 @@ Status BuildBloomFilter_Parallel( return Status::OK(); }, [&](size_t thread_index) -> Status { - lk.unlock(); cv.notify_one(); return Status::OK(); }); @@ -98,12 +97,12 @@ Status BuildBloomFilter_Parallel( RETURN_NOT_OK(scheduler->StartScheduling( 0, [&](std::function func) -> Status { - return tp->Spawn([&, func] { + return tp->Spawn([&, func]() { size_t tid = thread_indexer(); - std::ignore = func(tid); + ARROW_DCHECK_OK(func(tid)); }); }, - static_cast(2 * num_threads), false)); + static_cast(num_threads), false)); { lk.lock(); RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 1f9157d6b4b..8624e18f15d 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -94,7 +94,7 @@ class HashJoinBasicImpl : public HashJoinImpl { // TODO(ARROW-15732) // Each side of join might have an IO thread being called from. // As of right now, we ignore the `num_threads` argument, so later we will have to - // readd `num_threads_ = num_threads;` + // re-add `num_threads_ = num_threads;` num_threads_ = GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1; // num_threads_ = GetCpuThreadPoolCapacity() + 1; @@ -110,7 +110,7 @@ class HashJoinBasicImpl : public HashJoinImpl { filter_ = std::move(filter); output_batch_callback_ = std::move(output_batch_callback); finished_callback_ = std::move(finished_callback); - local_states_.resize(GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1); + local_states_.resize(num_threads_); for (size_t i = 0; i < local_states_.size(); ++i) { local_states_[i].is_initialized = false; local_states_[i].is_has_match_initialized = false; diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 3518a95e7b2..f978271cd00 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -573,6 +573,13 @@ class HashJoinNode : public ExecNode { } } + // The Bloom filter is built on the build side of some upstream join. For a join to + // evaluate the Bloom filter on its input columns, it has to rearrange its input columns + // to match the column order of the Bloom filter. + // + // The first part of the pair is the HashJoin to actually perform the pushdown into. + // The second part is a mapping such that column_map[i] is the index of key i in + // the first part's input. std::pair> GetPushdownTarget() { ARROW_DCHECK(!disable_bloom_filter_); // We currently only push Bloom filters on the probe side, and only if that input is diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 37de1da8037..4946e5b7d62 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1999,10 +1999,10 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { std::vector reference; for (bool bloom_filters : {false, true}) { - bool parallel = true; + bool kParallel = true; ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); auto exec_ctx = arrow::internal::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); + default_memory_pool(), kParallel ? arrow::internal::GetCpuThreadPool() : nullptr); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); ExecNode* left_source; @@ -2010,7 +2010,7 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { left_source, MakeExecNode("source", plan.get(), {}, SourceNodeOptions{input_left.schema, - input_left.gen(parallel, /*slow=*/false)})); + input_left.gen(kParallel, /*slow=*/false)})); std::vector joins(num_joins); for (int i = 0; i < num_joins; i++) { opts[i].disable_bloom_filter = !bloom_filters; @@ -2019,7 +2019,7 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { right_source, MakeExecNode("source", plan.get(), {}, SourceNodeOptions{input_right[i].schema, - input_right[i].gen(parallel, /*slow=*/false)})); + input_right[i].gen(kParallel, /*slow=*/false)})); std::vector inputs; if (i == 0) @@ -2030,8 +2030,8 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { MakeExecNode("hashjoin", plan.get(), inputs, opts[i])); } AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {joins.back()}, - SinkNodeOptions{&sink_gen})); + ASSERT_OK( + MakeExecNode("sink", plan.get(), {joins.back()}, SinkNodeOptions{&sink_gen})); ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); if (!bloom_filters) reference = std::move(result); From 9f67171480afa33b53c67b653241ee0e374caecb Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 26 Apr 2022 12:50:32 -0700 Subject: [PATCH 22/42] ARROW_EXPORT --- cpp/src/arrow/compute/exec/bloom_filter.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index a7a5f3f1449..dcac19d570b 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -277,7 +277,7 @@ class ARROW_EXPORT BloomFilterBuilder { static std::unique_ptr Make(BloomFilterBuildStrategy strategy); }; -class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { +class ARROW_EXPORT BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, @@ -297,7 +297,7 @@ class BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { BlockedBloomFilter* build_target_; }; -class BloomFilterBuilder_Parallel : public BloomFilterBuilder { +class ARROW_EXPORT BloomFilterBuilder_Parallel : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, From 7aa7458f6c9e7027ca564a0386efe59fc70b6e15 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 26 Apr 2022 14:56:43 -0700 Subject: [PATCH 23/42] More ARROW_EXPORT --- cpp/src/arrow/compute/exec/task_util.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/task_util.h b/cpp/src/arrow/compute/exec/task_util.h index 5dc66aee167..c219d4a8f63 100644 --- a/cpp/src/arrow/compute/exec/task_util.h +++ b/cpp/src/arrow/compute/exec/task_util.h @@ -32,7 +32,7 @@ namespace compute { // whenever it is modified by a concurrent thread on a different CPU core. // template -class AtomicWithPadding { +class ARROW_EXPORT AtomicWithPadding { private: static constexpr int kCacheLineSize = 64; uint8_t padding_before[kCacheLineSize]; From a10a6a89d96c744f74ed53d0ee93df50f9d85083 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 27 Apr 2022 16:25:16 -0700 Subject: [PATCH 24/42] Remove line --- cpp/src/arrow/compute/exec/hash_join.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 8624e18f15d..2291985442a 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -96,7 +96,6 @@ class HashJoinBasicImpl : public HashJoinImpl { // As of right now, we ignore the `num_threads` argument, so later we will have to // re-add `num_threads_ = num_threads;` num_threads_ = GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1; - // num_threads_ = GetCpuThreadPoolCapacity() + 1; START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", {{"detail", filter.ToString()}, From 9f23c880180dce11619a16a03fde6545950f49f2 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 29 Apr 2022 11:06:12 -0600 Subject: [PATCH 25/42] Pray to the mighty barney that his condition variable may work --- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index e7d10f9ad5e..02b0f18ac16 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -63,7 +63,6 @@ Status BuildBloomFilter_Parallel( int64_t num_batches, std::function get_hash32_impl, std::function get_hash64_impl, BlockedBloomFilter* target) { - std::mutex mutex; ThreadIndexer thread_indexer; std::unique_ptr scheduler = TaskScheduler::Make(); std::vector> thread_local_hashes32(num_threads); @@ -72,7 +71,8 @@ Status BuildBloomFilter_Parallel( for (std::vector& h : thread_local_hashes64) h.resize(kBatchSizeMax); std::condition_variable cv; - std::unique_lock lk(mutex, std::defer_lock); + std::mutex mutex; + bool done = false; auto group = scheduler->RegisterTaskGroup( [&](size_t thread_index, int64_t task_id) -> Status { int batch_size = static_cast(std::min(num_rows - task_id * kBatchSizeMax, @@ -89,7 +89,11 @@ Status BuildBloomFilter_Parallel( return Status::OK(); }, [&](size_t thread_index) -> Status { - cv.notify_one(); + { + std::unique_lock lk(mutex); + done = true; + } + cv.notify_all(); return Status::OK(); }); scheduler->RegisterEnd(); @@ -104,9 +108,9 @@ Status BuildBloomFilter_Parallel( }, static_cast(num_threads), false)); { - lk.lock(); + std::unique_lock lk(mutex); RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); - cv.wait(lk); + cv.wait(lk, [&]() { return done; }); } return Status::OK(); } From 54d5022f16b3d5a34c4fb5dfc1ee93ce8f556d1e Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 29 Apr 2022 12:03:45 -0600 Subject: [PATCH 26/42] Add an underscore --- cpp/src/arrow/compute/exec/hash_join.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 2291985442a..8d059fb8f86 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -100,7 +100,7 @@ class HashJoinBasicImpl : public HashJoinImpl { START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", {{"detail", filter.ToString()}, {"join.kind", ToString(join_type)}, - {"join.threads", static_cast(num_threads)}}); + {"join.threads", static_cast(num_threads_)}}); ctx_ = ctx; join_type_ = join_type; From f1335e1e456b15fbeb06a4c0f33451c9c475f135 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 29 Apr 2022 13:29:53 -0600 Subject: [PATCH 27/42] Document parameter, explicitly initialize variable in constructor --- cpp/src/arrow/compute/exec/hash_join.cc | 2 ++ cpp/src/arrow/compute/exec/partition_util.h | 1 + 2 files changed, 3 insertions(+) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 8d059fb8f86..6fd464d3bcb 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -40,6 +40,8 @@ class HashJoinBasicImpl : public HashJoinImpl { struct ThreadLocalState; public: + HashJoinBasicImpl() : num_expected_bloom_filters_(0) {} + Status InputReceived(size_t thread_index, int side, ExecBatch batch) override { if (cancelled_) { return Status::Cancelled("Hash join cancelled"); diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h index 702e1b8f78b..7ab64fbb091 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -100,6 +100,7 @@ class PartitionLocks { void CleanUp(); /// \brief Acquire a partition to work on one /// + /// \param thread_id The index of the thread trying to acquire the partition lock /// \param num_prtns Length of prtns_to_try, must be <= num_prtns used in Init /// \param prtns_to_try An array of partitions that still have remaining work /// \param limit_retries If false, this method will spinwait forever until success From 9929b8458b8344fe153f24e6aa4e48f72e32c588 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 29 Apr 2022 14:42:31 -0600 Subject: [PATCH 28/42] Document another thing --- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 6 +++--- cpp/src/arrow/compute/exec/partition_util.h | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 4946e5b7d62..b2a737e77c2 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1954,7 +1954,7 @@ HashJoinNodeOptions GenerateHashJoinNodeOptions(Random64Bit& rng, int num_left_c return opts; } -void TestSingleChainOfHashJoins(Random64Bit& rng) { + void TestSingleChainOfHashJoins(Random64Bit& rng, int test_id) { int num_joins = rng.from_range(2, 5); std::vector opts; int num_left_cols = rng.from_range(1, 8); @@ -2042,10 +2042,10 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { TEST(HashJoin, ChainedIntegerHashJoins) { Random64Bit rng(42); - int num_tests = 10; + int num_tests = 100; for (int i = 0; i < num_tests; i++) { ARROW_SCOPED_TRACE("Test ", std::to_string(i)); - TestSingleChainOfHashJoins(rng); + TestSingleChainOfHashJoins(rng, i); } } diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/compute/exec/partition_util.h index 7ab64fbb091..07fb91f2f1a 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -94,6 +94,7 @@ class PartitionLocks { ~PartitionLocks(); /// \brief Initializes the control, must be called before use /// + /// \param num_threads Maximum number of threads that will access the partitions /// \param num_prtns Number of partitions to synchronize void Init(size_t num_threads, int num_prtns); /// \brief Cleans up the control, it should not be used after this call From f68ea78b694c085312a4099a66575dc610feb6fe Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Sat, 30 Apr 2022 15:12:46 -0600 Subject: [PATCH 29/42] Fix dumb bug --- cpp/src/arrow/compute/exec/hash_join_node.cc | 2 +- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index f978271cd00..d7716b725b5 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -596,7 +596,7 @@ class HashJoinNode : public ExecNode { for (ExecNode* candidate = inputs()[0]; candidate->kind_name() == this->kind_name(); candidate = candidate->inputs()[0]) { - auto* candidate_as_join = checked_cast(inputs()[0]); + auto* candidate_as_join = checked_cast(candidate); SchemaProjectionMap candidate_output_to_input = candidate_as_join->schema_mgr_->proj_maps[0].map(HashJoinProjection::OUTPUT, HashJoinProjection::INPUT); diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index b2a737e77c2..125ad295f8d 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1954,7 +1954,7 @@ HashJoinNodeOptions GenerateHashJoinNodeOptions(Random64Bit& rng, int num_left_c return opts; } - void TestSingleChainOfHashJoins(Random64Bit& rng, int test_id) { +void TestSingleChainOfHashJoins(Random64Bit& rng) { int num_joins = rng.from_range(2, 5); std::vector opts; int num_left_cols = rng.from_range(1, 8); @@ -2045,7 +2045,7 @@ TEST(HashJoin, ChainedIntegerHashJoins) { int num_tests = 100; for (int i = 0; i < num_tests; i++) { ARROW_SCOPED_TRACE("Test ", std::to_string(i)); - TestSingleChainOfHashJoins(rng, i); + TestSingleChainOfHashJoins(rng); } } From 75bf76525de610f8516476df7009e27af0f27d5b Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Sat, 30 Apr 2022 15:39:32 -0600 Subject: [PATCH 30/42] Run fewer tests --- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 125ad295f8d..7e8e2cf6fc7 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -2042,7 +2042,7 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { TEST(HashJoin, ChainedIntegerHashJoins) { Random64Bit rng(42); - int num_tests = 100; + int num_tests = 30; for (int i = 0; i < num_tests; i++) { ARROW_SCOPED_TRACE("Test ", std::to_string(i)); TestSingleChainOfHashJoins(rng); From 1aaeb6fc43b364522b69e2d10ec461dc77afecbd Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 5 May 2022 13:54:44 -0700 Subject: [PATCH 31/42] Respond to michal comments --- cpp/src/arrow/compute/exec/hash_join.cc | 30 +++++----- cpp/src/arrow/compute/exec/hash_join_node.cc | 58 +++++++++++++++++--- cpp/src/arrow/compute/exec/partition_util.cc | 4 +- 3 files changed, 67 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 6fd464d3bcb..c49bf16257a 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -116,10 +116,6 @@ class HashJoinBasicImpl : public HashJoinImpl { local_states_[i].is_initialized = false; local_states_[i].is_has_match_initialized = false; } - temp_stacks_.resize(num_threads_); - for (size_t i = 0; i < temp_stacks_.size(); i++) - RETURN_NOT_OK(temp_stacks_[i].Init( - ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); dict_probe_.Init(num_threads_); @@ -204,7 +200,7 @@ class HashJoinBasicImpl : public HashJoinImpl { encoder->Clear(); } - void InitLocalStateIfNeeded(size_t thread_index) { + Status InitLocalStateIfNeeded(size_t thread_index) { DCHECK_LT(thread_index, local_states_.size()); ThreadLocalState& local_state = local_states_[thread_index]; if (!local_state.is_initialized) { @@ -214,9 +210,11 @@ class HashJoinBasicImpl : public HashJoinImpl { if (has_payload) { InitEncoder(0, HashJoinProjection::PAYLOAD, &local_state.exec_batch_payloads); } - + RETURN_NOT_OK(local_state.temp_stack.Init( + ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); local_state.is_initialized = true; } + return Status::OK(); } Status EncodeBatch(int side, HashJoinProjection projection_handle, RowEncoder* encoder, @@ -484,7 +482,7 @@ class HashJoinBasicImpl : public HashJoinImpl { (schema_mgr_->proj_maps[1].num_cols(HashJoinProjection::PAYLOAD) > 0); ThreadLocalState& local_state = local_states_[thread_index]; - InitLocalStateIfNeeded(thread_index); + RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); ExecBatch left_key; ExecBatch left_payload; @@ -596,7 +594,7 @@ class HashJoinBasicImpl : public HashJoinImpl { Status ProbeBatch(size_t thread_index, const ExecBatch& batch) { ThreadLocalState& local_state = local_states_[thread_index]; - InitLocalStateIfNeeded(thread_index); + RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); local_state.exec_batch_keys.Clear(); @@ -658,11 +656,12 @@ class HashJoinBasicImpl : public HashJoinImpl { Status ApplyBloomFiltersToBatch(size_t thread_index, ExecBatch& batch) { if (batch.length == 0) return Status::OK(); - size_t bit_vector_bytes = (batch.length + 7) / 8; + int64_t bit_vector_bytes = bit_util::BytesForBits(batch.length); std::vector selected(bit_vector_bytes); std::vector hashes(batch.length); std::vector bv(bit_vector_bytes); + RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); // Start with full selection for the current minibatch memset(selected.data(), 0xff, bit_vector_bytes); for (size_t ifilter = 0; ifilter < num_expected_bloom_filters_; ifilter++) { @@ -674,7 +673,7 @@ class HashJoinBasicImpl : public HashJoinImpl { ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(keys))); RETURN_NOT_OK(Hashing32::HashBatch( key_batch, hashes.data(), ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], 0, key_batch.length)); + &local_states_[thread_index].temp_stack, 0, key_batch.length)); pushed_bloom_filters_[ifilter]->Find(ctx_->cpu_info()->hardware_flags(), key_batch.length, hashes.data(), bv.data()); @@ -717,7 +716,9 @@ class HashJoinBasicImpl : public HashJoinImpl { } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); - util::TempVectorHolder hash_holder(&temp_stacks_[thread_index], + RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); + ThreadLocalState& tls = local_states_[thread_index]; + util::TempVectorHolder hash_holder(&tls.temp_stack, util::MiniBatch::kMiniBatchLength); uint32_t* hashes = hash_holder.mutable_data(); for (int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) { @@ -725,7 +726,7 @@ class HashJoinBasicImpl : public HashJoinImpl { static_cast(util::MiniBatch::kMiniBatchLength)); RETURN_NOT_OK(Hashing32::HashBatch(key_batch, hashes, ctx_->cpu_info()->hardware_flags(), - &temp_stacks_[thread_index], i, length)); + &tls.temp_stack, i, length)); RETURN_NOT_OK(bloom_filter_builder_->PushNextBatch(thread_index, length, hashes)); } return Status::OK(); @@ -950,7 +951,7 @@ class HashJoinBasicImpl : public HashJoinImpl { hash_table_scan_unit_ * (task_id + 1))); ThreadLocalState& local_state = local_states_[thread_index]; - InitLocalStateIfNeeded(thread_index); + RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); std::vector& id_left = local_state.no_match; std::vector& id_right = local_state.match; @@ -1115,10 +1116,9 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector match_right; bool is_has_match_initialized; std::vector has_match; + util::TempVectorStack temp_stack; }; std::vector local_states_; - std::vector - temp_stacks_; // One per thread, but initialized earlier than ThreadLocalState // Shared runtime state // diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index d7716b725b5..2f24c8a34a0 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -580,8 +580,42 @@ class HashJoinNode : public ExecNode { // The first part of the pair is the HashJoin to actually perform the pushdown into. // The second part is a mapping such that column_map[i] is the index of key i in // the first part's input. + // If we should disable Bloom filter, returns nullptr and an empty vector, and sets + // the disable_bloom_filter_ flag. std::pair> GetPushdownTarget() { - ARROW_DCHECK(!disable_bloom_filter_); + // A build-side Bloom filter tells us if a row is definitely not in the build side. + // This allows us to early-eliminate rows or early-accept rows depending on the type + // of join. Left Outer Join and Full Outer Join output all rows, so a build-side Bloom + // filter would only allow us to early-output. Left Antijoin outputs only if there is + // no match, so again early output. We don't implement early output for now, so we + // must disallow these types of joins. + bool bloom_filter_does_not_apply_to_join = join_type_ == JoinType::LEFT_ANTI || + join_type_ == JoinType::LEFT_OUTER || + join_type_ == JoinType::FULL_OUTER; + disable_bloom_filter_ = disable_bloom_filter_ || bloom_filter_does_not_apply_to_join; + + SchemaProjectionMap build_keys_to_input = + schema_mgr_->proj_maps[1].map(HashJoinProjection::KEY, HashJoinProjection::INPUT); + // Bloom filter currently doesn't support dictionaries. + for (int i = 0; i < build_keys_to_input.num_cols; i++) { + int idx = build_keys_to_input.get(i); + bool is_dict = + inputs_[1]->output_schema()->field(idx)->type()->id() == Type::DICTIONARY; + if (is_dict) { + disable_bloom_filter_ = true; + break; + } + } + + bool all_comparisons_is = true; + for (JoinKeyCmp cmp : key_cmp_) all_comparisons_is &= cmp == JoinKeyCmp::IS; + + if ((join_type_ == JoinType::RIGHT_OUTER || join_type_ == JoinType::FULL_OUTER) && + all_comparisons_is) + disable_bloom_filter_ = true; + + if (disable_bloom_filter_) return {nullptr, {}}; + // We currently only push Bloom filters on the probe side, and only if that input is // also a join. SchemaProjectionMap probe_key_to_input = @@ -604,7 +638,9 @@ class HashJoinNode : public ExecNode { // Check if any of the keys are missing, if they are, break bool break_outer = false; for (int i = 0; i < num_keys; i++) { - // The output is from the candidate's build side + // Since all of the probe side columns are before the build side columns, + // if the index of an output is greater than the number of probe-side input + // columns, it must have come from the candidate's build side. if (bloom_to_target[i] >= candidate_output_to_input.num_cols) { break_outer = true; break; @@ -615,6 +651,17 @@ class HashJoinNode : public ExecNode { } if (break_outer) break; + // The Bloom filter will filter out nulls, which may cause a Right/Full Outer Join + // to incorrectly output some rows with nulls padding the probe-side rows. This may + // cause a row with all null keys to be emitted. This is normally not an issue + // with EQ, but if all comparisons are IS (i.e. all-null is accepted), this could + // produce incorrect rows. + bool can_produce_build_side_nulls = + candidate_as_join->join_type_ == JoinType::RIGHT_OUTER || + candidate_as_join->join_type_ == JoinType::FULL_OUTER; + + if (all_comparisons_is || can_produce_build_side_nulls) break; + // All keys are present, we can update the mapping for (int i = 0; i < num_keys; i++) { int candidate_input_idx = candidate_output_to_input.get(bloom_to_target[i]); @@ -628,15 +675,10 @@ class HashJoinNode : public ExecNode { Status PrepareToProduce() override { bool use_sync_execution = !(plan_->exec_context()->executor()); size_t num_threads = use_sync_execution ? 1 : thread_indexer_.Capacity(); - bool bloom_filter_does_not_apply_to_join = join_type_ == JoinType::LEFT_ANTI || - join_type_ == JoinType::LEFT_OUTER || - join_type_ == JoinType::FULL_OUTER; - disable_bloom_filter_ = disable_bloom_filter_ || bloom_filter_does_not_apply_to_join; HashJoinImpl* pushdown_target = nullptr; std::vector column_map; - if (!disable_bloom_filter_) - std::tie(pushdown_target, column_map) = GetPushdownTarget(); + std::tie(pushdown_target, column_map) = GetPushdownTarget(); return impl_->Init( plan_->exec_context(), join_type_, use_sync_execution, num_threads, diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/compute/exec/partition_util.cc index b31ac2616d3..e99007c45a3 100644 --- a/cpp/src/arrow/compute/exec/partition_util.cc +++ b/cpp/src/arrow/compute/exec/partition_util.cc @@ -65,7 +65,7 @@ bool PartitionLocks::AcquirePartitionLock(size_t thread_id, int num_prtns_to_try std::atomic* lock = lock_ptr(prtn_id); bool expected = false; - if (lock->compare_exchange_weak(expected, true)) { + if (lock->compare_exchange_weak(expected, true, std::memory_order_acquire)) { *locked_prtn_id = prtn_id; *locked_prtn_id_pos = prtn_id_pos; return true; @@ -82,7 +82,7 @@ bool PartitionLocks::AcquirePartitionLock(size_t thread_id, int num_prtns_to_try void PartitionLocks::ReleasePartitionLock(int prtn_id) { ARROW_DCHECK(prtn_id >= 0 && prtn_id < num_prtns_); std::atomic* lock = lock_ptr(prtn_id); - lock->store(false); + lock->store(false, std::memory_order_release); } } // namespace compute From 643eeb5ffdf6e662f6a96c288f414367ca679242 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 11:15:17 -0700 Subject: [PATCH 32/42] Fix CV and fix TSAN BloomFilterBuilder (thanks Michal and Weston) --- cpp/src/arrow/compute/exec/bloom_filter.cc | 17 +++-- cpp/src/arrow/compute/exec/bloom_filter.h | 8 --- .../arrow/compute/exec/bloom_filter_avx2.cc | 2 +- .../arrow/compute/exec/bloom_filter_test.cc | 6 +- cpp/src/arrow/compute/exec/util.cc | 71 ++++++++++++++++--- cpp/src/arrow/compute/exec/util.h | 2 + 6 files changed, 77 insertions(+), 29 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 999875532e6..303b02bc37d 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -114,7 +114,7 @@ Status BlockedBloomFilter::CreateEmpty(int64_t num_rows_to_insert, MemoryPool* p } template -NO_TSAN void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { +void BlockedBloomFilter::InsertImp(int64_t num_rows, const T* hashes) { for (int64_t i = 0; i < num_rows; ++i) { Insert(hashes[i]); } @@ -143,9 +143,8 @@ void BlockedBloomFilter::Insert(int64_t hardware_flags, int64_t num_rows, } template -NO_TSAN void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, - uint8_t* result_bit_vector, - bool enable_prefetch) const { +void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, + uint8_t* result_bit_vector, bool enable_prefetch) const { int64_t num_processed = 0; uint64_t bits = 0ULL; @@ -364,7 +363,14 @@ Status BloomFilterBuilder_Parallel::PushNextBatch(size_t thread_id, int64_t num_ template void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num_rows, const T* hashes) { - int num_prtns = 1 << log_num_prtns_; + // Partition IDs are calculated using the higher bits of the block ID. This + // ensures that each block is contained entirely within a partition and prevents + // concurrent access to a block. + constexpr int kLogBlocksKeptTogether = 7; + const int log_num_prtns_max = + std::max(0, build_target_->log_num_blocks() - kLogBlocksKeptTogether); + const int log_num_prtns_mod = std::min(log_num_prtns_, log_num_prtns_max); + int num_prtns = 1 << log_num_prtns_mod; ThreadLocalState& local_state = thread_local_states_[thread_id]; local_state.partition_ranges.resize(num_prtns + 1); local_state.partitioned_hashes_64.resize(num_rows); @@ -376,7 +382,6 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num PartitionSort::Eval( num_rows, num_prtns, partition_ranges, [hashes, num_prtns](int64_t row_id) { - constexpr int kLogBlocksKeptTogether = 7; constexpr int kPrtnIdBitOffset = BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/compute/exec/bloom_filter.h index dcac19d570b..06920c6c14f 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/compute/exec/bloom_filter.h @@ -33,12 +33,6 @@ namespace arrow { namespace compute { -#if defined(__clang__) || defined(__GNUC__) -#define NO_TSAN __attribute__((no_sanitize_thread)) -#else -#define NO_TSAN -#endif - // A set of pre-generated bit masks from a 64-bit word. // // It is used to map selected bits of hash to a bit mask that will be used in @@ -117,7 +111,6 @@ class ARROW_EXPORT BlockedBloomFilter { public: BlockedBloomFilter() : log_num_blocks_(0), num_blocks_(0), blocks_(NULLPTR) {} - NO_TSAN inline bool Find(uint64_t hash) const { uint64_t m = mask(hash); uint64_t b = blocks_[block_id(hash)]; @@ -168,7 +161,6 @@ class ARROW_EXPORT BlockedBloomFilter { private: Status CreateEmpty(int64_t num_rows_to_insert, MemoryPool* pool); - NO_TSAN inline void Insert(uint64_t hash) { uint64_t m = mask(hash); uint64_t& b = blocks_[block_id(hash)]; diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc index 3050fa1b5b7..40069618023 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc @@ -102,7 +102,7 @@ int64_t BlockedBloomFilter::Find_avx2(int64_t num_rows, const uint64_t* hashes, } template -NO_TSAN int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { +int64_t BlockedBloomFilter::InsertImp_avx2(int64_t num_rows, const T* hashes) { constexpr int unroll = 4; for (int64_t i = 0; i < num_rows / unroll; ++i) { diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index 02b0f18ac16..087845be41f 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -72,7 +72,6 @@ Status BuildBloomFilter_Parallel( std::condition_variable cv; std::mutex mutex; - bool done = false; auto group = scheduler->RegisterTaskGroup( [&](size_t thread_index, int64_t task_id) -> Status { int batch_size = static_cast(std::min(num_rows - task_id * kBatchSizeMax, @@ -91,9 +90,8 @@ Status BuildBloomFilter_Parallel( [&](size_t thread_index) -> Status { { std::unique_lock lk(mutex); - done = true; + cv.notify_all(); } - cv.notify_all(); return Status::OK(); }); scheduler->RegisterEnd(); @@ -110,7 +108,7 @@ Status BuildBloomFilter_Parallel( { std::unique_lock lk(mutex); RETURN_NOT_OK(scheduler->StartTaskGroup(0, group, num_batches)); - cv.wait(lk, [&]() { return done; }); + cv.wait(lk); } return Status::OK(); } diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index f6ac70ad45a..ae70cfcd46f 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -29,6 +29,38 @@ using bit_util::CountTrailingZeros; namespace util { +inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes) { + // This will not be correct on big-endian architectures. +#if !ARROW_LITTLE_ENDIAN + ARROW_DCHECK(false); +#endif + ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); + if (num_bytes == 8) { + return util::SafeLoad(reinterpret_cast(bytes)); + } else { + uint64_t word = 0; + for (int i = 0; i < num_bytes; ++i) { + word |= static_cast(bytes[i]) << (8 * i); + } + return word; + } +} + +inline void bit_util::SafeStoreUpTo8Bytes(uint8_t* bytes, int num_bytes, uint64_t value) { + // This will not be correct on big-endian architectures. +#if !ARROW_LITTLE_ENDIAN + ARROW_DCHECK(false); +#endif + ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); + if (num_bytes == 8) { + util::SafeStore(reinterpret_cast(bytes), value); + } else { + for (int i = 0; i < num_bytes; ++i) { + bytes[i] = static_cast(value >> (8 * i)); + } + } +} + inline void bit_util::bits_to_indexes_helper(uint64_t word, uint16_t base_index, int* num_indexes, uint16_t* indexes) { int n = *num_indexes; @@ -86,8 +118,8 @@ void bit_util::bits_to_indexes_internal(int64_t hardware_flags, const int num_bi #endif // Optionally process the last partial word with masking out bits outside range if (tail) { - uint64_t word = - util::SafeLoad(&reinterpret_cast(bits)[num_bits / unroll]); + const uint8_t* bits_tail = bits + (num_bits - tail) / 8; + uint64_t word = SafeLoadUpTo8Bytes(bits_tail, (tail + 7) / 8); if (bit_to_search == 0) { word = ~word; } @@ -109,8 +141,7 @@ void bit_util::bits_to_indexes(int bit_to_search, int64_t hardware_flags, int nu *num_indexes = 0; uint16_t base_index = 0; if (bit_offset != 0) { - uint64_t bits_head = - util::SafeLoad(reinterpret_cast(bits)) >> bit_offset; + uint64_t bits_head = bits[0] >> bit_offset; int bits_in_first_byte = std::min(num_bits, 8 - bit_offset); bits_to_indexes(bit_to_search, hardware_flags, bits_in_first_byte, reinterpret_cast(&bits_head), num_indexes, indexes); @@ -143,8 +174,7 @@ void bit_util::bits_filter_indexes(int bit_to_search, int64_t hardware_flags, bit_offset %= 8; if (bit_offset != 0) { int num_indexes_head = 0; - uint64_t bits_head = - util::SafeLoad(reinterpret_cast(bits)) >> bit_offset; + uint64_t bits_head = bits[0] >> bit_offset; int bits_in_first_byte = std::min(num_bits, 8 - bit_offset); bits_filter_indexes(bit_to_search, hardware_flags, bits_in_first_byte, reinterpret_cast(&bits_head), input_indexes, @@ -185,8 +215,7 @@ void bit_util::bits_to_bytes(int64_t hardware_flags, const int num_bits, bits += bit_offset / 8; bit_offset %= 8; if (bit_offset != 0) { - uint64_t bits_head = - util::SafeLoad(reinterpret_cast(bits)) >> bit_offset; + uint64_t bits_head = bits[0] >> bit_offset; int bits_in_first_byte = std::min(num_bits, 8 - bit_offset); bits_to_bytes(hardware_flags, bits_in_first_byte, reinterpret_cast(&bits_head), bytes); @@ -207,7 +236,7 @@ void bit_util::bits_to_bytes(int64_t hardware_flags, const int num_bits, #endif // Processing 8 bits at a time constexpr int unroll = 8; - for (int i = num_processed / unroll; i < (num_bits + unroll - 1) / unroll; ++i) { + for (int i = num_processed / unroll; i < num_bits / unroll; ++i) { uint8_t bits_next = bits[i]; // Clear the lowest bit and then make 8 copies of remaining 7 bits, each 7 bits apart // from the previous. @@ -219,6 +248,19 @@ void bit_util::bits_to_bytes(int64_t hardware_flags, const int num_bits, unpacked *= 255; util::SafeStore(&reinterpret_cast(bytes)[i], unpacked); } + int tail = num_bits % unroll; + if (tail) { + uint8_t bits_next = bits[(num_bits - tail) / unroll]; + // Clear the lowest bit and then make 8 copies of remaining 7 bits, each 7 bits apart + // from the previous. + uint64_t unpacked = static_cast(bits_next & 0xfe) * + ((1ULL << 7) | (1ULL << 14) | (1ULL << 21) | (1ULL << 28) | + (1ULL << 35) | (1ULL << 42) | (1ULL << 49)); + unpacked |= (bits_next & 1); + unpacked &= 0x0101010101010101ULL; + unpacked *= 255; + SafeStoreUpTo8Bytes(bytes + num_bits - tail, tail, unpacked); + } } void bit_util::bytes_to_bits(int64_t hardware_flags, const int num_bits, @@ -250,7 +292,7 @@ void bit_util::bytes_to_bits(int64_t hardware_flags, const int num_bits, #endif // Process 8 bits at a time constexpr int unroll = 8; - for (int i = num_processed / unroll; i < (num_bits + unroll - 1) / unroll; ++i) { + for (int i = num_processed / unroll; i < num_bits / unroll; ++i) { uint64_t bytes_next = util::SafeLoad(&reinterpret_cast(bytes)[i]); bytes_next &= 0x0101010101010101ULL; bytes_next |= (bytes_next >> 7); // Pairs of adjacent output bits in individual bytes @@ -258,6 +300,15 @@ void bit_util::bytes_to_bits(int64_t hardware_flags, const int num_bits, bytes_next |= (bytes_next >> 28); // All 8 output bits in the lowest byte bits[i] = static_cast(bytes_next & 0xff); } + int tail = num_bits % unroll; + if (tail) { + uint64_t bytes_next = SafeLoadUpTo8Bytes(bytes + num_bits - tail, tail); + bytes_next &= 0x0101010101010101ULL; + bytes_next |= (bytes_next >> 7); // Pairs of adjacent output bits in individual bytes + bytes_next |= (bytes_next >> 14); // 4 adjacent output bits in individual bytes + bytes_next |= (bytes_next >> 28); // All 8 output bits in the lowest byte + bits[num_bits / 8] = static_cast(bytes_next & 0xff); + } } bool bit_util::are_all_bytes_zero(int64_t hardware_flags, const uint8_t* bytes, diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 9a70e585e5e..839a8a7d29c 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -193,6 +193,8 @@ class bit_util { uint32_t num_bytes); private: + inline static uint64_t SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes); + inline static void SafeStoreUpTo8Bytes(uint8_t* bytes, int num_bytes, uint64_t value); inline static void bits_to_indexes_helper(uint64_t word, uint16_t base_index, int* num_indexes, uint16_t* indexes); inline static void bits_filter_indexes_helper(uint64_t word, From b28f4d5d3488aae4a0f1ab4f9ebf710e539be134 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 11:18:06 -0700 Subject: [PATCH 33/42] Fix on big endian --- cpp/src/arrow/compute/exec/bloom_filter.cc | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 303b02bc37d..a4731c08414 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -155,7 +155,11 @@ void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; bits |= result << (i & 63); if ((i & 63) == 63) { +#if ARROW_LITTLE_ENDIAN reinterpret_cast(result_bit_vector)[i / 64] = bits; +#else + reinterpret_cast(result_bit_vector)[i / 64] = BYTESWAP(bits); +#endif bits = 0ULL; } } @@ -166,7 +170,11 @@ void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; bits |= result << (i & 63); if ((i & 63) == 63) { +#if ARROW_LITTLE_ENDIAN reinterpret_cast(result_bit_vector)[i / 64] = bits; +#else + reinterpret_cast(result_bit_vector)[i / 64] = BYTESWAP(bits); +#endif bits = 0ULL; } } @@ -371,6 +379,7 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num std::max(0, build_target_->log_num_blocks() - kLogBlocksKeptTogether); const int log_num_prtns_mod = std::min(log_num_prtns_, log_num_prtns_max); int num_prtns = 1 << log_num_prtns_mod; + ThreadLocalState& local_state = thread_local_states_[thread_id]; local_state.partition_ranges.resize(num_prtns + 1); local_state.partitioned_hashes_64.resize(num_rows); From b18a7d3542abee72b7837f5de0aa1b9c0cc89868 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 12:54:21 -0700 Subject: [PATCH 34/42] Run fewer tests with ASAN --- cpp/src/arrow/compute/exec/bloom_filter.cc | 2 +- cpp/src/arrow/compute/exec/hash_join_node.cc | 2 +- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index a4731c08414..6a21f4ff117 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -390,7 +390,7 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num PartitionSort::Eval( num_rows, num_prtns, partition_ranges, - [hashes, num_prtns](int64_t row_id) { + [hashes, num_prtns, kLogBlocksKeptTogether](int64_t row_id) { constexpr int kPrtnIdBitOffset = BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 2f24c8a34a0..5e3b4198ea2 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -608,7 +608,7 @@ class HashJoinNode : public ExecNode { } bool all_comparisons_is = true; - for (JoinKeyCmp cmp : key_cmp_) all_comparisons_is &= cmp == JoinKeyCmp::IS; + for (JoinKeyCmp cmp : key_cmp_) all_comparisons_is &= (cmp == JoinKeyCmp::IS); if ((join_type_ == JoinType::RIGHT_OUTER || join_type_ == JoinType::FULL_OUTER) && all_comparisons_is) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 7e8e2cf6fc7..e52c012ed5e 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1000,6 +1000,8 @@ TEST(HashJoin, Random) { Random64Bit rng(42); #if defined(THREAD_SANITIZER) || defined(ARROW_VALGRIND) const int num_tests = 15; +#elsif defined(ADDRESS_SANITIZER) + const int num_tests = 50; #else const int num_tests = 100; #endif From caa3193a74ec90c4e10d01d5b440ae82d5aacd15 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 15:29:50 -0700 Subject: [PATCH 35/42] Fix weston comments --- cpp/src/arrow/compute/exec/bloom_filter.cc | 4 ++-- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 4 +--- cpp/src/arrow/compute/exec/exec_plan.h | 5 ++++- cpp/src/arrow/compute/exec/hash_join.cc | 7 +++++-- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 2 +- cpp/src/arrow/compute/exec/task_util.h | 4 ++-- cpp/src/arrow/compute/exec/util.h | 2 +- 7 files changed, 16 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index 6a21f4ff117..ecac707a665 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -390,12 +390,12 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num PartitionSort::Eval( num_rows, num_prtns, partition_ranges, - [hashes, num_prtns, kLogBlocksKeptTogether](int64_t row_id) { + [=](int64_t row_id) { constexpr int kPrtnIdBitOffset = BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); }, - [hashes, partitioned_hashes](int64_t row_id, int output_pos) { + [=](int64_t row_id, int output_pos) { partitioned_hashes[output_pos] = hashes[row_id]; }); diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index 087845be41f..d0d6e2642ed 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -88,11 +88,9 @@ Status BuildBloomFilter_Parallel( return Status::OK(); }, [&](size_t thread_index) -> Status { - { std::unique_lock lk(mutex); cv.notify_all(); - } - return Status::OK(); + return Status::OK(); }); scheduler->RegisterEnd(); auto tp = arrow::internal::GetCpuThreadPool(); diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 508a48209e5..dcf271bd360 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -212,11 +212,14 @@ class ARROW_EXPORT ExecNode { // A node with multiple outputs will also need to ensure it is applying backpressure if // any of its outputs is asking to pause - /// \brief Steps performed immediately before StartProducing is called + /// \brief Perform any needed initialization /// /// This hook performs any actions in between creation of ExecPlan and the call to /// StartProducing. An example could be Bloom filter pushdown. The order of ExecNodes /// that executes this method is undefined, but the calls are made synchronously. + /// + /// At this point a node can rely on all inputs & outputs (and the input schemas) + /// being well defined. virtual Status PrepareToProduce() { return Status::OK(); } /// \brief Start producing diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index c49bf16257a..15a006c81d5 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -162,6 +162,8 @@ class HashJoinBasicImpl : public HashJoinImpl { scheduler_->Abort(std::move(pos_abort_callback)); } + // Called by a downstream node after they have constructed a bloom filter + // that this node can use to filter inputs. Status PushBloomFilter(size_t thread_index, std::unique_ptr filter, std::vector column_map) override { bool proceed; @@ -662,7 +664,7 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector bv(bit_vector_bytes); RETURN_NOT_OK(InitLocalStateIfNeeded(thread_index)); - // Start with full selection for the current minibatch + // Start with full selection for the current batch memset(selected.data(), 0xff, bit_vector_bytes); for (size_t ifilter = 0; ifilter < num_expected_bloom_filters_; ifilter++) { std::vector keys(bloom_filter_column_maps_[ifilter].size()); @@ -790,6 +792,8 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::Cancelled("Hash join cancelled"); } + right_batches_.clear(); + bool proceed; { std::lock_guard lock(left_batches_mutex_); @@ -801,7 +805,6 @@ class HashJoinBasicImpl : public HashJoinImpl { } if (proceed) RETURN_NOT_OK(ProbeQueuedBatches(thread_index)); - right_batches_.clear(); return Status::OK(); } diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index e52c012ed5e..8a76a401317 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1000,7 +1000,7 @@ TEST(HashJoin, Random) { Random64Bit rng(42); #if defined(THREAD_SANITIZER) || defined(ARROW_VALGRIND) const int num_tests = 15; -#elsif defined(ADDRESS_SANITIZER) +#elif defined(ADDRESS_SANITIZER) const int num_tests = 50; #else const int num_tests = 100; diff --git a/cpp/src/arrow/compute/exec/task_util.h b/cpp/src/arrow/compute/exec/task_util.h index c219d4a8f63..44540d255df 100644 --- a/cpp/src/arrow/compute/exec/task_util.h +++ b/cpp/src/arrow/compute/exec/task_util.h @@ -32,7 +32,7 @@ namespace compute { // whenever it is modified by a concurrent thread on a different CPU core. // template -class ARROW_EXPORT AtomicWithPadding { +class AtomicWithPadding { private: static constexpr int kCacheLineSize = 64; uint8_t padding_before[kCacheLineSize]; @@ -53,7 +53,7 @@ class ARROW_EXPORT AtomicWithPadding { // // Also allows for executing next pending tasks immediately using a caller thread. // -class ARROW_EXPORT TaskScheduler { +class TaskScheduler { public: using TaskImpl = std::function; using TaskGroupContinuationImpl = std::function; diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 839a8a7d29c..980d82868ae 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -282,7 +282,7 @@ class ARROW_EXPORT AtomicCounter { std::atomic complete_{false}; }; -class ARROW_EXPORT ThreadIndexer { +class ThreadIndexer { public: size_t operator()(); From 16e7d76cbe581926121ecdf97f5b06905f1a5d73 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 15:40:42 -0700 Subject: [PATCH 36/42] clang-format --- cpp/src/arrow/compute/exec/bloom_filter_test.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/compute/exec/bloom_filter_test.cc index d0d6e2642ed..5e1dd0218cf 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter_test.cc @@ -88,9 +88,9 @@ Status BuildBloomFilter_Parallel( return Status::OK(); }, [&](size_t thread_index) -> Status { - std::unique_lock lk(mutex); - cv.notify_all(); - return Status::OK(); + std::unique_lock lk(mutex); + cv.notify_all(); + return Status::OK(); }); scheduler->RegisterEnd(); auto tp = arrow::internal::GetCpuThreadPool(); From 68c3ba6bb6b18f0d82544334e3cb0d5dfdfebcc2 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 16:38:52 -0700 Subject: [PATCH 37/42] Make windows happy again --- cpp/src/arrow/compute/exec/bloom_filter.cc | 5 +++-- cpp/src/arrow/compute/exec/task_util.h | 2 +- cpp/src/arrow/compute/exec/util.h | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index ecac707a665..ba3b59bf264 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -375,6 +375,9 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num // ensures that each block is contained entirely within a partition and prevents // concurrent access to a block. constexpr int kLogBlocksKeptTogether = 7; + constexpr int kPrtnIdBitOffset = + BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; + const int log_num_prtns_max = std::max(0, build_target_->log_num_blocks() - kLogBlocksKeptTogether); const int log_num_prtns_mod = std::min(log_num_prtns_, log_num_prtns_max); @@ -391,8 +394,6 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num PartitionSort::Eval( num_rows, num_prtns, partition_ranges, [=](int64_t row_id) { - constexpr int kPrtnIdBitOffset = - BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; return (hashes[row_id] >> (kPrtnIdBitOffset)) & (num_prtns - 1); }, [=](int64_t row_id, int output_pos) { diff --git a/cpp/src/arrow/compute/exec/task_util.h b/cpp/src/arrow/compute/exec/task_util.h index 44540d255df..5dc66aee167 100644 --- a/cpp/src/arrow/compute/exec/task_util.h +++ b/cpp/src/arrow/compute/exec/task_util.h @@ -53,7 +53,7 @@ class AtomicWithPadding { // // Also allows for executing next pending tasks immediately using a caller thread. // -class TaskScheduler { +class ARROW_EXPORT TaskScheduler { public: using TaskImpl = std::function; using TaskGroupContinuationImpl = std::function; diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 980d82868ae..839a8a7d29c 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -282,7 +282,7 @@ class ARROW_EXPORT AtomicCounter { std::atomic complete_{false}; }; -class ThreadIndexer { +class ARROW_EXPORT ThreadIndexer { public: size_t operator()(); From bdbcb01c1c890575e3b1f2f8ba404ecaf4d1daed Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 9 May 2022 16:39:44 -0700 Subject: [PATCH 38/42] clang-format --- cpp/src/arrow/compute/exec/bloom_filter.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index ba3b59bf264..ea9a7db7dce 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -376,7 +376,7 @@ void BloomFilterBuilder_Parallel::PushNextBatchImp(size_t thread_id, int64_t num // concurrent access to a block. constexpr int kLogBlocksKeptTogether = 7; constexpr int kPrtnIdBitOffset = - BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; + BloomFilterMasks::kLogNumMasks + 6 + kLogBlocksKeptTogether; const int log_num_prtns_max = std::max(0, build_target_->log_num_blocks() - kLogBlocksKeptTogether); From 36a229bff8f9d153a1c210cfff6351dc50da30bf Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 16 May 2022 13:25:30 -0700 Subject: [PATCH 39/42] Try fixing on big endian again --- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 2 +- cpp/src/arrow/compute/exec/util.cc | 11 +++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 8a76a401317..3bc2c6870b6 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1001,7 +1001,7 @@ TEST(HashJoin, Random) { #if defined(THREAD_SANITIZER) || defined(ARROW_VALGRIND) const int num_tests = 15; #elif defined(ADDRESS_SANITIZER) - const int num_tests = 50; + const int num_tests = 25; #else const int num_tests = 100; #endif diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index ae70cfcd46f..813c56f9a85 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -30,10 +30,6 @@ using bit_util::CountTrailingZeros; namespace util { inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes) { - // This will not be correct on big-endian architectures. -#if !ARROW_LITTLE_ENDIAN - ARROW_DCHECK(false); -#endif ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); if (num_bytes == 8) { return util::SafeLoad(reinterpret_cast(bytes)); @@ -42,14 +38,17 @@ inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes for (int i = 0; i < num_bytes; ++i) { word |= static_cast(bytes[i]) << (8 * i); } +#if ARROW_LITTLE_ENDIAN return word; +#else + return BYTESWAP(word); +#endif } } inline void bit_util::SafeStoreUpTo8Bytes(uint8_t* bytes, int num_bytes, uint64_t value) { - // This will not be correct on big-endian architectures. #if !ARROW_LITTLE_ENDIAN - ARROW_DCHECK(false); + value = BYTESWAP(value); #endif ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); if (num_bytes == 8) { From 6e5bf4c2e4cc01a56e72090e52614af68209632e Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 16 May 2022 13:47:32 -0700 Subject: [PATCH 40/42] clang-format --- cpp/src/arrow/compute/exec/util.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index 813c56f9a85..c8d63197406 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -48,7 +48,7 @@ inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes inline void bit_util::SafeStoreUpTo8Bytes(uint8_t* bytes, int num_bytes, uint64_t value) { #if !ARROW_LITTLE_ENDIAN - value = BYTESWAP(value); + value = BYTESWAP(value); #endif ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); if (num_bytes == 8) { From a3b2f938a22afeef40856be9833f5bb70c26ec10 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 16 May 2022 18:53:16 -0700 Subject: [PATCH 41/42] Disable bloom filter on big endian --- cpp/src/arrow/compute/exec/bloom_filter.cc | 8 -------- cpp/src/arrow/compute/exec/hash_join_node.cc | 7 +++++++ cpp/src/arrow/compute/exec/util.cc | 11 ++++++----- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/compute/exec/bloom_filter.cc index ea9a7db7dce..7b348ff687c 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/compute/exec/bloom_filter.cc @@ -155,11 +155,7 @@ void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; bits |= result << (i & 63); if ((i & 63) == 63) { -#if ARROW_LITTLE_ENDIAN reinterpret_cast(result_bit_vector)[i / 64] = bits; -#else - reinterpret_cast(result_bit_vector)[i / 64] = BYTESWAP(bits); -#endif bits = 0ULL; } } @@ -170,11 +166,7 @@ void BlockedBloomFilter::FindImp(int64_t num_rows, const T* hashes, uint64_t result = Find(hashes[i]) ? 1ULL : 0ULL; bits |= result << (i & 63); if ((i & 63) == 63) { -#if ARROW_LITTLE_ENDIAN reinterpret_cast(result_bit_vector)[i / 64] = bits; -#else - reinterpret_cast(result_bit_vector)[i / 64] = BYTESWAP(bits); -#endif bits = 0ULL; } } diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 5e3b4198ea2..d8d729dd1ae 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -583,6 +583,12 @@ class HashJoinNode : public ExecNode { // If we should disable Bloom filter, returns nullptr and an empty vector, and sets // the disable_bloom_filter_ flag. std::pair> GetPushdownTarget() { +#if !ARROW_LITTLE_ENDIAN + // TODO (ARROW-16591): Debug bloom_filter.cc to enable on Big endian. It probably just + // needs a few byte swaps in the proper spots. + disable_bloom_filter_ = true; + return {nullptr, {}}; +#else // A build-side Bloom filter tells us if a row is definitely not in the build side. // This allows us to early-eliminate rows or early-accept rows depending on the type // of join. Left Outer Join and Full Outer Join output all rows, so a build-side Bloom @@ -670,6 +676,7 @@ class HashJoinNode : public ExecNode { pushdown_target = candidate_as_join; } return std::make_pair(pushdown_target->impl_.get(), std::move(bloom_to_target)); +#endif // ARROW_LITTLE_ENDIAN } Status PrepareToProduce() override { diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index c8d63197406..ae70cfcd46f 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -30,6 +30,10 @@ using bit_util::CountTrailingZeros; namespace util { inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes) { + // This will not be correct on big-endian architectures. +#if !ARROW_LITTLE_ENDIAN + ARROW_DCHECK(false); +#endif ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); if (num_bytes == 8) { return util::SafeLoad(reinterpret_cast(bytes)); @@ -38,17 +42,14 @@ inline uint64_t bit_util::SafeLoadUpTo8Bytes(const uint8_t* bytes, int num_bytes for (int i = 0; i < num_bytes; ++i) { word |= static_cast(bytes[i]) << (8 * i); } -#if ARROW_LITTLE_ENDIAN return word; -#else - return BYTESWAP(word); -#endif } } inline void bit_util::SafeStoreUpTo8Bytes(uint8_t* bytes, int num_bytes, uint64_t value) { + // This will not be correct on big-endian architectures. #if !ARROW_LITTLE_ENDIAN - value = BYTESWAP(value); + ARROW_DCHECK(false); #endif ARROW_DCHECK(num_bytes >= 0 && num_bytes <= 8); if (num_bytes == 8) { From 1a4ae698188d2b546b3a45637c5e3b966b296362 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 17 May 2022 15:25:48 -0700 Subject: [PATCH 42/42] Hopefully fix timeout --- cpp/src/arrow/compute/exec/hash_join_node_test.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 3bc2c6870b6..f8da71c7b54 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -1134,7 +1134,7 @@ TEST(HashJoin, Random) { // Turn the last key comparison into a residual filter expression Expression filter = literal(true); - if (key_cmp.size() > 1 && rng.from_range(0, 1) == 0) { + if (key_cmp.size() > 1 && rng.from_range(0, 4) == 0) { for (size_t i = 0; i < key_cmp.size(); i++) { FieldRef left = key_fields[0][i]; FieldRef right = key_fields[1][i]; @@ -1173,6 +1173,7 @@ TEST(HashJoin, Random) { } std::shared_ptr output_schema = std::make_shared(std::move(output_schema_fields)); + ASSERT_OK_AND_ASSIGN( auto batches, HashJoinWithExecPlan( rng, parallel, join_options, output_schema,