From 8f070307f277dfb81b10986a1c783098f3b9779e Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 9 Jun 2022 13:37:01 -0700 Subject: [PATCH] Add spilling for hash join --- cpp/src/arrow/CMakeLists.txt | 3 + cpp/src/arrow/buffer.h | 7 + .../arrow/compute/exec/accumulation_queue.cc | 122 ++++- .../arrow/compute/exec/accumulation_queue.h | 41 +- cpp/src/arrow/compute/exec/aggregate_node.cc | 34 +- cpp/src/arrow/compute/exec/asof_join_node.cc | 2 +- cpp/src/arrow/compute/exec/exec_plan.cc | 104 +---- cpp/src/arrow/compute/exec/exec_plan.h | 82 +--- cpp/src/arrow/compute/exec/filter_node.cc | 4 +- cpp/src/arrow/compute/exec/hash_join.cc | 69 ++- cpp/src/arrow/compute/exec/hash_join.h | 36 +- .../arrow/compute/exec/hash_join_benchmark.cc | 29 +- cpp/src/arrow/compute/exec/hash_join_node.cc | 399 +++++++++++----- cpp/src/arrow/compute/exec/partition_util.cc | 14 + cpp/src/arrow/compute/exec/partition_util.h | 21 +- cpp/src/arrow/compute/exec/project_node.cc | 4 +- cpp/src/arrow/compute/exec/query_context.cc | 118 +++++ cpp/src/arrow/compute/exec/query_context.h | 178 ++++++++ cpp/src/arrow/compute/exec/schema_util.h | 116 +++-- cpp/src/arrow/compute/exec/sink_node.cc | 8 +- cpp/src/arrow/compute/exec/source_node.cc | 8 +- cpp/src/arrow/compute/exec/spilling_join.cc | 210 +++++++++ cpp/src/arrow/compute/exec/spilling_join.h | 109 +++++ cpp/src/arrow/compute/exec/spilling_util.cc | 432 ++++++++++++++++++ cpp/src/arrow/compute/exec/spilling_util.h | 67 +++ cpp/src/arrow/compute/exec/swiss_join.cc | 66 ++- cpp/src/arrow/compute/exec/tpch_node.cc | 4 +- cpp/src/arrow/compute/light_array.cc | 20 +- cpp/src/arrow/compute/light_array.h | 5 +- cpp/src/arrow/dataset/scanner.cc | 6 +- cpp/src/arrow/datum.cc | 1 + cpp/src/arrow/io/file_test.cc | 6 +- cpp/src/arrow/memory_pool.cc | 169 ++++--- cpp/src/arrow/memory_pool.h | 32 +- cpp/src/arrow/memory_pool_internal.h | 8 +- cpp/src/arrow/memory_pool_jemalloc.cc | 20 +- cpp/src/arrow/stl_allocator.h | 10 +- cpp/src/arrow/util/atomic_util.h | 172 +++++++ cpp/src/arrow/util/io_util.cc | 34 ++ cpp/src/arrow/util/io_util.h | 6 + 40 files changed, 2187 insertions(+), 589 deletions(-) create mode 100644 cpp/src/arrow/compute/exec/query_context.cc create mode 100644 cpp/src/arrow/compute/exec/query_context.h create mode 100644 cpp/src/arrow/compute/exec/spilling_join.cc create mode 100644 cpp/src/arrow/compute/exec/spilling_join.h create mode 100644 cpp/src/arrow/compute/exec/spilling_util.cc create mode 100644 cpp/src/arrow/compute/exec/spilling_util.h create mode 100644 cpp/src/arrow/util/atomic_util.h diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 5070d22fc5524..8e785b913f7db 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -401,8 +401,11 @@ if(ARROW_COMPUTE) compute/exec/partition_util.cc compute/exec/options.cc compute/exec/project_node.cc + compute/exec/query_context.cc compute/exec/sink_node.cc compute/exec/source_node.cc + compute/exec/spilling_util.cc + compute/exec/spilling_join.cc compute/exec/swiss_join.cc compute/exec/task_util.cc compute/exec/tpch_node.cc diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h index 8be10d282b060..51367e6b8b412 100644 --- a/cpp/src/arrow/buffer.h +++ b/cpp/src/arrow/buffer.h @@ -460,6 +460,10 @@ class ARROW_EXPORT ResizableBuffer : public MutableBuffer { ARROW_EXPORT Result> AllocateBuffer(const int64_t size, MemoryPool* pool = NULLPTR); +ARROW_EXPORT +Result> AllocateBuffer(const int64_t size, + int64_t alignment, + MemoryPool* pool = NULLPTR); /// \brief Allocate a resizeable buffer from a memory pool, zero its padding. /// @@ -468,6 +472,9 @@ Result> AllocateBuffer(const int64_t size, ARROW_EXPORT Result> AllocateResizableBuffer( const int64_t size, MemoryPool* pool = NULLPTR); +ARROW_EXPORT +Result> AllocateResizableBuffer( + const int64_t size, const int64_t alignment, MemoryPool* pool = NULLPTR); /// \brief Allocate a bitmap buffer from a memory pool /// no guarantee on values is provided. diff --git a/cpp/src/arrow/compute/exec/accumulation_queue.cc b/cpp/src/arrow/compute/exec/accumulation_queue.cc index 192db52942820..b255b94f83fe6 100644 --- a/cpp/src/arrow/compute/exec/accumulation_queue.cc +++ b/cpp/src/arrow/compute/exec/accumulation_queue.cc @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/util/atomic_util.h" #include "arrow/compute/exec/accumulation_queue.h" +#include "arrow/compute/exec/key_hash.h" #include namespace arrow { -namespace util { -using arrow::compute::ExecBatch; +namespace compute { + AccumulationQueue::AccumulationQueue(AccumulationQueue&& that) { this->batches_ = std::move(that.batches_); this->row_count_ = that.row_count_; @@ -48,11 +50,127 @@ void AccumulationQueue::InsertBatch(ExecBatch batch) { batches_.emplace_back(std::move(batch)); } +void AccumulationQueue::SetBatch(size_t idx, ExecBatch batch) +{ + ARROW_DCHECK(idx < batches_.size()); + arrow::util::AtomicFetchSub(&row_count_, batches_[idx].length, std::memory_order_relaxed); + arrow::util::AtomicFetchAdd(&row_count_, batch.length, std::memory_order_relaxed); + batches_[idx] = std::move(batch); +} + void AccumulationQueue::Clear() { row_count_ = 0; batches_.clear(); } ExecBatch& AccumulationQueue::operator[](size_t i) { return batches_[i]; } + + Status SpillingAccumulationQueue::Init(QueryContext *ctx) + { + ctx_ = ctx; + partition_locks_.Init(ctx_->max_concurrency(), kNumPartitions); + return Status::OK(); + } + + Status SpillingAccumulationQueue::InsertBatch( + size_t thread_index, + ExecBatch batch) + { + Datum hash_datum = std::move(batch.values.back()); + const uint64_t *hashes = reinterpret_cast(hash_datum.array()->buffers[1]->data()); + std::vector row_ids(batch.length); + std::iota(row_ids.begin(), row_ids.end(), 0); + uint16_t part_starts[kNumPartitions + 1]; + PartitionSort::Eval( + util::MiniBatch::kMiniBatchLength, + kNumPartitions, + part_starts, + [&](int64_t i) + { + return hashes[i] & (kNumPartitions - 1); + }, + [&row_ids](int64_t i, int output_pos) + { + row_ids[i] = static_cast(output_pos); + }); + + int unprocessed_partition_ids[kNumPartitions]; + RETURN_NOT_OK(partition_locks_.ForEachPartition( + thread_index, + unprocessed_partition_ids, + [&](int part_id) + { + return part_starts[part_id + 1] == part_starts[part_id]; + }, + [&](int locked_part_id) + { + uint64_t num_total_rows_to_append = + part_starts[locked_part_id + 1] - part_starts[locked_part_id]; + + while(num_total_rows_to_append > 0) + { + int num_rows_to_append = std::min( + static_cast(num_total_rows_to_append), + static_cast(ExecBatchBuilder::num_rows_max() - builders_[locked_part_id].num_rows())); + + RETURN_NOT_OK(builders_[locked_part_id].AppendSelected( + ctx_->memory_pool(), + batch, + num_rows_to_append, + row_ids.data() + part_starts[locked_part_id], + batch.num_values())); + + if(builders_[locked_part_id].is_full()) + { + ExecBatch batch = builders_[locked_part_id].Flush(); + if(locked_part_id < spilling_cursor_) + RETURN_NOT_OK(files_[locked_part_id].SpillBatch(ctx_, std::move(batch))); + else + queues_[locked_part_id].InsertBatch(std::move(batch)); + } + part_starts[locked_part_id] += num_rows_to_append; + num_total_rows_to_append -= num_rows_to_append; + } + return Status::OK(); + })); + return Status::OK(); + } + + Status SpillingAccumulationQueue::GetPartition( + size_t thread_index, + int partition, + std::function on_finished) + { + if(partition >= spilling_cursor_) + return on_finished(thread_index, std::move(queues_[partition])); + + queues_[partition].Resize(files_[partition].num_batches()); + return files_[partition].ReadBackBatches( + ctx_, + [this, partition](size_t idx, ExecBatch batch) + { + queues_[partition].SetBatch(idx, std::move(batch)); + return Status::OK(); + }, + [this, partition, on_finished](size_t thread_index) + { + return on_finished(thread_index, std::move(queues_[partition])); + }); + } + + Result SpillingAccumulationQueue::AdvanceSpillCursor() + { + int to_spill = spilling_cursor_.fetch_add(1); + if(to_spill >= kNumPartitions) + return false; + + auto lock = partition_locks_.AcquirePartitionLock(to_spill); + + size_t num_batches = queues_[to_spill].batch_count(); + for(size_t i = 0; i < num_batches; i++) + RETURN_NOT_OK(files_[to_spill].SpillBatch(ctx_, std::move(queues_[to_spill][i]))); + return true; + } + } // namespace util } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/accumulation_queue.h b/cpp/src/arrow/compute/exec/accumulation_queue.h index 4b23e5ffcac54..9f24f934d3ca0 100644 --- a/cpp/src/arrow/compute/exec/accumulation_queue.h +++ b/cpp/src/arrow/compute/exec/accumulation_queue.h @@ -21,10 +21,13 @@ #include #include "arrow/compute/exec.h" +#include "arrow/compute/light_array.h" +#include "arrow/compute/exec/partition_util.h" +#include "arrow/compute/exec/task_util.h" +#include "arrow/compute/exec/spilling_util.h" namespace arrow { -namespace util { -using arrow::compute::ExecBatch; +namespace compute { /// \brief A container that accumulates batches until they are ready to /// be processed. @@ -42,9 +45,11 @@ class AccumulationQueue { void Concatenate(AccumulationQueue&& that); void InsertBatch(ExecBatch batch); + void SetBatch(size_t idx, ExecBatch batch); int64_t row_count() { return row_count_; } size_t batch_count() { return batches_.size(); } bool empty() const { return batches_.empty(); } + void Resize(size_t size) { batches_.resize(size); } void Clear(); ExecBatch& operator[](size_t i); @@ -53,5 +58,35 @@ class AccumulationQueue { std::vector batches_; }; -} // namespace util +class SpillingAccumulationQueue +{ +public: + static constexpr int kNumPartitions = 64; + Status Init(QueryContext *ctx); + // Assumes that the final column in batch contains 64-bit hashes of the columns. + Status InsertBatch( + size_t thread_index, + ExecBatch batch); + Status GetPartition( + size_t thread_index, + int partition, + std::function on_finished); + Result AdvanceSpillCursor(); + +private: + std::atomic spilling_cursor_{0}; // denotes the first in-memory partition + QueryContext* ctx_; + PartitionLocks partition_locks_; + + AccumulationQueue queues_[kNumPartitions]; + AccumulationQueue hash_queues_[kNumPartitions]; + + ExecBatchBuilder builders_[kNumPartitions]; + ExecBatchBuilder hash_builders_[kNumPartitions]; + + SpillFile files_[kNumPartitions]; + SpillFile hash_files_[kNumPartitions]; +}; + +} // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index cca266ad691d3..7997b5482d11c 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -83,7 +83,7 @@ class ScalarAggregateNode : public ExecNode { auto aggregates = aggregate_options.aggregates; const auto& input_schema = *inputs[0]->output_schema(); - auto exec_ctx = plan->exec_context(); + auto exec_ctx = plan->query_context()->exec_context(); std::vector kernels(aggregates.size()); std::vector>> states(kernels.size()); @@ -113,7 +113,7 @@ class ScalarAggregateNode : public ExecNode { } KernelContext kernel_ctx{exec_ctx}; - states[i].resize(plan->max_concurrency()); + states[i].resize(plan->query_context()->max_concurrency()); RETURN_NOT_OK(Kernel::InitAll(&kernel_ctx, KernelInitArgs{kernels[i], { @@ -150,7 +150,7 @@ class ScalarAggregateNode : public ExecNode { {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, {"function.kind", std::string(kind_name()) + "::Consume"}}); - KernelContext batch_ctx{plan()->exec_context()}; + KernelContext batch_ctx{plan()->query_context()->exec_context()}; batch_ctx.SetState(states_[i][thread_index].get()); ExecSpan single_column_batch{{batch.values[target_field_ids_[i]]}, batch.length}; @@ -168,7 +168,7 @@ class ScalarAggregateNode : public ExecNode { {"batch.length", batch.length}}); DCHECK_EQ(input, inputs_[0]); - auto thread_index = plan_->GetThreadIndex(); + auto thread_index = plan_->query_context()->GetThreadIndex(); if (ErrorIfNotOk(DoConsume(ExecSpan(batch), thread_index))) return; @@ -245,7 +245,7 @@ class ScalarAggregateNode : public ExecNode { {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, {"function.kind", std::string(kind_name()) + "::Finalize"}}); - KernelContext ctx{plan()->exec_context()}; + KernelContext ctx{plan()->query_context()->exec_context()}; ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll( kernels_[i], &ctx, std::move(states_[i]))); RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); @@ -267,20 +267,19 @@ class ScalarAggregateNode : public ExecNode { class GroupByNode : public ExecNode { public: - GroupByNode(ExecNode* input, std::shared_ptr output_schema, ExecContext* ctx, + GroupByNode(ExecNode* input, std::shared_ptr output_schema, std::vector key_field_ids, std::vector agg_src_field_ids, std::vector aggs, std::vector agg_kernels) : ExecNode(input->plan(), {input}, {"groupby"}, std::move(output_schema), /*num_outputs=*/1), - ctx_(ctx), key_field_ids_(std::move(key_field_ids)), agg_src_field_ids_(std::move(agg_src_field_ids)), aggs_(std::move(aggs)), agg_kernels_(std::move(agg_kernels)) {} Status Init() override { - output_task_group_id_ = plan_->RegisterTaskGroup( + output_task_group_id_ = plan_->query_context()->RegisterTaskGroup( [this](size_t, int64_t task_id) { OutputNthBatch(task_id); return Status::OK(); @@ -326,7 +325,7 @@ class GroupByNode : public ExecNode { agg_src_types[i] = input_schema->field(agg_src_field_id)->type().get(); } - auto ctx = input->plan()->exec_context(); + auto ctx = plan->query_context()->exec_context(); // Construct aggregates ARROW_ASSIGN_OR_RAISE(auto agg_kernels, @@ -354,7 +353,7 @@ class GroupByNode : public ExecNode { } return input->plan()->EmplaceNode( - input, schema(std::move(output_fields)), ctx, std::move(key_field_ids), + input, schema(std::move(output_fields)), std::move(key_field_ids), std::move(agg_src_field_ids), std::move(aggs), std::move(agg_kernels)); } @@ -366,7 +365,7 @@ class GroupByNode : public ExecNode { {{"group_by", ToStringExtra()}, {"node.label", label()}, {"batch.length", batch.length}}); - size_t thread_index = plan_->GetThreadIndex(); + size_t thread_index = plan_->query_context()->GetThreadIndex(); if (thread_index >= local_states_.size()) { return Status::IndexError("thread index ", thread_index, " is out of range [0, ", local_states_.size(), ")"); @@ -393,7 +392,8 @@ class GroupByNode : public ExecNode { {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, {"function.kind", std::string(kind_name()) + "::Consume"}}); - KernelContext kernel_ctx{ctx_}; + auto ctx = plan_->query_context()->exec_context(); + KernelContext kernel_ctx{ctx}; kernel_ctx.SetState(state->agg_states[i].get()); ExecSpan agg_batch({batch[agg_src_field_ids_[i]], ExecValue(*id_batch.array())}, @@ -429,7 +429,9 @@ class GroupByNode : public ExecNode { {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, {"function.kind", std::string(kind_name()) + "::Merge"}}); - KernelContext batch_ctx{ctx_}; + + auto ctx = plan_->query_context()->exec_context(); + KernelContext batch_ctx{ctx}; DCHECK(state0->agg_states[i]); batch_ctx.SetState(state0->agg_states[i].get()); @@ -497,7 +499,7 @@ class GroupByNode : public ExecNode { int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size()); outputs_[0]->InputFinished(this, static_cast(num_output_batches)); - RETURN_NOT_OK(plan_->StartTaskGroup(output_task_group_id_, num_output_batches)); + RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_, num_output_batches)); return Status::OK(); } @@ -548,7 +550,7 @@ class GroupByNode : public ExecNode { {"node.detail", ToString()}, {"node.kind", kind_name()}}); - local_states_.resize(plan_->max_concurrency()); + local_states_.resize(plan_->query_context()->max_concurrency()); return Status::OK(); } @@ -593,7 +595,7 @@ class GroupByNode : public ExecNode { }; ThreadLocalState* GetLocalState() { - size_t thread_index = plan_->GetThreadIndex(); + size_t thread_index = plan_->query_context()->GetThreadIndex(); return &local_states_[thread_index]; } diff --git a/cpp/src/arrow/compute/exec/asof_join_node.cc b/cpp/src/arrow/compute/exec/asof_join_node.cc index 3da612aa03e41..0ca90256f7af2 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node.cc +++ b/cpp/src/arrow/compute/exec/asof_join_node.cc @@ -550,7 +550,7 @@ class AsofJoinNode : public ExecNode { if (dst.empty()) { return NULLPTR; } else { - return dst.Materialize(plan()->exec_context()->memory_pool(), output_schema(), + return dst.Materialize(plan()->query_context()->memory_pool(), output_schema(), state_); } } diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 15d95690076a1..5c84e24e03829 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -45,9 +45,11 @@ namespace compute { namespace { struct ExecPlanImpl : public ExecPlan { - explicit ExecPlanImpl(ExecContext* exec_context, - std::shared_ptr metadata = NULLPTR) - : ExecPlan(exec_context), metadata_(std::move(metadata)) {} + explicit ExecPlanImpl( + QueryOptions options, + ExecContext *exec_context, + std::shared_ptr metadata = NULLPTR) + : ExecPlan(options, exec_context), metadata_(std::move(metadata)) {} ~ExecPlanImpl() override { if (started_ && !finished_.is_finished()) { @@ -57,9 +59,6 @@ struct ExecPlanImpl : public ExecPlan { } } - size_t GetThreadIndex() { return thread_indexer_(); } - size_t max_concurrency() const { return thread_indexer_.Capacity(); } - ExecNode* AddNode(std::unique_ptr node) { if (node->label().empty()) { node->SetLabel(std::to_string(auto_label_counter_++)); @@ -74,45 +73,6 @@ struct ExecPlanImpl : public ExecPlan { return nodes_.back().get(); } - Result> BeginExternalTask() { - Future<> completion_future = Future<>::Make(); - ARROW_ASSIGN_OR_RAISE(bool task_added, - task_group_.AddTaskIfNotEnded(completion_future)); - if (task_added) { - return std::move(completion_future); - } - // Return an invalid future if we were already finished to signal to the - // caller that they should not begin the task - return Future<>{}; - } - - Status ScheduleTask(std::function fn) { - auto executor = exec_context_->executor(); - if (!executor) return fn(); - // Adds a task which submits fn to the executor and tracks its progress. If we're - // already stopping then the task is ignored and fn is not executed. - return task_group_ - .AddTaskIfNotEnded([executor, fn]() { return executor->Submit(std::move(fn)); }) - .status(); - } - - Status ScheduleTask(std::function fn) { - std::function indexed_fn = [this, fn]() { - size_t thread_index = GetThreadIndex(); - return fn(thread_index); - }; - return ScheduleTask(std::move(indexed_fn)); - } - - int RegisterTaskGroup(std::function task, - std::function on_finished) { - return task_scheduler_->RegisterTaskGroup(std::move(task), std::move(on_finished)); - } - - Status StartTaskGroup(int task_group_id, int64_t num_tasks) { - return task_scheduler_->StartTaskGroup(GetThreadIndex(), task_group_id, num_tasks); - } - Status Validate() const { if (nodes_.empty()) { return Status::Invalid("ExecPlan has no node"); @@ -140,6 +100,9 @@ struct ExecPlanImpl : public ExecPlan { return Status::Invalid("restarted ExecPlan"); } + QueryContext *ctx = query_context(); + RETURN_NOT_OK(ctx->Init(ctx->max_concurrency())); + std::vector> futures; for (auto& n : nodes_) { RETURN_NOT_OK(n->Init()); @@ -151,17 +114,17 @@ struct ExecPlanImpl : public ExecPlan { EndTaskGroup(); }); - task_scheduler_->RegisterEnd(); + ctx->scheduler()->RegisterEnd(); int num_threads = 1; bool sync_execution = true; - if (auto executor = exec_context()->executor()) { + if (auto executor = query_context()->executor()) { num_threads = executor->GetCapacity(); sync_execution = false; } - RETURN_NOT_OK(task_scheduler_->StartScheduling( + RETURN_NOT_OK(ctx->scheduler()->StartScheduling( 0 /* thread_index */, - [this](std::function fn) -> Status { - return this->ScheduleTask(std::move(fn)); + [ctx](std::function fn) -> Status { + return ctx->ScheduleTask(std::move(fn)); }, /*concurrent_tasks=*/2 * num_threads, sync_execution)); @@ -197,7 +160,7 @@ struct ExecPlanImpl : public ExecPlan { void EndTaskGroup() { bool expected = false; if (group_ended_.compare_exchange_strong(expected, true)) { - task_group_.End().AddCallback([this](const Status& st) { + query_context()->task_group()->End().AddCallback([this](const Status& st) { MARK_SPAN(span_, error_st_ & st); END_SPAN(span_); finished_.MarkFinished(error_st_ & st); @@ -209,7 +172,7 @@ struct ExecPlanImpl : public ExecPlan { DCHECK(started_) << "stopped an ExecPlan which never started"; EVENT(span_, "StopProducing"); stopped_ = true; - task_scheduler_->Abort( + query_context()->scheduler()->Abort( [this]() { StopProducingImpl(sorted_nodes_.begin(), sorted_nodes_.end()); }); } @@ -326,10 +289,7 @@ struct ExecPlanImpl : public ExecPlan { util::tracing::Span span_; std::shared_ptr metadata_; - ThreadIndexer thread_indexer_; std::atomic group_ended_{false}; - util::AsyncTaskGroup task_group_; - std::unique_ptr task_scheduler_ = TaskScheduler::Make(); }; ExecPlanImpl* ToDerived(ExecPlan* ptr) { return checked_cast(ptr); } @@ -351,8 +311,15 @@ util::optional GetNodeIndex(const std::vector& nodes, const uint32_t ExecPlan::kMaxBatchSize; Result> ExecPlan::Make( - ExecContext* ctx, std::shared_ptr metadata) { - return std::shared_ptr(new ExecPlanImpl{ctx, metadata}); + QueryOptions opts, + ExecContext *ctx, + std::shared_ptr metadata) { + return std::shared_ptr(new ExecPlanImpl{opts, ctx, std::move(metadata)}); +} + +Result> ExecPlan::Make( + ExecContext *ctx, std::shared_ptr metadata) { + return Make({}, ctx, std::move(metadata)); } ExecNode* ExecPlan::AddNode(std::unique_ptr node) { @@ -365,27 +332,6 @@ const ExecPlan::NodeVector& ExecPlan::sources() const { const ExecPlan::NodeVector& ExecPlan::sinks() const { return ToDerived(this)->sinks_; } -size_t ExecPlan::GetThreadIndex() { return ToDerived(this)->GetThreadIndex(); } -size_t ExecPlan::max_concurrency() const { return ToDerived(this)->max_concurrency(); } - -Result> ExecPlan::BeginExternalTask() { - return ToDerived(this)->BeginExternalTask(); -} - -Status ExecPlan::ScheduleTask(std::function fn) { - return ToDerived(this)->ScheduleTask(std::move(fn)); -} -Status ExecPlan::ScheduleTask(std::function fn) { - return ToDerived(this)->ScheduleTask(std::move(fn)); -} -int ExecPlan::RegisterTaskGroup(std::function task, - std::function on_finished) { - return ToDerived(this)->RegisterTaskGroup(std::move(task), std::move(on_finished)); -} -Status ExecPlan::StartTaskGroup(int task_group_id, int64_t num_tasks) { - return ToDerived(this)->StartTaskGroup(task_group_id, num_tasks); -} - Status ExecPlan::Validate() { return ToDerived(this)->Validate(); } Status ExecPlan::StartProducing() { return ToDerived(this)->StartProducing(); } @@ -475,7 +421,7 @@ MapNode::MapNode(ExecPlan* plan, std::vector inputs, std::move(output_schema), /*num_outputs=*/1) { if (async_mode) { - executor_ = plan_->exec_context()->executor(); + executor_ = plan_->query_context()->executor(); } else { executor_ = nullptr; } diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 5e52f606a69d5..0fb78ea9a382f 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -24,6 +24,7 @@ #include "arrow/compute/exec.h" #include "arrow/compute/exec/util.h" +#include "arrow/compute/exec/query_context.h" #include "arrow/compute/type_fwd.h" #include "arrow/type_fwd.h" #include "arrow/util/async_util.h" @@ -46,11 +47,16 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { virtual ~ExecPlan() = default; - ExecContext* exec_context() const { return exec_context_; } + QueryContext* query_context() { return &query_context_; } /// Make an empty exec plan static Result> Make( - ExecContext* = default_exec_context(), + QueryOptions options, + ExecContext *exec_context = default_exec_context(), + std::shared_ptr metadata = NULLPTR); + + static Result> Make( + ExecContext *exec_context = default_exec_context(), std::shared_ptr metadata = NULLPTR); ExecNode* AddNode(std::unique_ptr node); @@ -63,60 +69,6 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { return out; } - /// \brief Returns the index of the current thread. - size_t GetThreadIndex(); - /// \brief Returns the maximum number of threads that the plan could use. - /// - /// GetThreadIndex will always return something less than this, so it is safe to - /// e.g. make an array of thread-locals off this. - size_t max_concurrency() const; - - /// \brief Start an external task - /// - /// This should be avoided if possible. It is kept in for now for legacy - /// purposes. This should be called before the external task is started. If - /// a valid future is returned then it should be marked complete when the - /// external task has finished. - /// - /// \return an invalid future if the plan has already ended, otherwise this - /// returns a future that must be completed when the external task - /// finishes. - Result> BeginExternalTask(); - - /// \brief Add a single function as a task to the plan's task group. - /// - /// \param fn The task to run. Takes no arguments and returns a Status. - Status ScheduleTask(std::function fn); - - /// \brief Add a single function as a task to the plan's task group. - /// - /// \param fn The task to run. Takes the thread index and returns a Status. - Status ScheduleTask(std::function fn); - // Register/Start TaskGroup is a way of performing a "Parallel For" pattern: - // - The task function takes the thread index and the index of the task - // - The on_finished function takes the thread index - // Returns an integer ID that will be used to reference the task group in - // StartTaskGroup. At runtime, call StartTaskGroup with the ID and the number of times - // you'd like the task to be executed. The need to register a task group before use will - // be removed after we rewrite the scheduler. - /// \brief Register a "parallel for" task group with the scheduler - /// - /// \param task The function implementing the task. Takes the thread_index and - /// the task index. - /// \param on_finished The function that gets run once all tasks have been completed. - /// Takes the thread_index. - /// - /// Must be called inside of ExecNode::Init. - int RegisterTaskGroup(std::function task, - std::function on_finished); - - /// \brief Start the task group with the specified ID. This can only - /// be called once per task_group_id. - /// - /// \param task_group_id The ID of the task group to run - /// \param num_tasks The number of times to run the task - Status StartTaskGroup(int task_group_id, int64_t num_tasks); - /// The initial inputs const NodeVector& sources() const; @@ -146,25 +98,11 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { /// \brief Return the plan's attached metadata std::shared_ptr metadata() const; - /// \brief Should the plan use a legacy batching strategy - /// - /// This is currently in place only to support the Scanner::ToTable - /// method. This method relies on batch indices from the scanner - /// remaining consistent. This is impractical in the ExecPlan which - /// might slice batches as needed (e.g. for a join) - /// - /// However, it still works for simple plans and this is the only way - /// we have at the moment for maintaining implicit order. - bool UseLegacyBatching() const { return use_legacy_batching_; } - // For internal use only, see above comment - void SetUseLegacyBatching(bool value) { use_legacy_batching_ = value; } - std::string ToString() const; protected: - ExecContext* exec_context_; - bool use_legacy_batching_ = false; - explicit ExecPlan(ExecContext* exec_context) : exec_context_(exec_context) {} + QueryContext query_context_; + explicit ExecPlan(QueryOptions options, ExecContext *exec_ctx) : query_context_(options, *exec_ctx) {} }; class ARROW_EXPORT ExecNode { diff --git a/cpp/src/arrow/compute/exec/filter_node.cc b/cpp/src/arrow/compute/exec/filter_node.cc index b424da35f8529..6e5bea622b398 100644 --- a/cpp/src/arrow/compute/exec/filter_node.cc +++ b/cpp/src/arrow/compute/exec/filter_node.cc @@ -51,7 +51,7 @@ class FilterNode : public MapNode { auto filter_expression = filter_options.filter_expression; if (!filter_expression.IsBound()) { ARROW_ASSIGN_OR_RAISE(filter_expression, - filter_expression.Bind(*schema, plan->exec_context())); + filter_expression.Bind(*schema, plan->query_context()->exec_context())); } if (filter_expression.type()->id() != Type::BOOL) { @@ -77,7 +77,7 @@ class FilterNode : public MapNode { {"filter.length", target.length}}); ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target, - plan()->exec_context())); + plan()->query_context()->exec_context())); if (mask.is_scalar()) { const auto& mask_scalar = mask.scalar_as(); diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 5cf66b3d09e48..b8813e869fda2 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -40,16 +40,14 @@ class HashJoinBasicImpl : public HashJoinImpl { struct ThreadLocalState; public: - Status Init(ExecContext* ctx, JoinType join_type, size_t num_threads, + Status Init(QueryContext* ctx, JoinType join_type, size_t num_threads, const HashJoinProjectionMaps* proj_map_left, const HashJoinProjectionMaps* proj_map_right, - std::vector key_cmp, Expression filter, - RegisterTaskGroupCallback register_task_group_callback, - StartTaskGroupCallback start_task_group_callback, - OutputBatchCallback output_batch_callback, - FinishedCallback finished_callback) override { + std::vector *key_cmp, + Expression *filter, + CallbackRecord callback_record) override { START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", - {{"detail", filter.ToString()}, + {{"detail", filter->ToString()}, {"join.kind", arrow::compute::ToString(join_type)}, {"join.threads", static_cast(num_threads)}}); @@ -58,12 +56,9 @@ class HashJoinBasicImpl : public HashJoinImpl { join_type_ = join_type; schema_[0] = proj_map_left; schema_[1] = proj_map_right; - key_cmp_ = std::move(key_cmp); - filter_ = std::move(filter); - register_task_group_callback_ = std::move(register_task_group_callback); - start_task_group_callback_ = std::move(start_task_group_callback); - output_batch_callback_ = std::move(output_batch_callback); - finished_callback_ = std::move(finished_callback); + key_cmp_ = key_cmp; + filter_ = filter; + callback_record_ = std::move(callback_record); local_states_.resize(num_threads_); for (size_t i = 0; i < local_states_.size(); ++i) { @@ -99,7 +94,7 @@ class HashJoinBasicImpl : public HashJoinImpl { for (int icol = 0; icol < num_cols; ++icol) { data_types[icol] = schema_[side]->data_type(projection_handle, icol); } - encoder->Init(data_types, ctx_); + encoder->Init(data_types, ctx_->exec_context()); encoder->Clear(); } @@ -156,7 +151,7 @@ class HashJoinBasicImpl : public HashJoinImpl { bool is_null = non_null_bit_vectors[icol] && !bit_util::GetBit(non_null_bit_vectors[icol], non_null_bit_vector_offsets[icol] + irow); - if (key_cmp_[icol] == JoinKeyCmp::EQ && is_null) { + if ((*key_cmp_)[icol] == JoinKeyCmp::EQ && is_null) { no_match = true; break; } @@ -233,7 +228,7 @@ class HashJoinBasicImpl : public HashJoinImpl { : opt_right_payload->values[from_payload.get(icol)]; } - output_batch_callback_(0, std::move(result)); + callback_record_.output_batch(0, std::move(result)); // Update the counter of produced batches // @@ -245,7 +240,7 @@ class HashJoinBasicImpl : public HashJoinImpl { std::vector& no_match, std::vector& match_left, std::vector& match_right) { - if (filter_ == literal(true)) { + if (*filter_ == literal(true)) { return Status::OK(); } ARROW_DCHECK_EQ(match_left.size(), match_right.size()); @@ -298,7 +293,7 @@ class HashJoinBasicImpl : public HashJoinImpl { AppendFields(right_to_key, right_to_pay, right_key, right_payload); ARROW_ASSIGN_OR_RAISE(Datum mask, - ExecuteScalarExpression(filter_, concatenated, ctx_)); + ExecuteScalarExpression(*filter_, concatenated, ctx_->exec_context())); size_t num_probed_rows = match.size() + no_match.size(); if (mask.is_scalar()) { @@ -398,7 +393,7 @@ class HashJoinBasicImpl : public HashJoinImpl { ARROW_ASSIGN_OR_RAISE(right_key, hash_table_keys_.Decode(batch_size_next, opt_right_ids)); // Post process build side keys that use dictionary - RETURN_NOT_OK(dict_build_.PostDecode(*schema_[1], &right_key, ctx_)); + RETURN_NOT_OK(dict_build_.PostDecode(*schema_[1], &right_key, ctx_->exec_context())); } if (has_right_payload) { ARROW_ASSIGN_OR_RAISE(right_payload, @@ -511,12 +506,12 @@ class HashJoinBasicImpl : public HashJoinImpl { local_state.match_right.clear(); bool use_key_batch_for_dicts = - dict_probe_.BatchRemapNeeded(thread_index, *schema_[0], *schema_[1], ctx_); + dict_probe_.BatchRemapNeeded(thread_index, *schema_[0], *schema_[1], ctx_->exec_context()); RowEncoder* row_encoder_for_lookups = &local_state.exec_batch_keys; if (use_key_batch_for_dicts) { RETURN_NOT_OK(dict_probe_.EncodeBatch(thread_index, *schema_[0], *schema_[1], dict_build_, batch, &row_encoder_for_lookups, - &batch_key_for_lookups, ctx_)); + &batch_key_for_lookups, ctx_->exec_context())); } // Collect information about all nulls in key columns. @@ -550,7 +545,7 @@ class HashJoinBasicImpl : public HashJoinImpl { } void RegisterBuildHashTable() { - task_group_build_ = register_task_group_callback_( + task_group_build_ = callback_record_.register_task_group( [this](size_t thread_index, int64_t task_id) -> Status { return BuildHashTable_exec_task(thread_index, task_id); }, @@ -561,7 +556,7 @@ class HashJoinBasicImpl : public HashJoinImpl { Status BuildHashTable_exec_task(size_t thread_index, int64_t /*task_id*/) { AccumulationQueue batches = std::move(build_batches_); - dict_build_.InitEncoder(*schema_[1], &hash_table_keys_, ctx_); + dict_build_.InitEncoder(*schema_[1], &hash_table_keys_, ctx_->exec_context()); bool has_payload = (schema_[1]->num_cols(HashJoinProjection::PAYLOAD) > 0); if (has_payload) { InitEncoder(1, HashJoinProjection::PAYLOAD, &hash_table_payloads_); @@ -578,11 +573,11 @@ class HashJoinBasicImpl : public HashJoinImpl { } else if (hash_table_empty_) { hash_table_empty_ = false; - RETURN_NOT_OK(dict_build_.Init(*schema_[1], &batch, ctx_)); + RETURN_NOT_OK(dict_build_.Init(*schema_[1], &batch, ctx_->exec_context())); } int32_t num_rows_before = hash_table_keys_.num_rows(); RETURN_NOT_OK(dict_build_.EncodeBatch(thread_index, *schema_[1], batch, - &hash_table_keys_, ctx_)); + &hash_table_keys_, ctx_->exec_context())); if (has_payload) { RETURN_NOT_OK( EncodeBatch(1, HashJoinProjection::PAYLOAD, &hash_table_payloads_, batch)); @@ -594,7 +589,7 @@ class HashJoinBasicImpl : public HashJoinImpl { } if (hash_table_empty_) { - RETURN_NOT_OK(dict_build_.Init(*schema_[1], nullptr, ctx_)); + RETURN_NOT_OK(dict_build_.Init(*schema_[1], nullptr, ctx_->exec_context())); } return Status::OK(); @@ -610,12 +605,12 @@ class HashJoinBasicImpl : public HashJoinImpl { BuildFinishedCallback on_finished) override { build_finished_callback_ = std::move(on_finished); build_batches_ = std::move(batches); - return start_task_group_callback_(task_group_build_, - /*num_tasks=*/1); + return callback_record_.start_task_group(task_group_build_, + /*num_tasks=*/1); } void RegisterScanHashTable() { - task_group_scan_ = register_task_group_callback_( + task_group_scan_ = callback_record_.register_task_group( [this](size_t thread_index, int64_t task_id) -> Status { return ScanHashTable_exec_task(thread_index, task_id); }, @@ -684,13 +679,12 @@ class HashJoinBasicImpl : public HashJoinImpl { return Status::Cancelled("Hash join cancelled"); } END_SPAN(span_); - finished_callback_(num_batches_produced_.load()); - return Status::OK(); + return callback_record_.finished(num_batches_produced_.load()); } Status ScanHashTable(size_t thread_index) { MergeHasMatch(); - return start_task_group_callback_(task_group_scan_, ScanHashTable_num_tasks()); + return callback_record_.start_task_group(task_group_scan_, ScanHashTable_num_tasks()); } Status ProbingFinished(size_t thread_index) override { @@ -735,22 +729,19 @@ class HashJoinBasicImpl : public HashJoinImpl { // Metadata // - ExecContext* ctx_; + QueryContext* ctx_; JoinType join_type_; size_t num_threads_; const HashJoinProjectionMaps* schema_[2]; - std::vector key_cmp_; - Expression filter_; + std::vector *key_cmp_; + Expression *filter_; int task_group_build_; int task_group_scan_; // Callbacks // - RegisterTaskGroupCallback register_task_group_callback_; - StartTaskGroupCallback start_task_group_callback_; - OutputBatchCallback output_batch_callback_; + CallbackRecord callback_record_; BuildFinishedCallback build_finished_callback_; - FinishedCallback finished_callback_; // Thread local runtime state // diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/hash_join.h index 0c5e43467e911..9217d48143479 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/compute/exec/hash_join.h @@ -34,33 +34,37 @@ namespace arrow { namespace compute { -using arrow::util::AccumulationQueue; - class HashJoinImpl { public: - using OutputBatchCallback = std::function; - using BuildFinishedCallback = std::function; - using FinishedCallback = std::function; - using RegisterTaskGroupCallback = std::function, std::function)>; - using StartTaskGroupCallback = std::function; - using AbortContinuationImpl = std::function; + using OutputBatchCallback = std::function; + using BuildFinishedCallback = std::function; + using FinishedCallback = std::function; + using RegisterTaskGroupCallback = std::function, std::function)>; + using StartTaskGroupCallback = std::function; + using AbortContinuationImpl = std::function; + + struct CallbackRecord + { + RegisterTaskGroupCallback register_task_group; + StartTaskGroupCallback start_task_group; + OutputBatchCallback output_batch; + FinishedCallback finished; + }; virtual ~HashJoinImpl() = default; - virtual Status Init(ExecContext* ctx, JoinType join_type, size_t num_threads, + virtual Status Init(QueryContext* ctx, JoinType join_type, size_t num_threads, const HashJoinProjectionMaps* proj_map_left, const HashJoinProjectionMaps* proj_map_right, - std::vector key_cmp, Expression filter, - RegisterTaskGroupCallback register_task_group_callback, - StartTaskGroupCallback start_task_group_callback, - OutputBatchCallback output_batch_callback, - FinishedCallback finished_callback) = 0; + std::vector *key_cmp, + Expression *filter, + CallbackRecord callback_record) = 0; virtual Status BuildHashTable(size_t thread_index, AccumulationQueue batches, BuildFinishedCallback on_finished) = 0; virtual Status ProbeSingleBatch(size_t thread_index, ExecBatch batch) = 0; virtual Status ProbingFinished(size_t thread_index) = 0; - virtual void Abort(TaskScheduler::AbortContinuationImpl pos_abort_callback) = 0; + virtual void Abort(AbortContinuationImpl pos_abort_callback) = 0; virtual std::string ToString() const = 0; static Result> MakeBasic(); diff --git a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc index 94201a849fa43..919c060415fab 100644 --- a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc +++ b/cpp/src/arrow/compute/exec/hash_join_benchmark.cc @@ -126,14 +126,11 @@ class JoinBenchmark { stats_.num_probe_rows = settings.num_probe_batches * settings.batch_size; - ctx_ = arrow::internal::make_unique(default_memory_pool(), - arrow::internal::GetCpuThreadPool()); - schema_mgr_ = arrow::internal::make_unique(); - Expression filter = literal(true); + filter_ = literal(true); DCHECK_OK(schema_mgr_->Init(settings.join_type, *l_batches_with_schema.schema, left_keys, *r_batches_with_schema.schema, right_keys, - filter, "l_", "r_")); + filter_, "l_", "r_")); if (settings.use_basic_implementation) { join_ = *HashJoinImpl::MakeBasic(); @@ -149,22 +146,26 @@ class JoinBenchmark { }; scheduler_ = TaskScheduler::Make(); + DCHECK_OK(ctx_.Init(settings.num_threads)); - auto register_task_group_callback = [&](std::function task, + HashJoinImpl::CallbackRecord callbacks; + callbacks.register_task_group = [&](std::function task, std::function cont) { return scheduler_->RegisterTaskGroup(std::move(task), std::move(cont)); }; - auto start_task_group_callback = [&](int task_group_id, int64_t num_tasks) { + callbacks.start_task_group = [&](int task_group_id, int64_t num_tasks) { return scheduler_->StartTaskGroup(omp_get_thread_num(), task_group_id, num_tasks); }; + callbacks.output_batch = [](int64_t, ExecBatch) {}; + callbacks.finished = [](int64_t){ return Status::OK(); }; DCHECK_OK(join_->Init( - ctx_.get(), settings.join_type, settings.num_threads, - &(schema_mgr_->proj_maps[0]), &(schema_mgr_->proj_maps[1]), std::move(key_cmp), - std::move(filter), std::move(register_task_group_callback), - std::move(start_task_group_callback), [](int64_t, ExecBatch) {}, - [](int64_t x) {})); + &ctx_, settings.join_type, settings.num_threads, + &(schema_mgr_->proj_maps[0]), &(schema_mgr_->proj_maps[1]), + &key_cmp_, + &filter_, + std::move(callbacks))); task_group_probe_ = scheduler_->RegisterTaskGroup( [this](size_t thread_index, int64_t task_id) -> Status { @@ -200,7 +201,9 @@ class JoinBenchmark { AccumulationQueue r_batches_; std::unique_ptr schema_mgr_; std::unique_ptr join_; - std::unique_ptr ctx_; + QueryContext ctx_; + std::vector key_cmp_; + Expression filter_; int task_group_probe_; struct { diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 44667b9f28382..fd0f04401c793 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -23,6 +23,7 @@ #include "arrow/compute/exec/hash_join.h" #include "arrow/compute/exec/hash_join_dict.h" #include "arrow/compute/exec/hash_join_node.h" +#include "arrow/compute/exec/spilling_join.h" #include "arrow/compute/exec/key_hash.h" #include "arrow/compute/exec/options.h" #include "arrow/compute/exec/schema_util.h" @@ -124,49 +125,25 @@ Status HashJoinSchema::Init( right_schema, right_keys, right_output, left_field_name_suffix, right_field_name_suffix)); - std::vector handles; - std::vector*> field_refs; - std::vector left_filter, right_filter; RETURN_NOT_OK( CollectFilterColumns(left_filter, right_filter, filter, left_schema, right_schema)); - - handles.push_back(HashJoinProjection::KEY); - field_refs.push_back(&left_keys); - ARROW_ASSIGN_OR_RAISE(auto left_payload, ComputePayload(left_schema, left_output, left_filter, left_keys)); - handles.push_back(HashJoinProjection::PAYLOAD); - field_refs.push_back(&left_payload); - - handles.push_back(HashJoinProjection::FILTER); - field_refs.push_back(&left_filter); - - handles.push_back(HashJoinProjection::OUTPUT); - field_refs.push_back(&left_output); - - RETURN_NOT_OK( - proj_maps[0].Init(HashJoinProjection::INPUT, left_schema, handles, field_refs)); - handles.clear(); - field_refs.clear(); - - handles.push_back(HashJoinProjection::KEY); - field_refs.push_back(&right_keys); + RETURN_NOT_OK(proj_maps[0].Init(HashJoinProjection::INPUT, left_schema)); + RETURN_NOT_OK(proj_maps[0].RegisterProjectedSchema(HashJoinProjection::KEY, left_keys, left_schema)); + RETURN_NOT_OK(proj_maps[0].RegisterProjectedSchema(HashJoinProjection::PAYLOAD, left_payload, left_schema)); + RETURN_NOT_OK(proj_maps[0].RegisterProjectedSchema(HashJoinProjection::FILTER, left_filter, left_schema)); + RETURN_NOT_OK(proj_maps[0].RegisterProjectedSchema(HashJoinProjection::OUTPUT, left_output, left_schema)); ARROW_ASSIGN_OR_RAISE(auto right_payload, ComputePayload(right_schema, right_output, right_filter, right_keys)); - handles.push_back(HashJoinProjection::PAYLOAD); - field_refs.push_back(&right_payload); - - handles.push_back(HashJoinProjection::FILTER); - field_refs.push_back(&right_filter); - - handles.push_back(HashJoinProjection::OUTPUT); - field_refs.push_back(&right_output); - - RETURN_NOT_OK( - proj_maps[1].Init(HashJoinProjection::INPUT, right_schema, handles, field_refs)); + RETURN_NOT_OK(proj_maps[1].Init(HashJoinProjection::INPUT, right_schema)); + RETURN_NOT_OK(proj_maps[1].RegisterProjectedSchema(HashJoinProjection::KEY, right_keys, right_schema)); + RETURN_NOT_OK(proj_maps[1].RegisterProjectedSchema(HashJoinProjection::PAYLOAD, right_payload, right_schema)); + RETURN_NOT_OK(proj_maps[1].RegisterProjectedSchema(HashJoinProjection::FILTER, right_filter, right_schema)); + RETURN_NOT_OK(proj_maps[1].RegisterProjectedSchema(HashJoinProjection::OUTPUT, right_output, right_schema)); return Status::OK(); } @@ -341,7 +318,7 @@ std::shared_ptr HashJoinSchema::MakeOutputSchema( Result HashJoinSchema::BindFilter(Expression filter, const Schema& left_schema, const Schema& right_schema, - ExecContext* exec_context) { + ExecContext *exec_context) { if (filter.IsBound() || filter == literal(true)) { return std::move(filter); } @@ -475,6 +452,7 @@ Status ValidateHashJoinNodeOptions(const HashJoinNodeOptions& join_options) { class HashJoinNode; + // This is a struct encapsulating things related to Bloom filters and pushing them around // between HashJoinNodes. The general strategy is to notify other joins at plan-creation // time for that join to expect a Bloom filter. Once the full build side has been @@ -550,7 +528,7 @@ struct BloomFilterPushdownContext { std::vector hashes(batch.length); std::vector bv(bit_vector_bytes); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, GetStack(thread_index)); + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, ctx_->GetTempStack(thread_index)); // Start with full selection for the current batch memset(selected.data(), 0xff, bit_vector_bytes); @@ -586,7 +564,7 @@ struct BloomFilterPushdownContext { 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_)); + Filter(batch.values[i], selected_datum, options, ctx_->exec_context())); first_nonscalar = std::min(first_nonscalar, i); ARROW_DCHECK_EQ(batch.values[i].length(), batch.values[first_nonscalar].length()); } @@ -617,25 +595,10 @@ struct BloomFilterPushdownContext { // the disable_bloom_filter_ flag. std::pair> GetPushdownTarget(HashJoinNode* start); - Result GetStack(size_t thread_index) { - if (!tld_[thread_index].is_init) { - RETURN_NOT_OK(tld_[thread_index].stack.Init( - ctx_->memory_pool(), 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); - tld_[thread_index].is_init = true; - } - return &tld_[thread_index].stack; - } - StartTaskGroupCallback start_task_group_callback_; bool disable_bloom_filter_; HashJoinSchema* schema_mgr_; - ExecContext* ctx_; - - struct ThreadLocalData { - bool is_init = false; - util::TempVectorStack stack; - }; - std::vector tld_; + QueryContext* ctx_; struct { int task_id_; @@ -661,6 +624,7 @@ struct BloomFilterPushdownContext { FilterFinishedCallback on_finished_; } eval_; }; + bool HashJoinSchema::HasDictionaries() const { for (int side = 0; side <= 1; ++side) { for (int icol = 0; icol < proj_maps[side].num_cols(HashJoinProjection::INPUT); @@ -694,7 +658,8 @@ class HashJoinNode : public ExecNode { HashJoinNode(ExecPlan* plan, NodeVector inputs, const HashJoinNodeOptions& join_options, std::shared_ptr output_schema, std::unique_ptr schema_mgr, Expression filter, - std::unique_ptr impl) + std::unique_ptr impl, + bool is_swiss) : ExecNode(plan, inputs, {"left", "right"}, /*output_schema=*/std::move(output_schema), /*num_outputs=*/1), @@ -703,6 +668,7 @@ class HashJoinNode : public ExecNode { filter_(std::move(filter)), schema_mgr_(std::move(schema_mgr)), impl_(std::move(impl)), + is_swiss_(is_swiss), disable_bloom_filter_(join_options.disable_bloom_filter) { complete_.store(false); } @@ -737,51 +703,164 @@ class HashJoinNode : public ExecNode { ARROW_ASSIGN_OR_RAISE(Expression filter, schema_mgr->BindFilter(join_options.filter, left_schema, - right_schema, plan->exec_context())); + right_schema, plan->query_context()->exec_context())); // Generate output schema std::shared_ptr output_schema = schema_mgr->MakeOutputSchema( join_options.output_suffix_for_left, join_options.output_suffix_for_right); + bool use_swiss = use_swiss_join(filter, schema_mgr); + std::unique_ptr impl; + if (use_swiss) + { + ARROW_ASSIGN_OR_RAISE(impl, HashJoinImpl::MakeSwiss()); + } + else + { + ARROW_ASSIGN_OR_RAISE(impl, HashJoinImpl::MakeBasic()); + } + + return plan->EmplaceNode( + plan, inputs, join_options, std::move(output_schema), std::move(schema_mgr), + std::move(filter), std::move(impl), use_swiss); + } + + const char* kind_name() const override { return "HashJoinNode"; } + // Create hash join implementation object // SwissJoin does not support: // a) 64-bit string offsets // b) residual predicates // c) dictionaries // - bool use_swiss_join; + static bool use_swiss_join( + const Expression &filter, + const std::unique_ptr &schema) + { #if ARROW_LITTLE_ENDIAN - use_swiss_join = (filter == literal(true)) && !schema_mgr->HasDictionaries() && - !schema_mgr->HasLargeBinary(); + return (filter == literal(true) + && !schema->HasDictionaries() + && !schema->HasLargeBinary()); #else - use_swiss_join = false; + return false; #endif - std::unique_ptr impl; - if (use_swiss_join) { - ARROW_ASSIGN_OR_RAISE(impl, HashJoinImpl::MakeSwiss()); - } else { - ARROW_ASSIGN_OR_RAISE(impl, HashJoinImpl::MakeBasic()); } - return plan->EmplaceNode( - plan, inputs, join_options, std::move(output_schema), std::move(schema_mgr), - std::move(filter), std::move(impl)); - } + Status AddHashColumn( + ExecBatch *batch, + size_t thread_index, + const SchemaProjectionMaps &map) + { + for(int i = 0; i < batch->num_values(); i++) + { + if(batch->values[i].is_scalar()) + { + ARROW_ASSIGN_OR_RAISE( + batch->values[i], + MakeArrayFromScalar( + *batch->values[i].scalar(), + batch->length, + plan_->query_context()->memory_pool())); + } + } - const char* kind_name() const override { return "HashJoinNode"; } + ARROW_ASSIGN_OR_RAISE(std::unique_ptr hash_buf, + AllocateBuffer(sizeof(uint64_t) * batch->length, + plan_->query_context()->memory_pool())); + uint64_t *hashes = reinterpret_cast(hash_buf->mutable_data()); + std::vector temp_column_arrays; + int num_keys = map.num_cols(HashJoinProjection::KEY); + std::vector key_cols(num_keys); + for(int i = 0; i < num_keys; i++) + key_cols[i] = (*batch).values[i]; + + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, + plan_->query_context()->GetTempStack(thread_index)); + + ExecBatch key_batch(std::move(key_cols), batch->length); + RETURN_NOT_OK(Hashing64::HashBatch(std::move(key_batch), + hashes, + temp_column_arrays, + plan_->query_context()->cpu_info()->hardware_flags(), + stack, + 0, + batch->length)); + + ArrayData hash_data(uint64(), batch->length, { nullptr, std::move(hash_buf)}); + batch->values.emplace_back(std::move(hash_data)); + return Status::OK(); + } - Status OnBuildSideBatch(size_t thread_index, ExecBatch batch) { - std::lock_guard guard(build_side_mutex_); - build_accumulator_.InsertBatch(std::move(batch)); - return Status::OK(); - } + Status OnSpillingStarted(size_t) + { + { + std::lock_guard build_guard(build_side_mutex_); + spilling_build_ = true; + } + RETURN_NOT_OK(plan_->query_context()->StartTaskGroup( + task_group_spill_build_, + build_accumulator_.batch_count())); + + { + std::lock_guard probe_guard(probe_side_mutex_); + spilling_probe_ = true; + } + RETURN_NOT_OK(plan_->query_context()->StartTaskGroup( + task_group_spill_probe_, + probe_accumulator_.batch_count())); + + return Status::OK(); + } + + Status OnBuildSideAccumSpilled(size_t thread_index) + { + bool expected = false; + // If the CAS failed, it means that build_accum_spilled_ is already true, + // meaning that build side is finished. + if(!build_accum_spilled_.compare_exchange_strong(expected, true)) + return spilling_join_.OnBuildSideFinished(thread_index); + return Status::OK(); + } + + Status OnProbeSideAccumSpilled(size_t thread_index) + { + bool expected = false; + // If the CAS failed, it means that probe_accum_spilled_ is already true, + // meaning that build side is finished. + if(!probe_accum_spilled_.compare_exchange_strong(expected, true)) + return spilling_join_.OnProbeSideFinished(thread_index); + return Status::OK(); + } + + Status OnBuildSideBatch(size_t thread_index, ExecBatch batch) + { + { + std::lock_guard guard(build_side_mutex_); + if(!spilling_build_) + { + build_accumulator_.InsertBatch(std::move(batch)); + return Status::OK(); + } + } + RETURN_NOT_OK(spilling_join_.OnBuildSideBatch(thread_index, std::move(batch))); + return Status::OK(); + } Status OnBuildSideFinished(size_t thread_index) { - return pushdown_context_.BuildBloomFilter( - thread_index, std::move(build_accumulator_), - [this](size_t thread_index, AccumulationQueue batches) { - return OnBloomFilterFinished(thread_index, std::move(batches)); - }); + + if(!spilling_build_) + { + return pushdown_context_.BuildBloomFilter( + thread_index, std::move(build_accumulator_), + [this](size_t thread_index, AccumulationQueue batches) { + return OnBloomFilterFinished(thread_index, std::move(batches)); + }); + } + + bool expected = false; + if(!build_accum_spilled_.compare_exchange_strong(expected, true)) + return spilling_join_.OnBuildSideFinished(thread_index); + return Status::OK(); } Status OnBloomFilterFinished(size_t thread_index, AccumulationQueue batches) { @@ -805,6 +884,9 @@ class HashJoinNode : public ExecNode { } Status OnProbeSideBatch(size_t thread_index, ExecBatch batch) { + if(spilling_probe_) + return spilling_join_.OnProbeSideBatch(thread_index, std::move(batch)); + { std::lock_guard guard(probe_side_mutex_); if (!bloom_filters_ready_) { @@ -826,6 +908,14 @@ class HashJoinNode : public ExecNode { } Status OnProbeSideFinished(size_t thread_index) { + if(spilling_probe_) + { + bool expected = false; + if(!probe_accum_spilled_.compare_exchange_strong(expected, true)) + return spilling_join_.OnProbeSideFinished(thread_index); + return Status::OK(); + } + bool probing_finished; { std::lock_guard guard(probe_side_mutex_); @@ -839,7 +929,7 @@ class HashJoinNode : public ExecNode { Status OnFiltersReceived() { std::unique_lock guard(probe_side_mutex_); bloom_filters_ready_ = true; - size_t thread_index = plan_->GetThreadIndex(); + size_t thread_index = plan_->query_context()->GetThreadIndex(); AccumulationQueue batches = std::move(probe_accumulator_); guard.unlock(); return pushdown_context_.FilterBatches( @@ -868,8 +958,8 @@ class HashJoinNode : public ExecNode { std::lock_guard guard(probe_side_mutex_); queued_batches_to_probe_ = std::move(probe_accumulator_); } - return plan_->StartTaskGroup(task_group_probe_, - queued_batches_to_probe_.batch_count()); + return plan_->query_context()->StartTaskGroup(task_group_probe_, + queued_batches_to_probe_.batch_count()); } Status OnQueuedBatchesProbed(size_t thread_index) { @@ -890,7 +980,7 @@ class HashJoinNode : public ExecNode { return; } - size_t thread_index = plan_->GetThreadIndex(); + size_t thread_index = plan_->query_context()->GetThreadIndex(); int side = (input == inputs_[0]) ? 0 : 1; EVENT(span_, "InputReceived", {{"batch.length", batch.length}, {"side", side}}); @@ -898,6 +988,12 @@ class HashJoinNode : public ExecNode { START_COMPUTE_SPAN_WITH_PARENT(span, span_, "InputReceived", {{"batch.length", batch.length}}); + if(ErrorIfNotOk(AddHashColumn(&batch, thread_index, schema_mgr_->proj_maps[side]))) + { + StopProducing(); + return; + } + Status status = side == 0 ? OnProbeSideBatch(thread_index, std::move(batch)) : OnBuildSideBatch(thread_index, std::move(batch)); @@ -928,7 +1024,7 @@ class HashJoinNode : public ExecNode { void InputFinished(ExecNode* input, int total_batches) override { ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end()); - size_t thread_index = plan_->GetThreadIndex(); + size_t thread_index = plan_->query_context()->GetThreadIndex(); int side = (input == inputs_[0]) ? 0 : 1; EVENT(span_, "InputFinished", {{"side", side}, {"batches.length", total_batches}}); @@ -946,46 +1042,104 @@ class HashJoinNode : public ExecNode { } Status Init() override { - RETURN_NOT_OK(ExecNode::Init()); - if (plan_->UseLegacyBatching()) { + QueryContext *ctx = plan_->query_context(); + if (ctx->options().use_legacy_batching) { return Status::Invalid( "The plan was configured to use legacy batching but contained a join node " "which is incompatible with legacy batching"); } - bool use_sync_execution = !(plan_->exec_context()->executor()); + + bool use_sync_execution = !(ctx->executor()); // TODO(ARROW-15732) // Each side of join might have an IO thread being called from. Once this is fixed // we will change it back to just the CPU's thread pool capacity. size_t num_threads = (GetCpuThreadPoolCapacity() + io::GetIOThreadPoolCapacity() + 1); + + auto register_task_group = [ctx](std::function fn, + std::function on_finished) + { + return ctx->RegisterTaskGroup(std::move(fn), std::move(on_finished)); + }; + + auto start_task_group = [ctx](int task_group_id, int64_t num_tasks) + { + return ctx->StartTaskGroup(task_group_id, num_tasks); + }; + + auto output_batch = [this](int64_t, ExecBatch batch) { this->OutputBatchCallback(batch); }; + auto finished = [this](int64_t total_num_batches) { return this->FinishedCallback(total_num_batches); }; + pushdown_context_.Init( - this, num_threads, - [this](std::function fn, - std::function on_finished) { - return plan_->RegisterTaskGroup(std::move(fn), std::move(on_finished)); - }, - [this](int task_group_id, int64_t num_tasks) { - return plan_->StartTaskGroup(task_group_id, num_tasks); - }, - [this]() { return OnFiltersReceived(); }, disable_bloom_filter_, + this, + num_threads, + register_task_group, + start_task_group, + [this]() { return OnFiltersReceived(); }, + disable_bloom_filter_, use_sync_execution); + HashJoinImpl::CallbackRecord join_callbacks; + join_callbacks.register_task_group = register_task_group; + join_callbacks.start_task_group = start_task_group; + join_callbacks.output_batch = output_batch; + join_callbacks.finished = finished; + RETURN_NOT_OK(impl_->Init( - plan_->exec_context(), join_type_, num_threads, &(schema_mgr_->proj_maps[0]), - &(schema_mgr_->proj_maps[1]), key_cmp_, filter_, - [this](std::function fn, - std::function on_finished) { - return plan_->RegisterTaskGroup(std::move(fn), std::move(on_finished)); + ctx, join_type_, num_threads, &(schema_mgr_->proj_maps[0]), + &(schema_mgr_->proj_maps[1]), &key_cmp_, &filter_, + std::move(join_callbacks))); + + SpillingHashJoin::CallbackRecord spilling_callbacks; + spilling_callbacks.register_task_group = register_task_group; + spilling_callbacks.start_task_group = start_task_group; + spilling_callbacks.output_batch = output_batch; + spilling_callbacks.finished = finished; + spilling_callbacks.start_spilling = [this](size_t thread_index) + { + return OnSpillingStarted(thread_index); + }; + spilling_callbacks.pause_probe_side = [this](int counter) + { + inputs_[0]->PauseProducing(this, counter); + }; + spilling_callbacks.resume_probe_side = [this](int counter) + { + inputs_[0]->ResumeProducing(this, counter); + }; + + RETURN_NOT_OK(spilling_join_.Init( + ctx, + join_type_, + num_threads, + &(schema_mgr_->proj_maps[0]), + &(schema_mgr_->proj_maps[1]), + &key_cmp_, + &filter_, + std::move(spilling_callbacks), + is_swiss_)); + + task_group_spill_build_ = ctx->RegisterTaskGroup( + [this](size_t thread_index, int64_t task_id) -> Status + { + return spilling_join_.OnBuildSideBatch(thread_index, std::move(build_accumulator_[task_id])); }, - [this](int task_group_id, int64_t num_tasks) { - return plan_->StartTaskGroup(task_group_id, num_tasks); + [this](size_t thread_index) -> Status + { + return OnBuildSideAccumSpilled(thread_index); + }); + + task_group_spill_probe_ = ctx->RegisterTaskGroup( + [this](size_t thread_index, int64_t task_id) -> Status + { + return spilling_join_.OnProbeSideBatch(thread_index, std::move(probe_accumulator_[task_id])); }, - [this](int64_t, ExecBatch batch) { this->OutputBatchCallback(batch); }, - [this](int64_t total_num_batches) { - this->FinishedCallback(total_num_batches); - })); + [this](size_t thread_index) -> Status + { + return OnProbeSideAccumSpilled(thread_index); + }); - task_group_probe_ = plan_->RegisterTaskGroup( + task_group_probe_ = ctx->RegisterTaskGroup( [this](size_t thread_index, int64_t task_id) -> Status { return impl_->ProbeSingleBatch(thread_index, std::move(queued_batches_to_probe_[task_id])); @@ -1041,12 +1195,13 @@ class HashJoinNode : public ExecNode { outputs_[0]->InputReceived(this, std::move(batch)); } - void FinishedCallback(int64_t total_num_batches) { + Status FinishedCallback(int64_t total_num_batches) { bool expected = false; if (complete_.compare_exchange_strong(expected, true)) { outputs_[0]->InputFinished(this, static_cast(total_num_batches)); finished_.MarkFinished(); } + return Status::OK(); } private: @@ -1057,14 +1212,19 @@ class HashJoinNode : public ExecNode { Expression filter_; std::unique_ptr schema_mgr_; std::unique_ptr impl_; - util::AccumulationQueue build_accumulator_; - util::AccumulationQueue probe_accumulator_; - util::AccumulationQueue queued_batches_to_probe_; + bool is_swiss_; + + AccumulationQueue build_accumulator_; + AccumulationQueue probe_accumulator_; + AccumulationQueue queued_batches_to_probe_; std::mutex build_side_mutex_; std::mutex probe_side_mutex_; + int task_group_spill_build_; + int task_group_spill_probe_; int task_group_probe_; + bool bloom_filters_ready_ = false; bool hash_table_ready_ = false; bool queued_batches_filtered_ = false; @@ -1072,8 +1232,15 @@ class HashJoinNode : public ExecNode { bool probe_side_finished_ = false; friend struct BloomFilterPushdownContext; + bool disable_bloom_filter_; BloomFilterPushdownContext pushdown_context_; + SpillingHashJoin spilling_join_; + bool spilling_build_ = false; + bool spilling_probe_ = false; + + std::atomic build_accum_spilled_{false}; + std::atomic probe_accum_spilled_{false}; }; void BloomFilterPushdownContext::Init( @@ -1083,8 +1250,7 @@ void BloomFilterPushdownContext::Init( FiltersReceivedCallback on_bloom_filters_received, bool disable_bloom_filter, bool use_sync_execution) { schema_mgr_ = owner->schema_mgr_.get(); - ctx_ = owner->plan_->exec_context(); - tld_.resize(num_threads); + ctx_ = owner->plan_->query_context(); disable_bloom_filter_ = disable_bloom_filter; std::tie(push_.pushdown_target_, push_.column_map_) = GetPushdownTarget(owner); eval_.all_received_callback_ = std::move(on_bloom_filters_received); @@ -1131,7 +1297,7 @@ Status BloomFilterPushdownContext::BuildBloomFilter(size_t thread_index, return build_.on_finished_(thread_index, std::move(build_.batches_)); RETURN_NOT_OK(build_.builder_->Begin( - /*num_threads=*/tld_.size(), ctx_->cpu_info()->hardware_flags(), + /*num_threads=*/ctx_->max_concurrency(), ctx_->cpu_info()->hardware_flags(), ctx_->memory_pool(), build_.batches_.row_count(), build_.batches_.batch_count(), push_.bloom_filter_.get())); @@ -1163,7 +1329,7 @@ Status BloomFilterPushdownContext::BuildBloomFilter_exec_task(size_t thread_inde } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, GetStack(thread_index)); + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, ctx_->GetTempStack(thread_index)); util::TempVectorHolder hash_holder(stack, util::MiniBatch::kMiniBatchLength); uint32_t* hashes = hash_holder.mutable_data(); for (int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) { @@ -1283,6 +1449,7 @@ std::pair> BloomFilterPushdownContext::GetPushdo #endif // ARROW_LITTLE_ENDIAN } + namespace internal { void RegisterHashJoinNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("hashjoin", HashJoinNode::Make)); diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/compute/exec/partition_util.cc index e99007c45a335..90ff48ffa5b84 100644 --- a/cpp/src/arrow/compute/exec/partition_util.cc +++ b/cpp/src/arrow/compute/exec/partition_util.cc @@ -17,6 +17,7 @@ #include "arrow/compute/exec/partition_util.h" #include +#include namespace arrow { namespace compute { @@ -79,6 +80,19 @@ bool PartitionLocks::AcquirePartitionLock(size_t thread_id, int num_prtns_to_try return false; } +PartitionLocks::AutoReleaseLock PartitionLocks::AcquirePartitionLock(int prtn_id) +{ + std::atomic *lock = lock_ptr(prtn_id); + bool expected = false; + for(;;) + { + if(lock->compare_exchange_strong(expected, true, std::memory_order_acquire)) + return { this, prtn_id }; + while(lock->load()) + std::this_thread::yield(); + } +} + void PartitionLocks::ReleasePartitionLock(int prtn_id) { ARROW_DCHECK(prtn_id >= 0 && prtn_id < num_prtns_); 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 b3f302511a75d..007eb3944bb69 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/compute/exec/partition_util.h @@ -115,6 +115,18 @@ class PartitionLocks { 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); + + class AutoReleaseLock + { + public: + AutoReleaseLock(PartitionLocks* locks, int prtn_id) + : locks(locks), prtn_id(prtn_id) {} + ~AutoReleaseLock() { locks->ReleasePartitionLock(prtn_id); } + PartitionLocks* locks; + int prtn_id; + }; + + ARROW_MUST_USE_RESULT AutoReleaseLock AcquirePartitionLock(int prtn_id); /// \brief Release a partition so that other threads can work on it void ReleasePartitionLock(int prtn_id); @@ -147,14 +159,7 @@ class PartitionLocks { /*limit_retries=*/false, /*max_retries=*/-1, &locked_prtn_id, &locked_prtn_id_pos); { - class AutoReleaseLock { - public: - AutoReleaseLock(PartitionLocks* locks, int prtn_id) - : locks(locks), prtn_id(prtn_id) {} - ~AutoReleaseLock() { locks->ReleasePartitionLock(prtn_id); } - PartitionLocks* locks; - int prtn_id; - } auto_release_lock(this, locked_prtn_id); + AutoReleaseLock auto_release_lock(this, locked_prtn_id); ARROW_RETURN_NOT_OK(process_prtn_fn(locked_prtn_id)); } if (locked_prtn_id_pos < num_unprocessed_partitions - 1) { diff --git a/cpp/src/arrow/compute/exec/project_node.cc b/cpp/src/arrow/compute/exec/project_node.cc index 76925eb613992..f8bab0c15e399 100644 --- a/cpp/src/arrow/compute/exec/project_node.cc +++ b/cpp/src/arrow/compute/exec/project_node.cc @@ -65,7 +65,7 @@ class ProjectNode : public MapNode { for (auto& expr : exprs) { if (!expr.IsBound()) { ARROW_ASSIGN_OR_RAISE( - expr, expr.Bind(*inputs[0]->output_schema(), plan->exec_context())); + expr, expr.Bind(*inputs[0]->output_schema(), plan->query_context()->exec_context())); } fields[i] = field(std::move(names[i]), expr.type()->GetSharedPtr()); ++i; @@ -89,7 +89,7 @@ class ProjectNode : public MapNode { SimplifyWithGuarantee(exprs_[i], target.guarantee)); ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target, - plan()->exec_context())); + plan()->query_context()->exec_context())); } return ExecBatch{std::move(values), target.length}; } diff --git a/cpp/src/arrow/compute/exec/query_context.cc b/cpp/src/arrow/compute/exec/query_context.cc new file mode 100644 index 0000000000000..ebad7671c78a9 --- /dev/null +++ b/cpp/src/arrow/compute/exec/query_context.cc @@ -0,0 +1,118 @@ +// 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/query_context.h" +#include "arrow/util/cpu_info.h" +#include "arrow/util/io_util.h" + +namespace arrow +{ + using internal::CpuInfo; + namespace compute + { + QueryOptions::QueryOptions() + : max_memory_bytes(static_cast(0.75f * ::arrow::internal::GetTotalMemoryBytes())), + use_legacy_batching(false) + {} + + QueryContext::QueryContext( + QueryOptions opts, + ExecContext exec_context) + : options_(opts), + exec_context_(exec_context), + io_context_(exec_context_.memory_pool()) + { + } + + const CpuInfo* QueryContext::cpu_info() const { return CpuInfo::GetInstance(); } + + Status QueryContext::Init(size_t max_num_threads) + { + tld_.resize(max_num_threads); + return Status::OK(); + } + + size_t QueryContext::GetThreadIndex() { return thread_indexer_(); } + + size_t QueryContext::max_concurrency() const { return thread_indexer_.Capacity(); } + + Result QueryContext::GetTempStack(size_t thread_index) + { + if(!tld_[thread_index].is_init) + { + RETURN_NOT_OK(tld_[thread_index].stack.Init( + memory_pool(), + 8 * util::MiniBatch::kMiniBatchLength * sizeof(uint64_t))); + tld_[thread_index].is_init = true; + } + return &tld_[thread_index].stack; + } + + + Result> QueryContext::BeginExternalTask() + { + Future<> completion_future = Future<>::Make(); + ARROW_ASSIGN_OR_RAISE(bool task_added, + task_group_.AddTaskIfNotEnded(completion_future)); + if (task_added) { + return std::move(completion_future); + } + // Return an invalid future if we were already finished to signal to the + // caller that they should not begin the task + return Future<>{}; + } + + Status QueryContext::ScheduleTask(std::function fn) + { + ::arrow::internal::Executor *exec = executor(); + if (!exec) return fn(); + // Adds a task which submits fn to the executor and tracks its progress. If we're + // already stopping then the task is ignored and fn is not executed. + return task_group_ + .AddTaskIfNotEnded([exec, fn]() { return exec->Submit(std::move(fn)); }) + .status(); + } + + Status QueryContext::ScheduleTask(std::function fn) + { + std::function indexed_fn = [this, fn]() + { + size_t thread_index = GetThreadIndex(); + return fn(thread_index); + }; + return ScheduleTask(std::move(indexed_fn)); + } + + Status QueryContext::ScheduleIOTask(std::function fn) + { + return task_group_ + .AddTaskIfNotEnded([this, fn]() { return io_context_.executor()->Submit(std::move(fn)); }) + .status(); + } + + int QueryContext::RegisterTaskGroup(std::function task, + std::function on_finished) + { + return task_scheduler_->RegisterTaskGroup(std::move(task), std::move(on_finished)); + } + + Status QueryContext::StartTaskGroup(int task_group_id, int64_t num_tasks) + { + return task_scheduler_->StartTaskGroup(GetThreadIndex(), task_group_id, num_tasks); + } + } +} diff --git a/cpp/src/arrow/compute/exec/query_context.h b/cpp/src/arrow/compute/exec/query_context.h new file mode 100644 index 0000000000000..019c80317e4e6 --- /dev/null +++ b/cpp/src/arrow/compute/exec/query_context.h @@ -0,0 +1,178 @@ +// 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/io/interfaces.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/exec/task_util.h" + +#pragma once + +namespace arrow +{ + namespace internal + { + class CpuInfo; + } + + using io::IOContext; + namespace compute + { + struct ARROW_EXPORT QueryOptions + { + QueryOptions(); + // 0 means unlimited + size_t max_memory_bytes; + + /// \brief Should the plan use a legacy batching strategy + /// + /// This is currently in place only to support the Scanner::ToTable + /// method. This method relies on batch indices from the scanner + /// remaining consistent. This is impractical in the ExecPlan which + /// might slice batches as needed (e.g. for a join) + /// + /// However, it still works for simple plans and this is the only way + /// we have at the moment for maintaining implicit order. + bool use_legacy_batching; + }; + + class ARROW_EXPORT QueryContext + { + public: + QueryContext( + QueryOptions opts = {}, + ExecContext exec_context = *default_exec_context()); + + Status Init(size_t max_num_threads); + + const ::arrow::internal::CpuInfo *cpu_info() const; + const QueryOptions &options() const { return options_; } + MemoryPool *memory_pool() const { return exec_context_.memory_pool(); } + ::arrow::internal::Executor *executor() const { return exec_context_.executor(); } + ExecContext *exec_context() { return &exec_context_; } + IOContext *io_context() { return &io_context_; } + TaskScheduler *scheduler() { return task_scheduler_.get(); } + util::AsyncTaskGroup *task_group() { return &task_group_; } + + size_t GetThreadIndex(); + size_t max_concurrency() const; + Result GetTempStack(size_t thread_index); + + /// \brief Start an external task + /// + /// This should be avoided if possible. It is kept in for now for legacy + /// purposes. This should be called before the external task is started. If + /// a valid future is returned then it should be marked complete when the + /// external task has finished. + /// + /// \return an invalid future if the plan has already ended, otherwise this + /// returns a future that must be completed when the external task + /// finishes. + Result> BeginExternalTask(); + + /// \brief Add a single function as a task to the query's task group + /// on the compute threadpool. + /// + /// \param fn The task to run. Takes no arguments and returns a Status. + Status ScheduleTask(std::function fn); + /// \brief Add a single function as a task to the query's task group + /// on the compute threadpool. + /// + /// \param fn The task to run. Takes the thread index and returns a Status. + Status ScheduleTask(std::function fn); + /// \brief Add a single function as a task to the query's task group on + /// the IO thread pool + /// + /// \param fn The task to run. Returns a status. + Status ScheduleIOTask(std::function fn); + + // Register/Start TaskGroup is a way of performing a "Parallel For" pattern: + // - The task function takes the thread index and the index of the task + // - The on_finished function takes the thread index + // Returns an integer ID that will be used to reference the task group in + // StartTaskGroup. At runtime, call StartTaskGroup with the ID and the number of times + // you'd like the task to be executed. The need to register a task group before use will + // be removed after we rewrite the scheduler. + /// \brief Register a "parallel for" task group with the scheduler + /// + /// \param task The function implementing the task. Takes the thread_index and + /// the task index. + /// \param on_finished The function that gets run once all tasks have been completed. + /// Takes the thread_index. + /// + /// Must be called inside of ExecNode::Init. + int RegisterTaskGroup(std::function task, + std::function on_finished); + + /// \brief Start the task group with the specified ID. This can only + /// be called once per task_group_id. + /// + /// \param task_group_id The ID of the task group to run + /// \param num_tasks The number of times to run the task + Status StartTaskGroup(int task_group_id, int64_t num_tasks); + + struct TempFileIOMark + { + QueryContext *ctx_; + size_t bytes_; + + TempFileIOMark(QueryContext *ctx, size_t bytes) + : + ctx_(ctx), + bytes_(bytes) + { + ctx_->in_flight_bytes_to_disk_ += bytes_; + } + + ~TempFileIOMark() + { + ctx_->in_flight_bytes_to_disk_ -= bytes_; + } + }; + + TempFileIOMark ReportTempFileIO(size_t bytes) + { + return { this, bytes }; + } + + size_t GetCurrentTempFileIO() + { + return in_flight_bytes_to_disk_.load(); + } + + private: + QueryOptions options_; + // To be replaced with Acero-specific context once scheduler is done and + // we don't need ExecContext for kernels + ExecContext exec_context_; + IOContext io_context_; + + util::AsyncTaskGroup task_group_; + std::unique_ptr task_scheduler_ = TaskScheduler::Make(); + + ThreadIndexer thread_indexer_; + struct ThreadLocalData + { + bool is_init = false; + util::TempVectorStack stack; + }; + std::vector tld_; + + std::atomic in_flight_bytes_to_disk_{0}; + }; + } +} diff --git a/cpp/src/arrow/compute/exec/schema_util.h b/cpp/src/arrow/compute/exec/schema_util.h index f2b14aa545060..a80238cc1571d 100644 --- a/cpp/src/arrow/compute/exec/schema_util.h +++ b/cpp/src/arrow/compute/exec/schema_util.h @@ -21,6 +21,7 @@ #include #include #include +#include #include "arrow/compute/light_array.h" // for KeyColumnMetadata #include "arrow/type.h" // for DataType, FieldRef, Field and Schema @@ -38,7 +39,8 @@ enum class HashJoinProjection : int { KEY = 1, PAYLOAD = 2, FILTER = 3, - OUTPUT = 4 + OUTPUT = 4, + NUM_VALUES = 5, }; struct SchemaProjectionMap { @@ -63,22 +65,46 @@ class SchemaProjectionMaps { public: static constexpr int kMissingField = -1; - Status Init(ProjectionIdEnum full_schema_handle, const Schema& schema, - const std::vector& projection_handles, - const std::vector*>& projections) { - ARROW_DCHECK(projection_handles.size() == projections.size()); - ARROW_RETURN_NOT_OK(RegisterSchema(full_schema_handle, schema)); - for (size_t i = 0; i < projections.size(); ++i) { - ARROW_RETURN_NOT_OK( - RegisterProjectedSchema(projection_handles[i], *(projections[i]), schema)); + Status Init(ProjectionIdEnum full_schema_handle, + const Schema& schema) + { + RETURN_NOT_OK(RegisterSchema(full_schema_handle, schema)); + const int id_base = 0; + std::vector &mapping = mappings_[id_base]; + std::vector &inverse = inverse_mappings_[id_base]; + mapping.resize(schema.num_fields()); + inverse.resize(schema.num_fields()); + std::iota(mapping.begin(), mapping.end(), 0); + std::iota(inverse.begin(), inverse.end(), 0); + return Status::OK(); } - RegisterEnd(); + + Status RegisterProjectedSchema(ProjectionIdEnum handle, + const std::vector& selected_fields, + const Schema& full_schema) { + FieldInfos out_fields; + const FieldVector& in_fields = full_schema.fields(); + out_fields.field_paths.resize(selected_fields.size()); + out_fields.field_names.resize(selected_fields.size()); + out_fields.data_types.resize(selected_fields.size()); + for (size_t i = 0; i < selected_fields.size(); ++i) { + // All fields must be found in schema without ambiguity + ARROW_ASSIGN_OR_RAISE(auto match, selected_fields[i].FindOne(full_schema)); + const std::string& name = in_fields[match[0]]->name(); + const std::shared_ptr& type = in_fields[match[0]]->type(); + out_fields.field_paths[i] = match[0]; + out_fields.field_names[i] = name; + out_fields.data_types[i] = type; + } + int id = schema_id(handle); + schemas_[id] = std::move(out_fields); + GenerateMapForProjection(id); return Status::OK(); } int num_cols(ProjectionIdEnum schema_handle) const { int id = schema_id(schema_handle); - return static_cast(schemas_[id].second.data_types.size()); + return static_cast(schemas_[id].data_types.size()); } bool is_empty(ProjectionIdEnum schema_handle) const { @@ -87,19 +113,19 @@ class SchemaProjectionMaps { const std::string& field_name(ProjectionIdEnum schema_handle, int field_id) const { int id = schema_id(schema_handle); - return schemas_[id].second.field_names[field_id]; + return schemas_[id].field_names[field_id]; } const std::shared_ptr& data_type(ProjectionIdEnum schema_handle, int field_id) const { int id = schema_id(schema_handle); - return schemas_[id].second.data_types[field_id]; + return schemas_[id].data_types[field_id]; } const std::vector>& data_types( ProjectionIdEnum schema_handle) const { int id = schema_id(schema_handle); - return schemas_[id].second.data_types; + return schemas_[id].data_types; } SchemaProjectionMap map(ProjectionIdEnum from, ProjectionIdEnum to) const { @@ -132,55 +158,21 @@ class SchemaProjectionMaps { out_fields.field_names[i] = name; out_fields.data_types[i] = type; } - schemas_.push_back(std::make_pair(handle, out_fields)); - return Status::OK(); - } - - Status RegisterProjectedSchema(ProjectionIdEnum handle, - const std::vector& selected_fields, - const Schema& full_schema) { - FieldInfos out_fields; - const FieldVector& in_fields = full_schema.fields(); - out_fields.field_paths.resize(selected_fields.size()); - out_fields.field_names.resize(selected_fields.size()); - out_fields.data_types.resize(selected_fields.size()); - for (size_t i = 0; i < selected_fields.size(); ++i) { - // All fields must be found in schema without ambiguity - ARROW_ASSIGN_OR_RAISE(auto match, selected_fields[i].FindOne(full_schema)); - const std::string& name = in_fields[match[0]]->name(); - const std::shared_ptr& type = in_fields[match[0]]->type(); - out_fields.field_paths[i] = match[0]; - out_fields.field_names[i] = name; - out_fields.data_types[i] = type; - } - schemas_.push_back(std::make_pair(handle, out_fields)); + schemas_[schema_id(handle)] = std::move(out_fields); return Status::OK(); } - void RegisterEnd() { - size_t size = schemas_.size(); - mappings_.resize(size); - inverse_mappings_.resize(size); - int id_base = 0; - for (size_t i = 0; i < size; ++i) { - GenerateMapForProjection(static_cast(i), id_base); - } - } - int schema_id(ProjectionIdEnum schema_handle) const { - for (size_t i = 0; i < schemas_.size(); ++i) { - if (schemas_[i].first == schema_handle) { - return static_cast(i); - } - } - // We should never get here - ARROW_DCHECK(false); - return -1; + int id = static_cast(schema_handle); + ARROW_DCHECK(id < static_cast(ProjectionIdEnum::NUM_VALUES)); + return id; } - void GenerateMapForProjection(int id_proj, int id_base) { - int num_cols_proj = static_cast(schemas_[id_proj].second.data_types.size()); - int num_cols_base = static_cast(schemas_[id_base].second.data_types.size()); + void GenerateMapForProjection(int id_proj) { + const int id_base = 0; + + int num_cols_proj = static_cast(schemas_[id_proj].data_types.size()); + int num_cols_base = static_cast(schemas_[id_base].data_types.size()); std::vector& mapping = mappings_[id_proj]; std::vector& inverse_mapping = inverse_mappings_[id_proj]; @@ -192,8 +184,8 @@ class SchemaProjectionMaps { mapping[i] = inverse_mapping[i] = i; } } else { - const FieldInfos& fields_proj = schemas_[id_proj].second; - const FieldInfos& fields_base = schemas_[id_base].second; + const FieldInfos& fields_proj = schemas_[id_proj]; + const FieldInfos& fields_base = schemas_[id_base]; for (int i = 0; i < num_cols_base; ++i) { inverse_mapping[i] = SchemaProjectionMap::kMissingField; } @@ -215,9 +207,9 @@ class SchemaProjectionMaps { } // vector used as a mapping from ProjectionIdEnum to fields - std::vector> schemas_; - std::vector> mappings_; - std::vector> inverse_mappings_; + std::array(ProjectionIdEnum::NUM_VALUES)> schemas_; + std::array, static_cast(ProjectionIdEnum::NUM_VALUES)> mappings_; + std::array, static_cast(ProjectionIdEnum::NUM_VALUES)> inverse_mappings_; }; using HashJoinProjectionMaps = SchemaProjectionMaps; diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 8af4e8e996cce..dab8d54d27240 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -393,7 +393,7 @@ static Result MakeTableConsumingSinkNode( const compute::ExecNodeOptions& options) { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "TableConsumingSinkNode")); const auto& sink_options = checked_cast(options); - MemoryPool* pool = plan->exec_context()->memory_pool(); + MemoryPool* pool = plan->query_context()->memory_pool(); auto tb_consumer = std::make_shared(sink_options.output_table, pool); auto consuming_sink_node_options = ConsumingSinkNodeOptions{tb_consumer}; @@ -423,7 +423,7 @@ struct OrderBySinkNode final : public SinkNode { RETURN_NOT_OK(ValidateOrderByOptions(sink_options)); ARROW_ASSIGN_OR_RAISE( std::unique_ptr impl, - OrderByImpl::MakeSort(plan->exec_context(), inputs[0]->output_schema(), + OrderByImpl::MakeSort(plan->query_context()->exec_context(), inputs[0]->output_schema(), sink_options.sort_options)); return plan->EmplaceNode(plan, std::move(inputs), std::move(impl), sink_options.generator); @@ -455,7 +455,7 @@ struct OrderBySinkNode final : public SinkNode { RETURN_NOT_OK(ValidateSelectKOptions(sink_options)); ARROW_ASSIGN_OR_RAISE( std::unique_ptr impl, - OrderByImpl::MakeSelectK(plan->exec_context(), inputs[0]->output_schema(), + OrderByImpl::MakeSelectK(plan->query_context()->exec_context(), inputs[0]->output_schema(), sink_options.select_k_options)); return plan->EmplaceNode(plan, std::move(inputs), std::move(impl), sink_options.generator); @@ -478,7 +478,7 @@ struct OrderBySinkNode final : public SinkNode { DCHECK_EQ(input, inputs_[0]); auto maybe_batch = batch.ToRecordBatch(inputs_[0]->output_schema(), - plan()->exec_context()->memory_pool()); + plan()->query_context()->memory_pool()); if (ErrorIfNotOk(maybe_batch.status())) { StopProducing(); if (input_counter_.Cancel()) { diff --git a/cpp/src/arrow/compute/exec/source_node.cc b/cpp/src/arrow/compute/exec/source_node.cc index a640cf737efa7..5cac549627100 100644 --- a/cpp/src/arrow/compute/exec/source_node.cc +++ b/cpp/src/arrow/compute/exec/source_node.cc @@ -89,7 +89,7 @@ struct SourceNode : ExecNode { } CallbackOptions options; - auto executor = plan()->exec_context()->executor(); + auto executor = plan()->query_context()->executor(); if (executor) { // These options will transfer execution to the desired Executor if necessary. // This can happen for in-memory scans where batches didn't require @@ -98,7 +98,7 @@ struct SourceNode : ExecNode { options.executor = executor; options.should_schedule = ShouldSchedule::IfDifferentExecutor; } - ARROW_ASSIGN_OR_RAISE(Future<> scan_task, plan_->BeginExternalTask()); + ARROW_ASSIGN_OR_RAISE(Future<> scan_task, plan_->query_context()->BeginExternalTask()); if (!scan_task.is_valid()) { finished_.MarkFinished(); // Plan has already been aborted, no need to start scanning @@ -119,7 +119,7 @@ struct SourceNode : ExecNode { return Break(batch_count_); } lock.unlock(); - bool use_legacy_batching = plan_->UseLegacyBatching(); + bool use_legacy_batching = plan_->query_context()->options().use_legacy_batching; ExecBatch morsel = std::move(*maybe_morsel); int64_t morsel_length = static_cast(morsel.length); if (use_legacy_batching || morsel_length == 0) { @@ -131,7 +131,7 @@ struct SourceNode : ExecNode { bit_util::CeilDiv(morsel_length, ExecPlan::kMaxBatchSize)); batch_count_ += num_batches; } - RETURN_NOT_OK(plan_->ScheduleTask([=]() { + RETURN_NOT_OK(plan_->query_context()->ScheduleTask([=]() { int64_t offset = 0; do { int64_t batch_size = std::min( diff --git a/cpp/src/arrow/compute/exec/spilling_join.cc b/cpp/src/arrow/compute/exec/spilling_join.cc new file mode 100644 index 0000000000000..e4d7ce9806bfc --- /dev/null +++ b/cpp/src/arrow/compute/exec/spilling_join.cc @@ -0,0 +1,210 @@ +#include "arrow/compute/exec/spilling_join.h" +#include "arrow/util/atomic_util.h" + +namespace arrow +{ + namespace compute + { + Status SpillingHashJoin::Init( + QueryContext *ctx, + JoinType join_type, + size_t num_threads, + SchemaProjectionMaps *proj_map_left, + SchemaProjectionMaps *proj_map_right, + std::vector *key_cmp, + Expression *filter, + CallbackRecord callback_record, + bool is_swiss) + { + ctx_ = ctx; + num_threads_ = num_threads; + callbacks_ = std::move(callback_record); + 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); + }; + + 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_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(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(kFuzzFactor * ctx_->options().max_memory_bytes); + size_t spill_threshold = kFuzzFactor * max_memory - num_threads_ * max_batch_size; + size_t bytes_allocated = static_cast(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) + { + 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(); + + bool expected = false; + if(spilling_.compare_exchange_strong(expected, true)) + return callbacks_.start_spilling(thread_index); + 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)); + bool expected = false; + if(!build_or_probe_finished_.compare_exchange_strong(expected, true)) + return BeginNextCollocatedJoin(thread_index); + return Status::OK(); + } + + Status SpillingHashJoin::OnProbeSideBatch(size_t thread_index, ExecBatch batch) + { + return probe_accumulator_.InsertBatch( + thread_index, + std::move(batch)); + } + + Status SpillingHashJoin::OnProbeSideFinished(size_t thread_index) + { + bool expected = false; + if(!build_or_probe_finished_.compare_exchange_strong(expected, true)) + return BeginNextCollocatedJoin(thread_index); + return Status::OK(); + } + + 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( + thread_index, + partition_idx_, + [this](size_t thread_index, AccumulationQueue queue) + { + return OnProbeSideReady(thread_index, std::move(queue)); + })); + return Status::OK(); + } + + Status SpillingHashJoin::OnHashTableFinished(size_t thread_index) + { + bool expected = false; + if(!probe_or_hash_table_ready_.compare_exchange_strong(expected, true)) + return ProbePartition(thread_index); + + return Status::OK(); + } + + Status SpillingHashJoin::OnProbeSideReady(size_t thread_index, AccumulationQueue queue) + { + probe_side_partition_ = std::move(queue); + bool expected = false; + if(!probe_or_hash_table_ready_.compare_exchange_strong(expected, true)) + return ProbePartition(thread_index); + return Status::OK(); + } + + Status SpillingHashJoin::ProbePartition(size_t thread_index) + { + return ctx_->StartTaskGroup( + task_group_probe_[partition_idx_], + probe_side_partition_.batch_count()); + } + + Status SpillingHashJoin::ProbeAndFilterBatch(size_t thread_index, ExecBatch batch) + { + // TODO: Apply Bloom filter + RETURN_NOT_OK(impls_[partition_idx_]->ProbeSingleBatch(thread_index, std::move(batch))); + return Status::OK(); + } + + Status SpillingHashJoin::OnProbingFinished(size_t thread_index) + { + return impls_[partition_idx_]->ProbingFinished(thread_index); + } + + Status SpillingHashJoin::OnCollocatedJoinFinished(int64_t num_batches) + { + total_batches_outputted_ += num_batches; + if(partition_idx_ > 0) + return BeginNextCollocatedJoin(ctx_->GetThreadIndex()); + return callbacks_.finished(total_batches_outputted_); + } + } +} + + diff --git a/cpp/src/arrow/compute/exec/spilling_join.h b/cpp/src/arrow/compute/exec/spilling_join.h new file mode 100644 index 0000000000000..ffb9bb433b146 --- /dev/null +++ b/cpp/src/arrow/compute/exec/spilling_join.h @@ -0,0 +1,109 @@ +// 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. + +#pragma once + +#include + +#include "arrow/compute/exec/query_context.h" +#include "arrow/compute/exec/hash_join.h" +#include "arrow/compute/exec/accumulation_queue.h" + +namespace arrow +{ + namespace compute + { + class SpillingHashJoin + { + public: + using OutputBatchCallback = std::function; + using BuildFinishedCallback = std::function; + using FinishedCallback = std::function; + using RegisterTaskGroupCallback = std::function, std::function)>; + using StartTaskGroupCallback = std::function; + using StartSpillingCallback = std::function; + using PauseProbeSideCallback = std::function; + using ResumeProbeSideCallback = std::function; + using AbortContinuationImpl = std::function; + + struct CallbackRecord + { + RegisterTaskGroupCallback register_task_group; + StartTaskGroupCallback start_task_group; + OutputBatchCallback output_batch; + FinishedCallback finished; + StartSpillingCallback start_spilling; + PauseProbeSideCallback pause_probe_side; + ResumeProbeSideCallback resume_probe_side; + }; + + Status Init( + QueryContext *ctx, + JoinType join_type, + size_t num_threads, + SchemaProjectionMaps *proj_map_left, + SchemaProjectionMaps *proj_map_right, + std::vector *key_cmp, + Expression *filter, + CallbackRecord callback_record, + bool is_swiss); + + Status CheckSpilling(size_t thread_index, ExecBatch &batch); + + Status OnBuildSideBatch(size_t thread_index, ExecBatch batch); + Status OnBuildSideFinished(size_t thread_index); + + Status OnProbeSideBatch(size_t thread_index, ExecBatch batch); + Status OnProbeSideFinished(size_t thread_index); + + private: + Status AdvanceSpillCursor(size_t thread_index); + Status BeginNextCollocatedJoin(size_t thread_index); + Status BuildHashTable(size_t thread_index, AccumulationQueue queue); + Status OnHashTableFinished(size_t thread_index); + Status OnProbeSideReady(size_t thread_index, AccumulationQueue queue); + Status ProbePartition(size_t thread_index); + Status ProbeAndFilterBatch(size_t thread_index, ExecBatch batch); + Status OnProbingFinished(size_t thread_index); + Status OnCollocatedJoinFinished(int64_t num_batches); + + QueryContext *ctx_; + size_t num_threads_; + CallbackRecord callbacks_; + bool is_swiss_; + + std::atomic backpressure_counter_{0}; + + SpillingAccumulationQueue build_accumulator_; + SpillingAccumulationQueue probe_accumulator_; + + AccumulationQueue probe_side_partition_; + + std::unique_ptr impls_[SpillingAccumulationQueue::kNumPartitions]; + int task_group_probe_[SpillingAccumulationQueue::kNumPartitions]; + + std::atomic max_batch_size_{0}; + + int64_t total_batches_outputted_ = 0; + int partition_idx_ = SpillingAccumulationQueue::kNumPartitions; + std::atomic build_or_probe_finished_{false}; + std::atomic spilling_{false}; + std::atomic probe_or_hash_table_ready_{false}; + }; + } +} diff --git a/cpp/src/arrow/compute/exec/spilling_util.cc b/cpp/src/arrow/compute/exec/spilling_util.cc new file mode 100644 index 0000000000000..a62bb17749939 --- /dev/null +++ b/cpp/src/arrow/compute/exec/spilling_util.cc @@ -0,0 +1,432 @@ +// 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 "spilling_util.h" +#include + +namespace arrow +{ +namespace compute +{ + struct ArrayInfo + { + int64_t num_children; + int64_t length; + int64_t null_count; + std::shared_ptr type; + std::array, 3> bufs; + std::array sizes; + std::shared_ptr dictionary; + }; + + struct SpillFile::BatchInfo + { + int64_t start; + std::vector arrays; + }; + +#ifdef _WIN32 +#include "windows_compatibility.h" + +const FileHandle kInvalidHandle = INVALID_HANDLE_VALUE; + +static Result OpenTemporaryFile() +{ + constexpr DWORD kTempFileNameSize = MAX_PATH + 1; + wchar_t tmp_name_buf[kTempFileNameSize]; + wchar_t tmp_path_buf[kTempFileNameSize]; + + DWORD ret; + ret = GetTempPath2W(kTempFileNameSize, tmp_path_buf); + if(ret > kTempFileNameSize || ret == 0) + return Status::IOError(); + if(GetTempFileNameW(tmp_path_buf, L"ARROW_TMP", 0, tmp_name_buf) == 0) + return Status::IOError(); + + HANDLE file_handle = CreateFileA( + tmp_name_buf, + GENERIC_READ | GENERIC_WRITE | FILE_APPEND_DATA, + 0, + NULL, + CREATE_ALWAYS, + FILE_FLAG_NO_BUFFERING | FILE_FLAG_OVERLAPPED | FILE_FLAG_DELETE_ON_CLOSE, + NULL); + if(file_handle == INVALID_HANDLE_VALUE) + return Status::IOError("Failed to create temp file"); + return file_handle; +} + +static Status CloseTemporaryFile(FileHandle handle) +{ + if(!CloseHandle(handle)) + return Status::IOError("Failed to close temp file"); + return Status::OK(); +} + +static Status WriteBatch_PlatformSpecific(FileHandle handle, const SpillFile::BatchInfo &info) +{ + OVERLAPPED overlapped; + int64_t offset = info.start; + for(const ArrayInfo &arr : info.arrays) + { + for(size_t i = 0; i < arr.bufs.size(); i++) + { + if(info.bufs[i] != 0) + { + overlapped.Offset = static_cast(offset & static_cast(~0)); + overlapped.OffsetHigh = static_cast((offset >> 32) & static_cast(~0)); + if(!WriteFile( + handle, + info.bufs[i]->data(), + info.bufs[i]->size(), + NULL, + &overlapped)) + return Status::IOError("Failed to spill!"); + + offset += info.sizes[i]; + info.bufs[i].reset(); + } + } + } + return Status::OK(); +} + + +static Result> ReconstructArray( + size_t &idx, + std::vector &arrs, + size_t ¤t_offset, + MemoryPool *pool) +{ + ArrayInfo &arr = arrs[idx++]; + std::shared_ptr data = std::make_shared(); + data->type = std::move(arr.type); + data->length = arr.length; + data->null_count = arr.null_count; + data->dictionary = std::move(arr.dictionary); + + data->buffers.resize(3); + for(int i = 0; i < 3; i++) + { + if(arr.sizes[i]) + { + ARROW_ASSIGN_OR_RAISE(data->buffers[i], AllocateBuffer(arr.sizes[i], pool)); + + OVERLAPPED overlapped; + overlapped.Offset = static_cast(current_offset & static_cast(~0)); + overlapped.OffsetHigh = static_cast((current_offset >> 32) & static_cast(~0)); + if(!ReadFile( + handle, + static_cast(data->buffers[i]->mutable_data()), + arr.sizes[i], + NULL, + &overlapped)) + return Status::IOError("Failed to read back spilled data!"); + current_offset += arr.sizes[i]; + } + } + data->child_data.resize(arr.num_children); + for(int i = 0; i < arr.num_children; i++) + { + ARROW_ASSIGN_OR_RAISE(data->child_data[i], ReconstructArray(idx, arrs, ios, pool)); + } + + return data; +} + +static Result ReadBatch_PlatformSpecific( + FileHandle handle, + SpillFile::BatchInfo &info, + MemoryPool *pool) +{ + std::vector batch; + size_t offset = info.start; + // ReconstructArray increments i + for(size_t i = 0; i < info.arrays.size();) + { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr ad, ReconstructArray(i, info.arrays, offset, pool)); + batch.emplace_back(std::move(ad)); + } + return ExecBatch::Make(std::move(batch)); +} + +#else +#include +#include +#include +#include +#include + +Result OpenTemporaryFile() +{ + static std::once_flag generate_tmp_file_name_flag; + + constexpr int kFileNameSize = 1024; + static char name[kFileNameSize]; + + char *name_ptr = name; + std::call_once(generate_tmp_file_name_flag, [name_ptr]() noexcept + { + const char *selectors[] = { "TMPDIR", "TMP", "TEMP", "TEMPDIR" }; + constexpr size_t kNumSelectors = sizeof(selectors) / sizeof(selectors[0]); +#ifdef __ANDROID__ + const char *backup = "/data/local/tmp/"; +#else + const char *backup = "/tmp/"; +#endif + const char *tmp_dir = backup; + for(size_t i = 0; i < kNumSelectors; i++) + { + const char *env = getenv(selectors[i]); + if(env) + { + tmp_dir = env; + break; + } + } + size_t tmp_dir_length = std::strlen(tmp_dir); + + const char *tmp_name_template = "/ARROW_TMP_XXXXXX"; + size_t tmp_name_length = std::strlen(tmp_name_template); + + if((tmp_dir_length + tmp_name_length) >= kFileNameSize) + { + tmp_dir = backup; + tmp_dir_length = std::strlen(backup); + } + + std::strncpy(name_ptr, tmp_dir, kFileNameSize); + std::strncpy(name_ptr + tmp_dir_length, tmp_name_template, kFileNameSize - tmp_dir_length); + }); + +#ifdef __APPLE__ + int fd = mkstemp(name); + if(fd == kInvalidHandle) + return Status::IOError(strerror(errno)); + if(fcntl(fd, F_NOCACHE, 1) == -1) + return Status::IOError(strerror(errno)); +#else + int fd = mkostemp(name, O_DIRECT); + if(fd == kInvalidHandle) + return Status::IOError(strerror(errno)); +#endif + + if(unlink(name) != 0) + return Status::IOError(strerror(errno)); + return fd; +} + +static Status CloseTemporaryFile(FileHandle handle) +{ + if(close(handle) == -1) + return Status::IOError(strerror(errno)); + return Status::OK(); +} + +static Status WriteBatch_PlatformSpecific(FileHandle handle, SpillFile::BatchInfo &info) +{ + std::vector ios; + for(const ArrayInfo &arr : info.arrays) + { + for(int i = 0; i < 3; i++) + { + if(arr.bufs[i]) + { + struct iovec io; + io.iov_base = static_cast(arr.bufs[i]->mutable_data()); + io.iov_len = static_cast(arr.bufs[i]->size()); + ios.push_back(io); + } + } + } + + if(pwritev(handle, ios.data(), static_cast(ios.size()), info.start) == -1) + return Status::IOError("Failed to spill!"); + + // Release all references to the buffers, freeing them. + for(ArrayInfo &arr : info.arrays) + for(int i = 0; i < 3; i++) + if(arr.bufs[i]) + arr.bufs[i].reset(); + return Status::OK(); +} + +static Result> ReconstructArray( + size_t &idx, + std::vector &arrs, + std::vector &ios, + MemoryPool *pool) +{ + ArrayInfo &arr = arrs[idx++]; + std::shared_ptr data = std::make_shared(); + data->type = std::move(arr.type); + data->length = arr.length; + data->null_count = arr.null_count; + data->dictionary = std::move(arr.dictionary); + + data->buffers.resize(3); + for(int i = 0; i < 3; i++) + { + if(arr.sizes[i]) + { + ARROW_ASSIGN_OR_RAISE(data->buffers[i], AllocateBuffer(arr.sizes[i], pool)); + struct iovec io; + io.iov_base = static_cast(data->buffers[i]->mutable_data()); + io.iov_len = static_cast(arr.bufs[i]->size()); + ios.push_back(io); + } + } + data->child_data.resize(arr.num_children); + for(int i = 0; i < arr.num_children; i++) + { + ARROW_ASSIGN_OR_RAISE(data->child_data[i], ReconstructArray(idx, arrs, ios, pool)); + } + + return data; +} + +static Result ReadBatch_PlatformSpecific( + FileHandle handle, + SpillFile::BatchInfo &info, + MemoryPool *pool) +{ + std::vector batch; + std::vector ios; + // ReconstructArray increments i + for(size_t i = 0; i < info.arrays.size();) + { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr ad, ReconstructArray(i, info.arrays, ios, pool)); + batch.emplace_back(std::move(ad)); + } + + if(preadv(handle, ios.data(), static_cast(ios.size()), info.start) == -1) + return Status::IOError("Failed to read back spilled data!"); + + return ExecBatch::Make(std::move(batch)); +} +#endif + + SpillFile::~SpillFile() + { + for(BatchInfo *b : batches_) + delete b; + } + +static Status CollectArrayInfo( + SpillFile::BatchInfo &batch_info, + int64_t &total_size, + ArrayData *array) +{ + if(array->offset != 0) + return Status::Invalid("We don't support spilling arrays with offsets"); + + batch_info.arrays.push_back({}); + ArrayInfo &array_info = batch_info.arrays.back(); + array_info.type = std::move(array->type); + array_info.length = array->length; + array_info.null_count = array->null_count.load(std::memory_order_relaxed); + + ARROW_DCHECK(array->buffers.size() <= array_info.bufs.size()); + array_info.num_children = array->child_data.size(); + for(size_t i = 0; i < array->buffers.size(); i++) + { + if(array->buffers[i]) + { + array_info.sizes[i] = array->buffers[i]->size(); + total_size += array_info.sizes[i]; + uintptr_t addr = array->buffers[i]->address(); + if((addr % 512) != 0) + return Status::Invalid("Buffer not aligned to 512 bytes!"); + array_info.bufs[i] = std::move(array->buffers[i]); + } + } + + array_info.dictionary = std::move(array->dictionary); + for(std::shared_ptr &child : array->child_data) + RETURN_NOT_OK(CollectArrayInfo(batch_info, total_size, child.get())); + + // Cleanup the ArrayData + array->type.reset(); + array->length = 0; + return Status::OK(); +} + +Status SpillFile::SpillBatch(QueryContext *ctx, ExecBatch batch) +{ + if(handle_ == kInvalidHandle) + { + ARROW_ASSIGN_OR_RAISE(handle_, OpenTemporaryFile()); + } + int64_t total_size = 0; + batches_.emplace_back(new BatchInfo); + BatchInfo *info = batches_.back(); + for(int i = 0; i < batch.num_values(); i++) + { + if (batch[i].is_scalar()) + return Status::Invalid("Cannot spill a Scalar"); + RETURN_NOT_OK(CollectArrayInfo(*info, total_size, batch[i].mutable_array())); + } + info->start = size_; + size_ += total_size; + + FileHandle handle = handle_; + RETURN_NOT_OK(ctx->ScheduleIOTask( + [handle, info, ctx, total_size]() + { + auto mark = ctx->ReportTempFileIO(total_size); + return WriteBatch_PlatformSpecific(handle, *info); + })); + return Status::OK(); +} + +Status SpillFile::ReadBackBatches( + QueryContext *ctx, + std::function fn, + std::function on_finished) +{ + readback_fn_ = std::move(fn); + on_readback_finished_ = std::move(on_finished); + + size_t num_batches = batches_.size(); + for(size_t i = 0; i < num_batches; i++) + { + BatchInfo *info = batches_[i]; + RETURN_NOT_OK(ctx->ScheduleIOTask( + [i, info, ctx, this]() + { + ARROW_ASSIGN_OR_RAISE( + ExecBatch batch, + ReadBatch_PlatformSpecific(handle_, *info, ctx->memory_pool())); + RETURN_NOT_OK(readback_fn_(i, std::move(batch))); + + if(++readback_counter_ == batches_.size()) + RETURN_NOT_OK(ctx->ScheduleTask(std::move(on_readback_finished_))); + return Status::OK(); + })); + } + return Status::OK(); +} + + Status SpillFile::Cleanup() + { + if(handle_ != kInvalidHandle) + RETURN_NOT_OK(CloseTemporaryFile(handle_)); + return Status::OK(); + } + +} +} diff --git a/cpp/src/arrow/compute/exec/spilling_util.h b/cpp/src/arrow/compute/exec/spilling_util.h new file mode 100644 index 0000000000000..0cbc7f0c99d4d --- /dev/null +++ b/cpp/src/arrow/compute/exec/spilling_util.h @@ -0,0 +1,67 @@ +// 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. + +#pragma once + +#include "arrow/compute/exec/exec_plan.h" + +namespace arrow +{ + namespace compute + { +#ifdef _WIN32 + using FileHandle = void *; + extern const FileHandle kInvalidHandle; +#else + using FileHandle = int; + constexpr FileHandle kInvalidHandle = -1; +#endif + + class SpillFile + { + public: + + ~SpillFile(); + // To spill a batch the following must be true: + // - Row offset for each column must be 0. + // - Column buffers must be aligned to 512 bits + // - No column can be a scalar + // These assumptions aren't as inconvenient as it seems because + // typically batches will be partitioned before being spilled, + // meaning the batches will come from ExecBatchBuilder, which + // ensures these assumptions hold. + Status SpillBatch(QueryContext *ctx, ExecBatch batch); + Status ReadBackBatches( + QueryContext *ctx, + std::function fn, + std::function on_finished); + Status Cleanup(); + size_t num_batches() const { return batches_.size(); } + + struct BatchInfo; + private: + FileHandle handle_ = kInvalidHandle; + size_t size_ = 0; + + std::atomic readback_counter_{0}; + std::function readback_fn_; + std::function on_readback_finished_; + + std::vector batches_; + }; + } +} diff --git a/cpp/src/arrow/compute/exec/swiss_join.cc b/cpp/src/arrow/compute/exec/swiss_join.cc index 5b01edb1198c2..2b2c23a0c2961 100644 --- a/cpp/src/arrow/compute/exec/swiss_join.cc +++ b/cpp/src/arrow/compute/exec/swiss_join.cc @@ -2022,16 +2022,13 @@ Status JoinProbeProcessor::OnFinished() { class SwissJoin : public HashJoinImpl { public: - Status Init(ExecContext* ctx, JoinType join_type, size_t num_threads, + Status Init(QueryContext* ctx, JoinType join_type, size_t num_threads, const HashJoinProjectionMaps* proj_map_left, const HashJoinProjectionMaps* proj_map_right, - std::vector key_cmp, Expression filter, - RegisterTaskGroupCallback register_task_group_callback, - StartTaskGroupCallback start_task_group_callback, - OutputBatchCallback output_batch_callback, - FinishedCallback finished_callback) override { + std::vector *key_cmp, Expression *filter, + CallbackRecord callback_record) override { START_COMPUTE_SPAN(span_, "SwissJoinImpl", - {{"detail", filter.ToString()}, + {{"detail", filter->ToString()}, {"join.kind", arrow::compute::ToString(join_type)}, {"join.threads", static_cast(num_threads)}}); @@ -2041,18 +2038,12 @@ class SwissJoin : public HashJoinImpl { pool_ = ctx->memory_pool(); join_type_ = join_type; - key_cmp_.resize(key_cmp.size()); - for (size_t i = 0; i < key_cmp.size(); ++i) { - key_cmp_[i] = key_cmp[i]; - } + key_cmp_ = key_cmp; schema_[0] = proj_map_left; schema_[1] = proj_map_right; - register_task_group_callback_ = std::move(register_task_group_callback); - start_task_group_callback_ = std::move(start_task_group_callback); - output_batch_callback_ = std::move(output_batch_callback); - finished_callback_ = std::move(finished_callback); + callback_record_ = std::move(callback_record); hash_table_ready_.store(false); cancelled_.store(false); @@ -2067,8 +2058,6 @@ class SwissJoin : public HashJoinImpl { for (int i = 0; i < num_threads_; ++i) { local_states_[i].hash_table_ready = false; local_states_[i].num_output_batches = 0; - RETURN_NOT_OK(CancelIfNotOK(local_states_[i].temp_stack.Init( - pool_, 1024 + 64 * util::MiniBatch::kMiniBatchLength))); local_states_[i].materialize.Init(pool_, proj_map_left, proj_map_right); } @@ -2079,7 +2068,7 @@ class SwissJoin : public HashJoinImpl { } probe_processor_.Init(proj_map_left->num_cols(HashJoinProjection::KEY), join_type_, - &hash_table_, materialize, &key_cmp_, output_batch_callback_); + &hash_table_, materialize, key_cmp_, callback_record_.output_batch); InitTaskGroups(); @@ -2087,17 +2076,17 @@ class SwissJoin : public HashJoinImpl { } void InitTaskGroups() { - task_group_build_ = register_task_group_callback_( + task_group_build_ = callback_record_.register_task_group( [this](size_t thread_index, int64_t task_id) -> Status { return BuildTask(thread_index, task_id); }, [this](size_t thread_index) -> Status { return BuildFinished(thread_index); }); - task_group_merge_ = register_task_group_callback_( + task_group_merge_ = callback_record_.register_task_group( [this](size_t thread_index, int64_t task_id) -> Status { return MergeTask(thread_index, task_id); }, [this](size_t thread_index) -> Status { return MergeFinished(thread_index); }); - task_group_scan_ = register_task_group_callback_( + task_group_scan_ = callback_record_.register_task_group( [this](size_t thread_index, int64_t task_id) -> Status { return ScanTask(thread_index, task_id); }, @@ -2116,10 +2105,11 @@ class SwissJoin : public HashJoinImpl { ExecBatch keypayload_batch; ARROW_ASSIGN_OR_RAISE(keypayload_batch, KeyPayloadFromInput(/*side=*/0, &batch)); + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack* temp_stack, ctx_->GetTempStack(thread_index)); return CancelIfNotOK(probe_processor_.OnNextBatch( - thread_index, keypayload_batch, &local_states_[thread_index].temp_stack, - &local_states_[thread_index].temp_column_arrays)); + thread_index, keypayload_batch, temp_stack, + &local_states_[thread_index].temp_column_arrays)); } Status ProbingFinished(size_t thread_index) override { @@ -2183,7 +2173,7 @@ class SwissJoin : public HashJoinImpl { // Process all input batches // return CancelIfNotOK( - start_task_group_callback_(task_group_build_, build_side_batches_.batch_count())); + callback_record_.start_task_group(task_group_build_, build_side_batches_.batch_count())); } Status BuildTask(size_t thread_id, int64_t batch_id) { @@ -2225,9 +2215,10 @@ class SwissJoin : public HashJoinImpl { input_batch.values[schema->num_cols(HashJoinProjection::KEY) + icol]; } } + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack* temp_stack, ctx_->GetTempStack(thread_id)); RETURN_NOT_OK(CancelIfNotOK(hash_table_build_.PushNextBatch( static_cast(thread_id), key_batch, no_payload ? nullptr : &payload_batch, - &local_states_[thread_id].temp_stack))); + temp_stack))); // Release input batch // @@ -2246,7 +2237,7 @@ class SwissJoin : public HashJoinImpl { // RETURN_NOT_OK(CancelIfNotOK(hash_table_build_.PreparePrtnMerge())); return CancelIfNotOK( - start_task_group_callback_(task_group_merge_, hash_table_build_.num_prtns())); + callback_record_.start_task_group(task_group_merge_, hash_table_build_.num_prtns())); } Status MergeTask(size_t /*thread_id*/, int64_t prtn_id) { @@ -2259,7 +2250,8 @@ class SwissJoin : public HashJoinImpl { Status MergeFinished(size_t thread_id) { RETURN_NOT_OK(status()); - hash_table_build_.FinishPrtnMerge(&local_states_[thread_id].temp_stack); + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack* temp_stack, ctx_->GetTempStack(thread_id)); + hash_table_build_.FinishPrtnMerge(temp_stack); return CancelIfNotOK(OnBuildHashTableFinished(static_cast(thread_id))); } @@ -2291,7 +2283,7 @@ class SwissJoin : public HashJoinImpl { hash_table_.MergeHasMatch(); int64_t num_tasks = bit_util::CeilDiv(hash_table_.num_rows(), kNumRowsPerScanTask); - return CancelIfNotOK(start_task_group_callback_(task_group_scan_, num_tasks)); + return CancelIfNotOK(callback_record_.start_task_group(task_group_scan_, num_tasks)); } else { return CancelIfNotOK(OnScanHashTableFinished()); } @@ -2311,7 +2303,7 @@ class SwissJoin : public HashJoinImpl { std::min((task_id + 1) * kNumRowsPerScanTask, hash_table_.num_rows()); // Get thread index and related temp vector stack // - util::TempVectorStack* temp_stack = &local_states_[thread_id].temp_stack; + ARROW_ASSIGN_OR_RAISE(util::TempVectorStack* temp_stack, ctx_->GetTempStack(thread_id)); // Split into mini-batches // @@ -2363,7 +2355,7 @@ class SwissJoin : public HashJoinImpl { Status status = local_states_[thread_id].materialize.AppendBuildOnly( num_output_rows, key_ids_buf.mutable_data(), payload_ids_buf.mutable_data(), [&](ExecBatch batch) { - output_batch_callback_(static_cast(thread_id), std::move(batch)); + callback_record_.output_batch(static_cast(thread_id), std::move(batch)); }); RETURN_NOT_OK(CancelIfNotOK(status)); if (!status.ok()) { @@ -2401,9 +2393,7 @@ class SwissJoin : public HashJoinImpl { num_produced_batches += materialize.num_produced_batches(); } - finished_callback_(num_produced_batches); - - return Status::OK(); + return callback_record_.finished(num_produced_batches); } Result KeyPayloadFromInput(int side, ExecBatch* input) { @@ -2467,12 +2457,12 @@ class SwissJoin : public HashJoinImpl { static constexpr int kNumRowsPerScanTask = 512 * 1024; - ExecContext* ctx_; + QueryContext* ctx_; int64_t hardware_flags_; MemoryPool* pool_; int num_threads_; JoinType join_type_; - std::vector key_cmp_; + std::vector *key_cmp_; const HashJoinProjectionMaps* schema_[2]; // Task scheduling @@ -2481,15 +2471,11 @@ class SwissJoin : public HashJoinImpl { int task_group_scan_; // Callbacks - RegisterTaskGroupCallback register_task_group_callback_; - StartTaskGroupCallback start_task_group_callback_; - OutputBatchCallback output_batch_callback_; + CallbackRecord callback_record_; BuildFinishedCallback build_finished_callback_; - FinishedCallback finished_callback_; struct ThreadLocalState { JoinResultMaterialize materialize; - util::TempVectorStack temp_stack; std::vector temp_column_arrays; int64_t num_output_batches; bool hash_table_ready; diff --git a/cpp/src/arrow/compute/exec/tpch_node.cc b/cpp/src/arrow/compute/exec/tpch_node.cc index d19f20eea7ca4..84e84858ad2b6 100644 --- a/cpp/src/arrow/compute/exec/tpch_node.cc +++ b/cpp/src/arrow/compute/exec/tpch_node.cc @@ -3375,7 +3375,7 @@ class TpchNode : public ExecNode { Status StartProducing() override { return generator_->StartProducing( - plan_->max_concurrency(), + plan_->query_context()->max_concurrency(), [this](ExecBatch batch) { this->OutputBatchCallback(std::move(batch)); }, [this](int64_t num_batches) { this->FinishedCallback(num_batches); }, [this](std::function func) -> Status { @@ -3413,7 +3413,7 @@ class TpchNode : public ExecNode { Status ScheduleTaskCallback(std::function func) { if (finished_.is_finished()) return Status::OK(); - return plan_->ScheduleTask([this, func](size_t thread_index) { + return plan_->query_context()->ScheduleTask([this, func](size_t thread_index) { Status status = func(thread_index); if (!status.ok()) { StopProducing(); diff --git a/cpp/src/arrow/compute/light_array.cc b/cpp/src/arrow/compute/light_array.cc index 4bf3574d09fdb..51c8bca8452e2 100644 --- a/cpp/src/arrow/compute/light_array.cc +++ b/cpp/src/arrow/compute/light_array.cc @@ -187,7 +187,8 @@ Status ColumnArraysFromExecBatch(const ExecBatch& batch, } void ResizableArrayData::Init(const std::shared_ptr& data_type, - MemoryPool* pool, int log_num_rows_min) { + MemoryPool* pool, int log_num_rows_min, + int64_t alignment) { #ifndef NDEBUG if (num_rows_allocated_ > 0) { ARROW_DCHECK(data_type_ != NULLPTR); @@ -200,6 +201,7 @@ void ResizableArrayData::Init(const std::shared_ptr& data_type, #endif Clear(/*release_buffers=*/false); log_num_rows_min_ = log_num_rows_min; + alignment_ = alignment; data_type_ = data_type; pool_ = pool; } @@ -236,7 +238,7 @@ Status ResizableArrayData::ResizeFixedLengthBuffers(int num_rows_new) { ARROW_ASSIGN_OR_RAISE( buffers_[kValidityBuffer], AllocateResizableBuffer( - bit_util::BytesForBits(num_rows_allocated_new) + kNumPaddingBytes, pool_)); + bit_util::BytesForBits(num_rows_allocated_new) + kNumPaddingBytes, alignment_, pool_)); memset(mutable_data(kValidityBuffer), 0, bit_util::BytesForBits(num_rows_allocated_new) + kNumPaddingBytes); if (column_metadata.is_fixed_length) { @@ -245,6 +247,7 @@ Status ResizableArrayData::ResizeFixedLengthBuffers(int num_rows_new) { buffers_[kFixedLengthBuffer], AllocateResizableBuffer( bit_util::BytesForBits(num_rows_allocated_new) + kNumPaddingBytes, + alignment_, pool_)); memset(mutable_data(kFixedLengthBuffer), 0, bit_util::BytesForBits(num_rows_allocated_new) + kNumPaddingBytes); @@ -253,18 +256,19 @@ Status ResizableArrayData::ResizeFixedLengthBuffers(int num_rows_new) { buffers_[kFixedLengthBuffer], AllocateResizableBuffer( num_rows_allocated_new * column_metadata.fixed_length + kNumPaddingBytes, + alignment_, pool_)); } } else { ARROW_ASSIGN_OR_RAISE( buffers_[kFixedLengthBuffer], AllocateResizableBuffer( - (num_rows_allocated_new + 1) * sizeof(uint32_t) + kNumPaddingBytes, pool_)); + (num_rows_allocated_new + 1) * sizeof(uint32_t) + kNumPaddingBytes, alignment_, pool_)); } ARROW_ASSIGN_OR_RAISE( buffers_[kVariableLengthBuffer], - AllocateResizableBuffer(sizeof(uint64_t) + kNumPaddingBytes, pool_)); + AllocateResizableBuffer(sizeof(uint64_t) + kNumPaddingBytes, alignment_, pool_)); var_len_buf_size_ = sizeof(uint64_t); } else { @@ -478,7 +482,7 @@ Status ExecBatchBuilder::AppendSelected(const std::shared_ptr& source ARROW_DCHECK(num_rows_before >= 0); int num_rows_after = num_rows_before + num_rows_to_append; if (target->num_rows() == 0) { - target->Init(source->type, pool, kLogNumRows); + target->Init(source->type, pool, kLogNumRows, kAlignment); } RETURN_NOT_OK(target->ResizeFixedLengthBuffers(num_rows_after)); @@ -625,7 +629,7 @@ Status ExecBatchBuilder::AppendNulls(const std::shared_ptr& type, int num_rows_before = target.num_rows(); int num_rows_after = num_rows_before + num_rows_to_append; if (target.num_rows() == 0) { - target.Init(type, pool, kLogNumRows); + target.Init(type, pool, kLogNumRows, kAlignment); } RETURN_NOT_OK(target.ResizeFixedLengthBuffers(num_rows_after)); @@ -686,7 +690,7 @@ Status ExecBatchBuilder::AppendSelected(MemoryPool* pool, const ExecBatch& batch const Datum& data = batch.values[col_ids ? col_ids[i] : i]; ARROW_DCHECK(data.is_array()); const std::shared_ptr& array_data = data.array(); - values_[i].Init(array_data->type, pool, kLogNumRows); + values_[i].Init(array_data->type, pool, kLogNumRows, kAlignment); } } @@ -717,7 +721,7 @@ Status ExecBatchBuilder::AppendNulls(MemoryPool* pool, if (values_.empty()) { values_.resize(types.size()); for (size_t i = 0; i < types.size(); ++i) { - values_[i].Init(types[i], pool, kLogNumRows); + values_[i].Init(types[i], pool, kLogNumRows, kAlignment); } } diff --git a/cpp/src/arrow/compute/light_array.h b/cpp/src/arrow/compute/light_array.h index f0e5c7068716a..ebe41215caa66 100644 --- a/cpp/src/arrow/compute/light_array.h +++ b/cpp/src/arrow/compute/light_array.h @@ -241,7 +241,7 @@ class ARROW_EXPORT ResizableArrayData { /// \param log_num_rows_min All resize operations will allocate at least enough /// space for (1 << log_num_rows_min) rows void Init(const std::shared_ptr& data_type, MemoryPool* pool, - int log_num_rows_min); + int log_num_rows_min, int64_t alignment = MemoryPool::kDefaultAlignment); /// \brief Resets the array back to an empty state /// \param release_buffers If true then allocated memory is released and the @@ -296,6 +296,7 @@ class ARROW_EXPORT ResizableArrayData { private: static constexpr int64_t kNumPaddingBytes = 64; int log_num_rows_min_; + int64_t alignment_; std::shared_ptr data_type_; MemoryPool* pool_; int num_rows_; @@ -348,11 +349,13 @@ class ARROW_EXPORT ExecBatchBuilder { ExecBatch Flush(); int num_rows() const { return values_.empty() ? 0 : values_[0].num_rows(); } + bool is_full() const { return num_rows() == num_rows_max(); } static int num_rows_max() { return 1 << kLogNumRows; } private: static constexpr int kLogNumRows = 15; + static constexpr int64_t kAlignment = 512; // Calculate how many rows to skip from the tail of the // sequence of selected rows, such that the total size of skipped rows is at diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index 0ef1d4577cd30..333d03fbc529c 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -361,8 +361,10 @@ Result AsyncScanner::ScanBatchesUnorderedAsync( auto exec_context = std::make_shared(scan_options_->pool, cpu_executor); - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(exec_context.get())); - plan->SetUseLegacyBatching(use_legacy_batching); + compute::QueryOptions query_options; + query_options.use_legacy_batching = true; + + ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(query_options, exec_context.get())); AsyncGenerator> sink_gen; auto exprs = scan_options_->projection.call()->arguments; diff --git a/cpp/src/arrow/datum.cc b/cpp/src/arrow/datum.cc index f06e97a20ec4b..86dada2185907 100644 --- a/cpp/src/arrow/datum.cc +++ b/cpp/src/arrow/datum.cc @@ -126,6 +126,7 @@ int64_t Datum::TotalBufferSize() const { case Datum::TABLE: return util::TotalBufferSize(*util::get>(this->value)); case Datum::SCALAR: + case Datum::NONE: return 0; default: DCHECK(false); diff --git a/cpp/src/arrow/io/file_test.cc b/cpp/src/arrow/io/file_test.cc index 8165c9c0b49c4..66a6144c08185 100644 --- a/cpp/src/arrow/io/file_test.cc +++ b/cpp/src/arrow/io/file_test.cc @@ -423,15 +423,15 @@ class MyMemoryPool : public MemoryPool { public: MyMemoryPool() : num_allocations_(0) {} - Status Allocate(int64_t size, uint8_t** out) override { + Status Allocate(int64_t size, int64_t /*alignment*/, uint8_t** out) override { *out = reinterpret_cast(std::malloc(size)); ++num_allocations_; return Status::OK(); } - void Free(uint8_t* buffer, int64_t size) override { std::free(buffer); } + void Free(uint8_t* buffer, int64_t size, int64_t /*alignment*/) override { std::free(buffer); } - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override { + Status Reallocate(int64_t old_size, int64_t new_size, int64_t /*alignment*/, uint8_t** ptr) override { *ptr = reinterpret_cast(std::realloc(*ptr, new_size)); if (*ptr == NULL) { diff --git a/cpp/src/arrow/memory_pool.cc b/cpp/src/arrow/memory_pool.cc index f8682ad313800..e5d1f6e2cda29 100644 --- a/cpp/src/arrow/memory_pool.cc +++ b/cpp/src/arrow/memory_pool.cc @@ -59,7 +59,7 @@ namespace memory_pool { namespace internal { -alignas(kAlignment) int64_t zero_size_area[1] = {kDebugXorSuffix}; +alignas(MemoryPool::kDefaultAlignment) int64_t zero_size_area[1] = {kDebugXorSuffix}; } // namespace internal @@ -224,12 +224,12 @@ bool IsDebugEnabled() { template class DebugAllocator { public: - static Status AllocateAligned(int64_t size, uint8_t** out) { + static Status AllocateAligned(int64_t size, int64_t alignment, uint8_t** out) { if (size == 0) { *out = memory_pool::internal::kZeroSizeArea; } else { ARROW_ASSIGN_OR_RAISE(int64_t raw_size, RawSize(size)); - RETURN_NOT_OK(WrappedAllocator::AllocateAligned(raw_size, out)); + RETURN_NOT_OK(WrappedAllocator::AllocateAligned(raw_size, alignment, out)); InitAllocatedArea(*out, size); } return Status::OK(); @@ -237,29 +237,29 @@ class DebugAllocator { static void ReleaseUnused() { WrappedAllocator::ReleaseUnused(); } - static Status ReallocateAligned(int64_t old_size, int64_t new_size, uint8_t** ptr) { + static Status ReallocateAligned(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { CheckAllocatedArea(*ptr, old_size, "reallocation"); if (*ptr == memory_pool::internal::kZeroSizeArea) { - return AllocateAligned(new_size, ptr); + return AllocateAligned(new_size, alignment, ptr); } if (new_size == 0) { // Note that an overflow check isn't needed as `old_size` is supposed to have // been successfully passed to AllocateAligned() before. - WrappedAllocator::DeallocateAligned(*ptr, old_size + kOverhead); + WrappedAllocator::DeallocateAligned(*ptr, old_size + kOverhead, alignment); *ptr = memory_pool::internal::kZeroSizeArea; return Status::OK(); } ARROW_ASSIGN_OR_RAISE(int64_t raw_new_size, RawSize(new_size)); RETURN_NOT_OK( - WrappedAllocator::ReallocateAligned(old_size + kOverhead, raw_new_size, ptr)); + WrappedAllocator::ReallocateAligned(old_size + kOverhead, raw_new_size, alignment, ptr)); InitAllocatedArea(*ptr, new_size); return Status::OK(); } - static void DeallocateAligned(uint8_t* ptr, int64_t size) { + static void DeallocateAligned(uint8_t* ptr, int64_t size, int64_t alignment) { CheckAllocatedArea(ptr, size, "deallocation"); if (ptr != memory_pool::internal::kZeroSizeArea) { - WrappedAllocator::DeallocateAligned(ptr, size + kOverhead); + WrappedAllocator::DeallocateAligned(ptr, size + kOverhead, alignment); } } @@ -295,7 +295,7 @@ class SystemAllocator { public: // Allocate memory according to the alignment requirements for Arrow // (as of May 2016 64 bytes) - static Status AllocateAligned(int64_t size, uint8_t** out) { + static Status AllocateAligned(int64_t size, int64_t alignment, uint8_t** out) { if (size == 0) { *out = memory_pool::internal::kZeroSizeArea; return Status::OK(); @@ -303,19 +303,19 @@ class SystemAllocator { #ifdef _WIN32 // Special code path for Windows *out = reinterpret_cast( - _aligned_malloc(static_cast(size), memory_pool::internal::kAlignment)); + _aligned_malloc(static_cast(size), static_cast(alignment))); if (!*out) { return Status::OutOfMemory("malloc of size ", size, " failed"); } #elif defined(sun) || defined(__sun) *out = reinterpret_cast( - memalign(memory_pool::internal::kAlignment, static_cast(size))); + memalign(static_cast(alignment), static_cast(size))); if (!*out) { return Status::OutOfMemory("malloc of size ", size, " failed"); } #else const int result = - posix_memalign(reinterpret_cast(out), memory_pool::internal::kAlignment, + posix_memalign(reinterpret_cast(out), static_cast(alignment), static_cast(size)); if (result == ENOMEM) { return Status::OutOfMemory("malloc of size ", size, " failed"); @@ -323,20 +323,24 @@ class SystemAllocator { if (result == EINVAL) { return Status::Invalid("invalid alignment parameter: ", - memory_pool::internal::kAlignment); + static_cast(alignment)); } #endif return Status::OK(); } - static Status ReallocateAligned(int64_t old_size, int64_t new_size, uint8_t** ptr) { + static Status ReallocateAligned( + int64_t old_size, + int64_t new_size, + int64_t alignment, + uint8_t** ptr) { uint8_t* previous_ptr = *ptr; if (previous_ptr == memory_pool::internal::kZeroSizeArea) { DCHECK_EQ(old_size, 0); - return AllocateAligned(new_size, ptr); + return AllocateAligned(new_size, alignment, ptr); } if (new_size == 0) { - DeallocateAligned(previous_ptr, old_size); + DeallocateAligned(previous_ptr, old_size, alignment); *ptr = memory_pool::internal::kZeroSizeArea; return Status::OK(); } @@ -344,7 +348,7 @@ class SystemAllocator { // Allocate new chunk uint8_t* out = nullptr; - RETURN_NOT_OK(AllocateAligned(new_size, &out)); + RETURN_NOT_OK(AllocateAligned(new_size, alignment, &out)); DCHECK(out); // Copy contents and release old memory chunk memcpy(out, *ptr, static_cast(std::min(new_size, old_size))); @@ -357,7 +361,7 @@ class SystemAllocator { return Status::OK(); } - static void DeallocateAligned(uint8_t* ptr, int64_t size) { + static void DeallocateAligned(uint8_t* ptr, int64_t size, int64_t /*alignment*/) { if (ptr == memory_pool::internal::kZeroSizeArea) { DCHECK_EQ(size, 0); } else { @@ -383,13 +387,13 @@ class SystemAllocator { // Helper class directing allocations to the mimalloc allocator. class MimallocAllocator { public: - static Status AllocateAligned(int64_t size, uint8_t** out) { + static Status AllocateAligned(int64_t size, int64_t alignment, uint8_t** out) { if (size == 0) { *out = memory_pool::internal::kZeroSizeArea; return Status::OK(); } *out = reinterpret_cast( - mi_malloc_aligned(static_cast(size), memory_pool::internal::kAlignment)); + mi_malloc_aligned(static_cast(size), static_cast(alignment))); if (*out == NULL) { return Status::OutOfMemory("malloc of size ", size, " failed"); } @@ -398,19 +402,19 @@ class MimallocAllocator { static void ReleaseUnused() { mi_collect(true); } - static Status ReallocateAligned(int64_t old_size, int64_t new_size, uint8_t** ptr) { + static Status ReallocateAligned(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { uint8_t* previous_ptr = *ptr; if (previous_ptr == memory_pool::internal::kZeroSizeArea) { DCHECK_EQ(old_size, 0); - return AllocateAligned(new_size, ptr); + return AllocateAligned(new_size, ptr, alignment); } if (new_size == 0) { - DeallocateAligned(previous_ptr, old_size); + DeallocateAligned(previous_ptr, old_size, ); *ptr = memory_pool::internal::kZeroSizeArea; return Status::OK(); } *ptr = reinterpret_cast(mi_realloc_aligned( - previous_ptr, static_cast(new_size), memory_pool::internal::kAlignment)); + previous_ptr, static_cast(new_size), alignment)); if (*ptr == NULL) { *ptr = previous_ptr; return Status::OutOfMemory("realloc of size ", new_size, " failed"); @@ -418,7 +422,7 @@ class MimallocAllocator { return Status::OK(); } - static void DeallocateAligned(uint8_t* ptr, int64_t size) { + static void DeallocateAligned(uint8_t* ptr, int64_t size, int64_t /*alignment*/) { if (ptr == memory_pool::internal::kZeroSizeArea) { DCHECK_EQ(size, 0); } else { @@ -448,14 +452,14 @@ class BaseMemoryPoolImpl : public MemoryPool { public: ~BaseMemoryPoolImpl() override {} - Status Allocate(int64_t size, uint8_t** out) override { + Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override { if (size < 0) { return Status::Invalid("negative malloc size"); } if (static_cast(size) >= std::numeric_limits::max()) { return Status::OutOfMemory("malloc size overflows size_t"); } - RETURN_NOT_OK(Allocator::AllocateAligned(size, out)); + RETURN_NOT_OK(Allocator::AllocateAligned(size, alignment, out)); #ifndef NDEBUG // Poison data if (size > 0) { @@ -469,14 +473,14 @@ class BaseMemoryPoolImpl : public MemoryPool { return Status::OK(); } - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override { + Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) override { if (new_size < 0) { return Status::Invalid("negative realloc size"); } if (static_cast(new_size) >= std::numeric_limits::max()) { return Status::OutOfMemory("realloc overflows size_t"); } - RETURN_NOT_OK(Allocator::ReallocateAligned(old_size, new_size, ptr)); + RETURN_NOT_OK(Allocator::ReallocateAligned(old_size, new_size, alignment, ptr)); #ifndef NDEBUG // Poison data if (new_size > old_size) { @@ -490,7 +494,7 @@ class BaseMemoryPoolImpl : public MemoryPool { return Status::OK(); } - void Free(uint8_t* buffer, int64_t size) override { + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override { #ifndef NDEBUG // Poison data if (size > 0) { @@ -499,7 +503,7 @@ class BaseMemoryPoolImpl : public MemoryPool { buffer[size - 1] = kDeallocPoison; } #endif - Allocator::DeallocateAligned(buffer, size); + Allocator::DeallocateAligned(buffer, size, alignment); stats_.UpdateAllocatedBytes(-size); } @@ -672,22 +676,24 @@ Status jemalloc_set_decay_ms(int ms) { LoggingMemoryPool::LoggingMemoryPool(MemoryPool* pool) : pool_(pool) {} -Status LoggingMemoryPool::Allocate(int64_t size, uint8_t** out) { - Status s = pool_->Allocate(size, out); - std::cout << "Allocate: size = " << size << std::endl; +Status LoggingMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { + Status s = pool_->Allocate(size, alignment, out); + std::cout << "Allocate: size = " << size << ", alignment = " << alignment << std::endl; return s; } -Status LoggingMemoryPool::Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) { +Status LoggingMemoryPool::Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { Status s = pool_->Reallocate(old_size, new_size, ptr); - std::cout << "Reallocate: old_size = " << old_size << " - new_size = " << new_size + std::cout << "Reallocate: old_size = " << old_size + << ", new_size = " << new_size + << ", alignment = " << alignment << std::endl; return s; } -void LoggingMemoryPool::Free(uint8_t* buffer, int64_t size) { - pool_->Free(buffer, size); - std::cout << "Free: size = " << size << std::endl; +void LoggingMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { + pool_->Free(buffer, size, alignment); + std::cout << "Free: size = " << size << ", alignment = " << alignment << std::endl; } int64_t LoggingMemoryPool::bytes_allocated() const { @@ -711,20 +717,20 @@ class ProxyMemoryPool::ProxyMemoryPoolImpl { public: explicit ProxyMemoryPoolImpl(MemoryPool* pool) : pool_(pool) {} - Status Allocate(int64_t size, uint8_t** out) { - RETURN_NOT_OK(pool_->Allocate(size, out)); + Status Allocate(int64_t size, int64_t alignment, uint8_t** out) { + RETURN_NOT_OK(pool_->Allocate(size, alignment, out)); stats_.UpdateAllocatedBytes(size); return Status::OK(); } - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) { - RETURN_NOT_OK(pool_->Reallocate(old_size, new_size, ptr)); + Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { + RETURN_NOT_OK(pool_->Reallocate(old_size, new_size, alignment, ptr)); stats_.UpdateAllocatedBytes(new_size - old_size); return Status::OK(); } - void Free(uint8_t* buffer, int64_t size) { - pool_->Free(buffer, size); + void Free(uint8_t* buffer, int64_t size, int64_t alignment) { + pool_->Free(buffer, size, alignment); stats_.UpdateAllocatedBytes(-size); } @@ -745,16 +751,16 @@ ProxyMemoryPool::ProxyMemoryPool(MemoryPool* pool) { ProxyMemoryPool::~ProxyMemoryPool() {} -Status ProxyMemoryPool::Allocate(int64_t size, uint8_t** out) { - return impl_->Allocate(size, out); +Status ProxyMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { + return impl_->Allocate(size, alignment, out); } -Status ProxyMemoryPool::Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) { - return impl_->Reallocate(old_size, new_size, ptr); +Status ProxyMemoryPool::Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { + return impl_->Reallocate(old_size, new_size, alignment, ptr); } -void ProxyMemoryPool::Free(uint8_t* buffer, int64_t size) { - return impl_->Free(buffer, size); +void ProxyMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { + return impl_->Free(buffer, size, alignment); } int64_t ProxyMemoryPool::bytes_allocated() const { return impl_->bytes_allocated(); } @@ -777,8 +783,8 @@ std::vector SupportedMemoryBackendNames() { /// A Buffer whose lifetime is tied to a particular MemoryPool class PoolBuffer final : public ResizableBuffer { public: - explicit PoolBuffer(std::shared_ptr mm, MemoryPool* pool) - : ResizableBuffer(nullptr, 0, std::move(mm)), pool_(pool) {} + explicit PoolBuffer(std::shared_ptr mm, MemoryPool* pool, int64_t alignment) + : ResizableBuffer(nullptr, 0, std::move(mm)), pool_(pool), alignment_(alignment) {} ~PoolBuffer() override { // Avoid calling pool_->Free if the global pools are destroyed @@ -789,7 +795,7 @@ class PoolBuffer final : public ResizableBuffer { // no guarantee of destructor order between thread/memory pools) uint8_t* ptr = mutable_data(); if (ptr && !global_state.is_finalizing()) { - pool_->Free(ptr, capacity_); + pool_->Free(ptr, capacity_, alignment_); } } @@ -801,9 +807,9 @@ class PoolBuffer final : public ResizableBuffer { if (!ptr || capacity > capacity_) { int64_t new_capacity = bit_util::RoundUpToMultipleOf64(capacity); if (ptr) { - RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &ptr)); + RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, alignment_, &ptr)); } else { - RETURN_NOT_OK(pool_->Allocate(new_capacity, &ptr)); + RETURN_NOT_OK(pool_->Allocate(new_capacity, alignment_, &ptr)); } data_ = ptr; capacity_ = new_capacity; @@ -822,7 +828,7 @@ class PoolBuffer final : public ResizableBuffer { int64_t new_capacity = bit_util::RoundUpToMultipleOf64(new_size); if (capacity_ != new_capacity) { // Buffer hasn't got yet the requested size. - RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, &ptr)); + RETURN_NOT_OK(pool_->Reallocate(capacity_, new_capacity, alignment_, &ptr)); data_ = ptr; capacity_ = new_capacity; } @@ -834,7 +840,7 @@ class PoolBuffer final : public ResizableBuffer { return Status::OK(); } - static std::shared_ptr MakeShared(MemoryPool* pool) { + static std::shared_ptr MakeShared(MemoryPool* pool, int64_t alignment) { std::shared_ptr mm; if (pool == nullptr) { pool = default_memory_pool(); @@ -842,10 +848,10 @@ class PoolBuffer final : public ResizableBuffer { } else { mm = CPUDevice::memory_manager(pool); } - return std::make_shared(std::move(mm), pool); + return std::make_shared(std::move(mm), pool, alignment); } - static std::unique_ptr MakeUnique(MemoryPool* pool) { + static std::unique_ptr MakeUnique(MemoryPool* pool, int64_t alignment) { std::shared_ptr mm; if (pool == nullptr) { pool = default_memory_pool(); @@ -853,11 +859,12 @@ class PoolBuffer final : public ResizableBuffer { } else { mm = CPUDevice::memory_manager(pool); } - return std::unique_ptr(new PoolBuffer(std::move(mm), pool)); + return std::unique_ptr(new PoolBuffer(std::move(mm), pool, alignment)); } private: MemoryPool* pool_; + int64_t alignment_; }; namespace { @@ -873,14 +880,40 @@ inline Result ResizePoolBuffer(PoolBufferPtr&& buffer, const int64_t } // namespace -Result> AllocateBuffer(const int64_t size, MemoryPool* pool) { - return ResizePoolBuffer>(PoolBuffer::MakeUnique(pool), size); + Result> AllocateBuffer(const int64_t size, MemoryPool *pool) + { + return AllocateBuffer(size, MemoryPool::kDefaultAlignment, pool); + } + +Result> AllocateBuffer( + const int64_t size, + const int64_t alignment, + MemoryPool* pool) { + return ResizePoolBuffer>( + PoolBuffer::MakeUnique( + pool, + alignment), + size); +} + +Result> AllocateResizableBuffer( + const int64_t size, + MemoryPool* pool) { + return AllocateResizableBuffer( + size, + MemoryPool::kDefaultAlignment, + pool); } -Result> AllocateResizableBuffer(const int64_t size, - MemoryPool* pool) { - return ResizePoolBuffer>(PoolBuffer::MakeUnique(pool), - size); +Result> AllocateResizableBuffer( + const int64_t size, + const int64_t alignment, + MemoryPool* pool) { + return ResizePoolBuffer>( + PoolBuffer::MakeUnique( + pool, + alignment), + size); } } // namespace arrow diff --git a/cpp/src/arrow/memory_pool.h b/cpp/src/arrow/memory_pool.h index 58b375af3a9d9..e4405987309a7 100644 --- a/cpp/src/arrow/memory_pool.h +++ b/cpp/src/arrow/memory_pool.h @@ -63,6 +63,8 @@ class MemoryPoolStats { /// take care of the required 64-byte alignment. class ARROW_EXPORT MemoryPool { public: + static constexpr int64_t kDefaultAlignment = 64; + virtual ~MemoryPool() = default; /// \brief EXPERIMENTAL. Create a new instance of the default MemoryPool @@ -71,13 +73,17 @@ class ARROW_EXPORT MemoryPool { /// Allocate a new memory region of at least size bytes. /// /// The allocated region shall be 64-byte aligned. - virtual Status Allocate(int64_t size, uint8_t** out) = 0; + Status Allocate(int64_t size, uint8_t** out) { return Allocate(size, kDefaultAlignment, out); } + + /// Allocate a new memory region of at least size bytes aligned to alignment. + virtual Status Allocate(int64_t size, int64_t alignment, uint8_t** out) = 0; /// Resize an already allocated memory section. /// /// As by default most default allocators on a platform don't support aligned /// reallocation, this function can involve a copy of the underlying data. - virtual Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) = 0; + virtual Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) = 0; + Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) { return Reallocate(old_size, new_size, kDefaultAlignment, ptr); } /// Free an allocated region. /// @@ -85,7 +91,9 @@ class ARROW_EXPORT MemoryPool { /// @param size Allocated size located at buffer. An allocator implementation /// may use this for tracking the amount of allocated bytes as well as for /// faster deallocation if supported by its backend. - virtual void Free(uint8_t* buffer, int64_t size) = 0; + /// @param alignment The alignment of the allocation. Defaults to 64 bytes. + virtual void Free(uint8_t* buffer, int64_t size, int64_t alignment) = 0; + void Free(uint8_t *buffer, int64_t size) { Free(buffer, size, kDefaultAlignment); } /// Return unused memory to the OS /// @@ -116,10 +124,13 @@ class ARROW_EXPORT LoggingMemoryPool : public MemoryPool { explicit LoggingMemoryPool(MemoryPool* pool); ~LoggingMemoryPool() override = default; - Status Allocate(int64_t size, uint8_t** out) override; - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override; + using MemoryPool::Allocate; + using MemoryPool::Reallocate; + using MemoryPool::Free; - void Free(uint8_t* buffer, int64_t size) override; + Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; + Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) override; + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; int64_t bytes_allocated() const override; @@ -140,10 +151,13 @@ class ARROW_EXPORT ProxyMemoryPool : public MemoryPool { explicit ProxyMemoryPool(MemoryPool* pool); ~ProxyMemoryPool() override; - Status Allocate(int64_t size, uint8_t** out) override; - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override; + using MemoryPool::Allocate; + using MemoryPool::Reallocate; + using MemoryPool::Free; - void Free(uint8_t* buffer, int64_t size) override; + Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; + Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) override; + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; int64_t bytes_allocated() const override; diff --git a/cpp/src/arrow/memory_pool_internal.h b/cpp/src/arrow/memory_pool_internal.h index df0ee646a47e8..6176a1da8fcf5 100644 --- a/cpp/src/arrow/memory_pool_internal.h +++ b/cpp/src/arrow/memory_pool_internal.h @@ -26,8 +26,6 @@ namespace memory_pool { namespace internal { -static constexpr size_t kAlignment = 64; - static constexpr int64_t kDebugXorSuffix = -0x181fe80e0b464188LL; // A static piece of memory for 0-size allocations, so as to return @@ -41,9 +39,9 @@ static uint8_t* const kZeroSizeArea = reinterpret_cast(&zero_size_area // Helper class directing allocations to the jemalloc allocator. class JemallocAllocator { public: - static Status AllocateAligned(int64_t size, uint8_t** out); - static Status ReallocateAligned(int64_t old_size, int64_t new_size, uint8_t** ptr); - static void DeallocateAligned(uint8_t* ptr, int64_t size); + static Status AllocateAligned(int64_t size, int64_t alignment, uint8_t** out); + static Status ReallocateAligned(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr); + static void DeallocateAligned(uint8_t* ptr, int64_t size, int64_t alignment); static void ReleaseUnused(); }; diff --git a/cpp/src/arrow/memory_pool_jemalloc.cc b/cpp/src/arrow/memory_pool_jemalloc.cc index 48a5bac137bd6..caea99f6e1a93 100644 --- a/cpp/src/arrow/memory_pool_jemalloc.cc +++ b/cpp/src/arrow/memory_pool_jemalloc.cc @@ -81,33 +81,35 @@ namespace memory_pool { namespace internal { -Status JemallocAllocator::AllocateAligned(int64_t size, uint8_t** out) { +Status JemallocAllocator::AllocateAligned(int64_t size, int64_t alignment, uint8_t** out) { if (size == 0) { *out = kZeroSizeArea; return Status::OK(); } *out = reinterpret_cast( - mallocx(static_cast(size), MALLOCX_ALIGN(kAlignment))); + mallocx(static_cast(size), MALLOCX_ALIGN(static_cast(alignment)))); if (*out == NULL) { return Status::OutOfMemory("malloc of size ", size, " failed"); } return Status::OK(); } -Status JemallocAllocator::ReallocateAligned(int64_t old_size, int64_t new_size, - uint8_t** ptr) { +Status JemallocAllocator::ReallocateAligned(int64_t old_size, + int64_t new_size, + int64_t alignment, + uint8_t** ptr) { uint8_t* previous_ptr = *ptr; if (previous_ptr == kZeroSizeArea) { DCHECK_EQ(old_size, 0); - return AllocateAligned(new_size, ptr); + return AllocateAligned(new_size, alignment, ptr); } if (new_size == 0) { - DeallocateAligned(previous_ptr, old_size); + DeallocateAligned(previous_ptr, old_size, alignment); *ptr = kZeroSizeArea; return Status::OK(); } *ptr = reinterpret_cast( - rallocx(*ptr, static_cast(new_size), MALLOCX_ALIGN(kAlignment))); + rallocx(*ptr, static_cast(new_size), MALLOCX_ALIGN(static_cast(alignment)))); if (*ptr == NULL) { *ptr = previous_ptr; return Status::OutOfMemory("realloc of size ", new_size, " failed"); @@ -115,11 +117,11 @@ Status JemallocAllocator::ReallocateAligned(int64_t old_size, int64_t new_size, return Status::OK(); } -void JemallocAllocator::DeallocateAligned(uint8_t* ptr, int64_t size) { +void JemallocAllocator::DeallocateAligned(uint8_t* ptr, int64_t size, int64_t alignment) { if (ptr == kZeroSizeArea) { DCHECK_EQ(size, 0); } else { - dallocx(ptr, MALLOCX_ALIGN(kAlignment)); + sdallocx(ptr, static_cast(size), MALLOCX_ALIGN(static_cast(alignment))); } } diff --git a/cpp/src/arrow/stl_allocator.h b/cpp/src/arrow/stl_allocator.h index b5ad2b53460f0..e6ad82fddce31 100644 --- a/cpp/src/arrow/stl_allocator.h +++ b/cpp/src/arrow/stl_allocator.h @@ -100,7 +100,11 @@ class STLMemoryPool : public MemoryPool { /// \brief Construct a memory pool from the given allocator explicit STLMemoryPool(const Allocator& alloc) : alloc_(alloc) {} - Status Allocate(int64_t size, uint8_t** out) override { + using MemoryPool::Allocate; + using MemoryPool::Reallocate; + using MemoryPool::Free; + + Status Allocate(int64_t size, int64_t /*alignment*/, uint8_t** out) override { try { *out = alloc_.allocate(size); } catch (std::bad_alloc& e) { @@ -110,7 +114,7 @@ class STLMemoryPool : public MemoryPool { return Status::OK(); } - Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override { + Status Reallocate(int64_t old_size, int64_t new_size, int64_t /*alignment*/, uint8_t** ptr) override { uint8_t* old_ptr = *ptr; try { *ptr = alloc_.allocate(new_size); @@ -123,7 +127,7 @@ class STLMemoryPool : public MemoryPool { return Status::OK(); } - void Free(uint8_t* buffer, int64_t size) override { + void Free(uint8_t* buffer, int64_t size, int64_t /*alignment*/) override { alloc_.deallocate(buffer, size); stats_.UpdateAllocatedBytes(-size); } diff --git a/cpp/src/arrow/util/atomic_util.h b/cpp/src/arrow/util/atomic_util.h new file mode 100644 index 0000000000000..356d7f3a6c418 --- /dev/null +++ b/cpp/src/arrow/util/atomic_util.h @@ -0,0 +1,172 @@ +// 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. + +#pragma once + +#include +#include + +namespace arrow +{ + namespace util + { + template + inline T AtomicMax(std::atomic &to_max, T val) + { + static_assert(std::is_arithmetic::value, "Maximum only makes sense on numeric types!"); + T local_to_max = to_max.load(std::memory_order_relaxed); + while(val > local_to_max + && !to_max.compare_exchange_weak( + local_to_max, + val, + std::memory_order_release, + std::memory_order_relaxed)) + {} + return local_to_max; + } + +#if defined(__clang) || defined(__GNUC__) + template + inline T AtomicLoad(T *addr, std::memory_order order = std::memory_order_seq_cst) noexcept + { + T ret; + __atomic_load(addr, &ret, order); + return ret; + } + + template + inline void AtomicStore(T *addr, T &val, std::memory_order order = std::memory_order_seq_cst) noexcept + { + __atomic_store(addr, val, order); + } + + template + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order order = std::memory_order_seq_cst) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + return __atomic_fetch_add(addr, val, order); + } + + template + inline T AtomicFetchSub(T *addr, T &val, std::memory_order order = std::memory_order_seq_cst) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return __atomic_fetch_sub(addr, val, order); + } + +#elif defined(_MSC_VER) + #include + template + inline T AtomicLoad(T *addr, std::memory_order /*order*/) noexcept + { + T val = *addr; + _ReadWriteBarrier(); + return val; + } + + template + inline void AtomicStore(T *addr, T &val, std::memory_order /*order*/) noexcept + { + _ReadWriteBarrier(); + *addr = val; + } + + template ::type> + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order /*order*/) noexcepet + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + return _InterlockedExchangeAdd8(addr, val); + } + + template ::type> + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + return _InterlockedExchangeAdd16(addr, val); + } + + template ::type> + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + return _InterlockedExchangeAdd(addr, val); + } + +#if _WIN64 + template ::type> + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + return _InterlockedExchangeAdd64(addr, val); + } +#else + template ::type> + inline T AtomicFetchAdd(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchAdd can only be used on integral types"); + _ReadWriteBarrier(); + T expected = *addr; + for(;;) + { + T new_val = expected + val; + T prev = _InterlockedCompareExchange64(addr, new_val, expected); + if(prev == expected) + return prev; + expected = prev; + } + } +#endif + + template ::type> + inline T AtomicFetchSub(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return _InterlockedExchangeAdd8(addr, -val); + } + + template ::type> + inline T AtomicFetchSub(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return _InterlockedExchangeAdd16(addr, -val); + } + + template ::type> + inline T AtomicFetchSub(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return _InterlockedExchangeAdd(addr, -val); + } + +#if _WIN64 + template ::type> + inline T AtomicFetchSub(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return _InterlockedExchangeAdd64(addr, -val); + } +#else + template ::type> + inline T AtomicFetchSub(T *addr, T &val, std::memory_order /*order*/) noexcept + { + static_assert(std::is_integral::value, "AtomicFetchSub can only be used on integral types"); + return AtomicFetchAdd(addr, -val, std::memory_order_seq_cst); + } +#endif +#endif + } +} diff --git a/cpp/src/arrow/util/io_util.cc b/cpp/src/arrow/util/io_util.cc index 11ae80d03e256..f45dbbd6f73c0 100644 --- a/cpp/src/arrow/util/io_util.cc +++ b/cpp/src/arrow/util/io_util.cc @@ -106,9 +106,11 @@ #elif __APPLE__ #include +#include #elif __linux__ #include +#include #endif namespace arrow { @@ -2107,5 +2109,37 @@ int64_t GetCurrentRSS() { #endif } + int64_t GetTotalMemoryBytes() + { +#if defined(_WIN32) + int64_t result_kb; + if(!GetPhysicallyInstalledSystemMemory(&result)) + { + ARROW_LOG(WARNING) << "FAiled to resolve total RAM size: " << std::strerror(GetLastError()); + return -1; + } + return (result_kb * 1024); +#elif defined(__APPLE__) + int64_t result; + size_t size = sizeof(result); + if(sysctlbyname("hw.memsize", &result, &size, nullptr, 0) == -1) + { + ARROW_LOG(WARNING) << "Failed to resolve total RAM size"; + return -1; + } + return result; +#elif defined(__linux__) + struct sysinfo info; + if(sysinfo(&info) == -1) + { + ARROW_LOG(WARNING) << "Failed to resolve total RAM size: " << std::strerror(errno); + return -1; + } + return static_cast(info.totalram * info.mem_unit); +#else + return 0; +#endif + } + } // namespace internal } // namespace arrow diff --git a/cpp/src/arrow/util/io_util.h b/cpp/src/arrow/util/io_util.h index df63de47e8386..43d85ec24e28b 100644 --- a/cpp/src/arrow/util/io_util.h +++ b/cpp/src/arrow/util/io_util.h @@ -410,5 +410,11 @@ uint64_t GetThreadId(); ARROW_EXPORT int64_t GetCurrentRSS(); +/// \brief Get the total memory available to the system in bytes +/// +/// This function supports Windows, Linux, and Mac and will return 0 otherwise +ARROW_EXPORT +int64_t GetTotalMemoryBytes(); + } // namespace internal } // namespace arrow