save-buffer commented on code in PR #13669: URL: https://github.com/apache/arrow/pull/13669#discussion_r972371570
########## 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()); Review Comment: Currently nothing, we need memory broker for that. I figured it would be ok to leave it (this seems like it would often just degenerate to spilling all partitions) for now. To fix it, we can either have some sort of generation counter, where we try to CAS `generation counter + 1` and if it fails recheck memory and see if we have to spill, or implement memory broker where you request some memory every time you need to allocate (or something like that). I was going to implement memory broker later. -- 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: github-unsubscr...@arrow.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org