save-buffer commented on code in PR #13669:
URL: https://github.com/apache/arrow/pull/13669#discussion_r972335507


##########
cpp/src/arrow/compute/exec/spilling_join.cc:
##########
@@ -0,0 +1,351 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/compute/exec/spilling_join.h"
+#include "arrow/util/atomic_util.h"
+#include "arrow/util/make_unique.h"
+
+namespace arrow
+{
+    namespace compute
+    {
+        void PartitionedBloomFilter::Find(
+                int64_t hardware_flags,
+                int64_t num_rows,
+                const uint64_t *hashes,
+                uint8_t *bv)
+        {
+            if(in_memory)
+                return in_memory->Find(hardware_flags, num_rows, hashes, bv);
+
+            for(int64_t i = 0; i < num_rows; i++)
+            {
+                uint64_t hash = hashes[i];
+                size_t partition = hash & 
(SpillingAccumulationQueue::kNumPartitions - 1);
+                bool found = partitions[partition] ? 
partitions[partition]->Find(hash) : true;
+                bit_util::SetBitTo(bv, i, found);
+            }
+        }
+
+        Status SpillingHashJoin::Init(
+            QueryContext *ctx,
+            JoinType join_type,
+            size_t num_threads,
+            SchemaProjectionMaps<HashJoinProjection> *proj_map_left,
+            SchemaProjectionMaps<HashJoinProjection> *proj_map_right,
+            std::vector<JoinKeyCmp> *key_cmp,
+            Expression *filter,
+            PartitionedBloomFilter *bloom_filter,
+            CallbackRecord callback_record,
+            bool is_swiss)
+        {
+            ctx_ = ctx;
+            num_threads_ = num_threads;
+            callbacks_ = std::move(callback_record);
+            bloom_filter_ = bloom_filter;
+            is_swiss_ = is_swiss;
+
+            HashJoinImpl::CallbackRecord join_callbacks;
+            join_callbacks.register_task_group = 
callbacks_.register_task_group;
+            join_callbacks.start_task_group = callbacks_.start_task_group;
+            join_callbacks.output_batch = callbacks_.output_batch;
+            join_callbacks.finished = [this](int64_t num_total_batches)
+            {
+                return this->OnCollocatedJoinFinished(num_total_batches);
+            };
+
+            builder_ = BloomFilterBuilder::Make(
+                num_threads_ == 1
+                ? BloomFilterBuildStrategy::SINGLE_THREADED
+                : BloomFilterBuildStrategy::PARALLEL);
+            RETURN_NOT_OK(build_accumulator_.Init(ctx));
+            RETURN_NOT_OK(probe_accumulator_.Init(ctx));
+
+            for(size_t i = 0; i < SpillingAccumulationQueue::kNumPartitions; 
i++)
+            {
+                ARROW_ASSIGN_OR_RAISE(impls_[i], is_swiss_ ? 
HashJoinImpl::MakeSwiss() : HashJoinImpl::MakeBasic());
+                RETURN_NOT_OK(impls_[i]->Init(ctx_,
+                                              join_type,
+                                              num_threads,
+                                              proj_map_left,
+                                              proj_map_right,
+                                              key_cmp,
+                                              filter,
+                                              join_callbacks));
+
+                task_group_bloom_[i] = callbacks_.register_task_group(
+                    [this](size_t thread_index, int64_t task_id)
+                    {
+                        return PushBloomFilterBatch(thread_index, task_id);
+                    },
+                    [this](size_t thread_index)
+                    {
+                        partition_idx_++;
+                        return BuildNextBloomFilter(thread_index);
+                    });
+
+                task_group_probe_[i] = callbacks_.register_task_group(
+                    [this](size_t thread_index, int64_t task_id)
+                    {
+                        return ProbeAndFilterBatch(thread_index, 
std::move(probe_side_partition_[task_id]));
+                    },
+                    [this](size_t thread_index)
+                    {
+                        return OnProbingFinished(thread_index);
+                    });
+            }
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::CheckSpilling(size_t thread_index, ExecBatch 
&batch)
+        {
+            size_t size_of_batch = 
static_cast<size_t>(batch.TotalBufferSize());
+            size_t max_batch_size = arrow::util::AtomicMax(max_batch_size_, 
size_of_batch);
+
+            // Spilling algorithm proven to not use more than
+            // (SpillThreshold + NumThreads * BatchSize) memory.
+            // Thus we want to spill when (SpillThreshold + NumThreads * 
BatchSize) = k * MaxMemory
+            // with some fuzz factor k (which is 0.8 here because that's what 
I decided). 
+            // Thus SpillThreshold = k * MaxMemory - NumThreads * BatchSize. 
+            constexpr float kFuzzFactor = 0.8f;
+            size_t max_memory = static_cast<size_t>(kFuzzFactor * 
ctx_->options().max_memory_bytes);
+            size_t spill_threshold =
+                static_cast<size_t>(
+                    std::max(
+                        static_cast<ssize_t>(kFuzzFactor * max_memory - 
num_threads_ * max_batch_size),
+                        static_cast<ssize_t>(0)));
+            size_t bytes_allocated = 
static_cast<size_t>(ctx_->memory_pool()->bytes_allocated());
+            size_t bytes_inflight = ctx_->GetCurrentTempFileIO();
+
+            size_t backpressure_threshold = spill_threshold / 2;
+            if(bytes_allocated > backpressure_threshold)
+                
callbacks_.pause_probe_side(backpressure_counter_.fetch_add(1));
+            if((bytes_allocated - bytes_inflight) > spill_threshold)
+                RETURN_NOT_OK(AdvanceSpillCursor(thread_index));
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::AdvanceSpillCursor(size_t thread_index)
+        {
+            bool expected = false;
+            if(!spilling_.load() && 
spilling_.compare_exchange_strong(expected, true))
+                return callbacks_.start_spilling(thread_index);
+
+            ARROW_ASSIGN_OR_RAISE(bool probe_advanced, 
probe_accumulator_.AdvanceSpillCursor());
+            if(probe_advanced) return Status::OK();
+
+            ARROW_ASSIGN_OR_RAISE(bool build_advanced, 
build_accumulator_.AdvanceSpillCursor());
+            if(build_advanced) return Status::OK();
+
+            ARROW_ASSIGN_OR_RAISE(bool probe_hash_advanced, 
probe_accumulator_.AdvanceHashCursor());
+            if(probe_hash_advanced) return Status::OK();
+
+            ARROW_ASSIGN_OR_RAISE(bool build_hash_advanced, 
build_accumulator_.AdvanceHashCursor());
+            if(build_hash_advanced) return Status::OK();
+
+            // Pray we don't run out of memory
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::OnBuildSideBatch(size_t thread_index, 
ExecBatch batch)
+        {
+            return build_accumulator_.InsertBatch(
+                thread_index,
+                std::move(batch));
+        }
+
+        Status SpillingHashJoin::OnBuildSideFinished(size_t thread_index)
+        {
+            callbacks_.resume_probe_side(backpressure_counter_.fetch_add(1));
+            return BuildBloomFilter(thread_index);
+        }
+
+        Status SpillingHashJoin::BuildBloomFilter(size_t thread_index)
+        {
+            // Disable Bloom filter if bloom_filter_ = nullptr by advancing to 
past
+            // the final Bloom filter
+            partition_idx_ = (bloom_filter_ == nullptr)
+                ? partition_idx_ = SpillingAccumulationQueue::kNumPartitions
+                : build_accumulator_.hash_cursor();
+            return BuildNextBloomFilter(thread_index);
+        }
+
+        Status SpillingHashJoin::PushBloomFilterBatch(size_t thread_index, 
int64_t batch_id)
+        {
+            const uint64_t *hashes = build_accumulator_.GetHashes(
+                partition_idx_,
+                static_cast<size_t>(batch_id));
+            size_t num_rows = build_accumulator_.row_count(
+                partition_idx_,
+                static_cast<size_t>(batch_id));
+            return builder_->PushNextBatch(
+                thread_index,
+                static_cast<int64_t>(num_rows),
+                hashes);
+        }
+
+        Status SpillingHashJoin::BuildNextBloomFilter(size_t thread_index)
+        {
+            if(partition_idx_ >= SpillingAccumulationQueue::kNumPartitions)
+                return OnBloomFilterFinished(thread_index);
+
+            size_t num_rows = build_accumulator_.row_count(partition_idx_);
+            size_t num_batches = 
build_accumulator_.batch_count(partition_idx_);
+
+            // partition_idx_ is incremented in the callback for the taskgroup
+            bloom_filter_->partitions[partition_idx_] =
+                arrow::internal::make_unique<BlockedBloomFilter>();
+
+            RETURN_NOT_OK(builder_->Begin(
+                              num_threads_,
+                              ctx_->cpu_info()->hardware_flags(),
+                              ctx_->memory_pool(),
+                              num_rows,
+                              num_batches,
+                              
bloom_filter_->partitions[partition_idx_].get()));
+                
+            return callbacks_.start_task_group(
+                task_group_bloom_[partition_idx_],
+                build_accumulator_.batch_count(partition_idx_));
+        }
+        
+        Status SpillingHashJoin::OnBloomFilterFinished(size_t thread_index)
+        {
+            RETURN_NOT_OK(callbacks_.bloom_filter_finished(thread_index));
+            bool expected = false;
+            if(!bloom_or_probe_finished_.compare_exchange_strong(expected, 
true))
+                return StartCollocatedJoins(thread_index);
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::OnBloomFiltersReceived(size_t thread_index)
+        {
+            bloom_ready_.store(true, std::memory_order_release);
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::OnProbeSideBatch(size_t thread_index, 
ExecBatch batch)
+        {
+            if(bloom_ready_.load())
+            {
+                RETURN_NOT_OK(callbacks_.apply_bloom_filter(
+                    thread_index,
+                    &batch));
+            }
+            return probe_accumulator_.InsertBatch(
+                              thread_index,
+                              std::move(batch));
+        }
+
+        Status SpillingHashJoin::OnProbeSideFinished(size_t thread_index)
+        {
+            bool expected = false;
+            if(!bloom_or_probe_finished_.compare_exchange_strong(expected, 
true))
+                return StartCollocatedJoins(thread_index);
+            return Status::OK();
+        }
+
+        Status SpillingHashJoin::StartCollocatedJoins(size_t thread_index)
+        {
+            partition_idx_ = SpillingAccumulationQueue::kNumPartitions;
+            return BeginNextCollocatedJoin(thread_index);
+        }
+
+        Status SpillingHashJoin::BeginNextCollocatedJoin(size_t thread_index)
+        {
+            partition_idx_ -= 1;
+            probe_or_hash_table_ready_.store(false);
+            return build_accumulator_
+                .GetPartition(
+                    thread_index,
+                    partition_idx_,
+                    [this](size_t thread_index, AccumulationQueue queue)
+                    {
+                        return BuildHashTable(thread_index, std::move(queue));
+                    });
+        }
+
+        Status SpillingHashJoin::BuildHashTable(size_t thread_index, 
AccumulationQueue queue)
+        {
+            RETURN_NOT_OK(
+                impls_[partition_idx_]->BuildHashTable(
+                    thread_index,
+                    std::move(queue),
+                    [this](size_t thread_index)
+                    {
+                        return OnHashTableFinished(thread_index);
+                    }));
+            RETURN_NOT_OK(
+                probe_accumulator_
+                .GetPartition(

Review Comment:
   I changed it to be streaming now. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to