From b583196b9e104e949a00171f547b8f4c73fb6698 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 20 Jul 2021 15:20:44 -0400 Subject: [PATCH 01/37] init --- cpp/src/arrow/CMakeLists.txt | 1 + cpp/src/arrow/compute/exec/exec_plan.cc | 295 ++++++++++++++++++++++++ cpp/src/arrow/compute/exec/hash_join.cc | 25 ++ cpp/src/arrow/compute/exec/hash_join.h | 29 +++ 4 files changed, 350 insertions(+) create mode 100644 cpp/src/arrow/compute/exec/hash_join.cc create mode 100644 cpp/src/arrow/compute/exec/hash_join.h diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index f13e5b1ef75..8f898caa7ed 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -406,6 +406,7 @@ if(ARROW_COMPUTE) compute/kernels/vector_replace.cc compute/kernels/vector_selection.cc compute/kernels/vector_sort.cc + compute/exec/hash_join.cc compute/exec/key_hash.cc compute/exec/key_map.cc compute/exec/key_compare.cc diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 4a4758c8471..5dee9fadc5f 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1308,5 +1308,300 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /*null_count=*/0); } +struct HashSemiIndexJoinNode : ExecNode { + HashSemiIndexJoinNode(ExecNode* left_input, ExecNode* right_input, std::string label, + std::shared_ptr output_schema, ExecContext* ctx, + const std::vector&& index_field_ids) + : ExecNode(left_input->plan(), std::move(label), {left_input, right_input}, + {"hashsemiindexjoin"}, std::move(output_schema), /*num_outputs=*/1), + ctx_(ctx), + index_field_ids_(std::move(index_field_ids)) { + // num_input_batches_processed_.store(0); + // num_input_batches_total_.store(-1); + // num_output_batches_processed_.store(0); + output_started_.store(false); + } + + const char* kind_name() override { return "HashSemiIndexJoinNode"; } + + private: + struct ThreadLocalState; + + public: + Status InitLocalStateIfNeeded(ThreadLocalState* state) { + // Get input schema + auto input_schema = inputs_[0]->output_schema(); + + if (!state->grouper) { + // Build vector of key field data types + std::vector key_descrs(key_field_ids_.size()); + for (size_t i = 0; i < key_field_ids_.size(); ++i) { + auto key_field_id = key_field_ids_[i]; + key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); + } + + // Construct grouper + ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + } + if (state->agg_states.empty()) { + // Build vector of aggregate source field data types + std::vector agg_src_descrs(agg_kernels_.size()); + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + auto agg_src_field_id = agg_src_field_ids_[i]; + agg_src_descrs[i] = + ValueDescr(input_schema->field(agg_src_field_id)->type(), ValueDescr::ARRAY); + } + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + ARROW_ASSIGN_OR_RAISE( + state->agg_states, + internal::InitKernels(agg_kernels_, ctx_, aggs_, agg_src_descrs)); + + ARROW_ASSIGN_OR_RAISE( + FieldVector agg_result_fields, + internal::ResolveKernels(aggs_, agg_kernels_, state->agg_states, ctx_, + agg_src_descrs)); + } + } + return Status::OK(); + } + + Status ProcessInputBatch(const ExecBatch& batch) { + SmallUniqueIdHolder id_holder(&local_state_id_assignment_); + int id = id_holder.get(); + ThreadLocalState* state = local_states_.get(id); + RETURN_NOT_OK(InitLocalStateIfNeeded(state)); + + // Create a batch with key columns + std::vector keys(key_field_ids_.size()); + for (size_t i = 0; i < key_field_ids_.size(); ++i) { + keys[i] = batch.values[key_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); + + // Execute aggregate kernels + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext kernel_ctx{ctx_}; + kernel_ctx.SetState(state->agg_states[i].get()); + + ARROW_ASSIGN_OR_RAISE( + auto agg_batch, + ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); + + RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); + RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + } + + return Status::OK(); + } + + Status Merge() { + int num_local_states = local_state_id_assignment_.num_ids(); + ThreadLocalState* state0 = local_states_.get(0); + for (int i = 1; i < num_local_states; ++i) { + ThreadLocalState* state = local_states_.get(i); + ARROW_DCHECK(state); + ARROW_DCHECK(state->grouper); + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum transposition, state0->grouper->Consume(other_keys)); + state->grouper.reset(); + + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + ARROW_DCHECK(state0->agg_states[i]); + batch_ctx.SetState(state0->agg_states[i].get()); + + RETURN_NOT_OK(agg_kernels_[i]->resize(&batch_ctx, state0->grouper->num_groups())); + RETURN_NOT_OK(agg_kernels_[i]->merge(&batch_ctx, std::move(*state->agg_states[i]), + *transposition.array())); + state->agg_states[i].reset(); + } + } + return Status::OK(); + } + + Status Finalize() { + out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); + auto it = out_data_.begin(); + + ThreadLocalState* state = local_states_.get(0); + num_out_groups_ = state->grouper->num_groups(); + + // Aggregate fields come before key fields to match the behavior of GroupBy function + + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + batch_ctx.SetState(state->agg_states[i].get()); + Datum out; + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); + *it++ = out.array(); + state->agg_states[i].reset(); + } + + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } + state->grouper.reset(); + + return Status::OK(); + } + + Status OutputNthBatch(int n) { + ARROW_DCHECK(output_started_.load()); + + // Check finished flag + if (finished_.is_finished()) { + return Status::OK(); + } + + // Slice arrays + int64_t batch_size = output_batch_size(); + int64_t batch_start = n * batch_size; + int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); + std::vector output_slices(out_data_.size()); + for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { + output_slices[out_field_id] = + out_data_[out_field_id]->Slice(batch_start, batch_length); + } + + ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); + outputs_[0]->InputReceived(this, n, output_batch); + + uint32_t num_output_batches_processed = + 1 + num_output_batches_processed_.fetch_add(1); + if (num_output_batches_processed * batch_size >= num_out_groups_) { + finished_.MarkFinished(); + } + + return Status::OK(); + } + + Status OutputResult() { + bool expected = false; + if (!output_started_.compare_exchange_strong(expected, true)) { + return Status::OK(); + } + + RETURN_NOT_OK(Merge()); + RETURN_NOT_OK(Finalize()); + + int batch_size = output_batch_size(); + int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; + outputs_[0]->InputFinished(this, num_result_batches); + + auto executor = arrow::internal::GetCpuThreadPool(); + for (int i = 0; i < num_result_batches; ++i) { + // Check finished flag + if (finished_.is_finished()) { + break; + } + + RETURN_NOT_OK(executor->Spawn([this, i]() { + Status status = OutputNthBatch(i); + if (!status.ok()) { + ErrorReceived(inputs_[0], status); + } + })); + } + + return Status::OK(); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + if (finished_.is_finished()) { + return; + } + + ARROW_DCHECK(num_input_batches_processed_.load() != num_input_batches_total_.load()); + + Status status = ProcessInputBatch(batch); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + + num_input_batches_processed_.fetch_add(1); + if (num_input_batches_processed_.load() == num_input_batches_total_.load()) { + status = OutputResult(); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + + outputs_[0]->ErrorReceived(this, std::move(error)); + StopProducing(); + } + + void InputFinished(ExecNode* input, int seq) override { + DCHECK_EQ(input, inputs_[0]); + + num_input_batches_total_.store(seq); + if (num_input_batches_processed_.load() == num_input_batches_total_.load()) { + Status status = OutputResult(); + + if (!status.ok()) { + ErrorReceived(input, status); + } + } + } + + Status StartProducing() override { + finished_ = Future<>::Make(); + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + DCHECK_EQ(output, outputs_[0]); + inputs_[0]->StopProducing(this); + + finished_.MarkFinished(); + } + + void StopProducing() override { StopProducing(outputs_[0]); } + + Future<> finished() override { return finished_; } + + private: + int output_batch_size() const { + int result = static_cast(ctx_->exec_chunksize()); + if (result < 0) { + result = 32 * 1024; + } + return result; + } + + ExecContext* ctx_; + Future<> finished_ = Future<>::MakeFinished(); + + // std::atomic num_input_batches_processed_; + // std::atomic num_input_batches_total_; + // std::atomic num_output_batches_processed_; + + const std::vector index_field_ids_; + + struct ThreadLocalState { + std::unique_ptr grouper; + std::vector> agg_states; + }; + SharedSequenceOfObjects local_states_; + SmallUniqueIdAssignment local_state_id_assignment_; + uint32_t num_out_groups_{0}; + ArrayDataVector out_data_; + std::atomic output_started_; +}; } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc new file mode 100644 index 00000000000..10b676be892 --- /dev/null +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -0,0 +1,25 @@ +// 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/hash_join.h" + +namespace arrow { +namespace compute { + + +} // namespace compute +} // namespace arrow \ No newline at end of file diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/hash_join.h new file mode 100644 index 00000000000..492cf0a0a49 --- /dev/null +++ b/cpp/src/arrow/compute/exec/hash_join.h @@ -0,0 +1,29 @@ +// 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. + +namespace arrow { +namespace compute { + +enum JoinType { + LEFT_SEMI_JOIN, + RIGHT_SEMI_JOIN, + LEFT_ANTI_SEMI_JOIN, + RIGHT_ANTI_SEMI_JOIN +}; + +} // namespace compute +} // namespace arrow \ No newline at end of file From 83e3994aeadd7d876d6eae8c8c94f7b7d135c277 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Mon, 26 Jul 2021 18:11:01 -0400 Subject: [PATCH 02/37] adding Grouper::Find --- cpp/src/arrow/compute/api_aggregate.h | 5 + cpp/src/arrow/compute/exec/exec_plan.cc | 458 +++++++++--------- .../arrow/compute/kernels/hash_aggregate.cc | 77 ++- .../compute/kernels/hash_aggregate_test.cc | 22 + 4 files changed, 309 insertions(+), 253 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index d66d4f1517c..195e76fe9f1 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -349,6 +349,11 @@ class ARROW_EXPORT Grouper { /// be as wide as necessary. virtual Result Consume(const ExecBatch& batch) = 0; + /// Finds/ queries the group IDs for the given ExecBatch for every index. Returns the + /// group IDs as an integer array. If a group ID not found, a UINT32_MAX will be + /// added to that index. This is a thread-safe lookup. + virtual Result Find(const ExecBatch& batch) = 0; + /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 5dee9fadc5f..969db5db889 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1308,300 +1308,278 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /*null_count=*/0); } -struct HashSemiIndexJoinNode : ExecNode { +/*struct HashSemiIndexJoinNode : ExecNode { HashSemiIndexJoinNode(ExecNode* left_input, ExecNode* right_input, std::string label, std::shared_ptr output_schema, ExecContext* ctx, const std::vector&& index_field_ids) : ExecNode(left_input->plan(), std::move(label), {left_input, right_input}, - {"hashsemiindexjoin"}, std::move(output_schema), /*num_outputs=*/1), - ctx_(ctx), - index_field_ids_(std::move(index_field_ids)) { - // num_input_batches_processed_.store(0); - // num_input_batches_total_.store(-1); - // num_output_batches_processed_.store(0); - output_started_.store(false); - } - - const char* kind_name() override { return "HashSemiIndexJoinNode"; } - - private: - struct ThreadLocalState; - - public: - Status InitLocalStateIfNeeded(ThreadLocalState* state) { - // Get input schema - auto input_schema = inputs_[0]->output_schema(); - - if (!state->grouper) { - // Build vector of key field data types - std::vector key_descrs(key_field_ids_.size()); - for (size_t i = 0; i < key_field_ids_.size(); ++i) { - auto key_field_id = key_field_ids_[i]; - key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); - } - - // Construct grouper - ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); - } - if (state->agg_states.empty()) { - // Build vector of aggregate source field data types - std::vector agg_src_descrs(agg_kernels_.size()); - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - auto agg_src_field_id = agg_src_field_ids_[i]; - agg_src_descrs[i] = - ValueDescr(input_schema->field(agg_src_field_id)->type(), ValueDescr::ARRAY); - } - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - ARROW_ASSIGN_OR_RAISE( - state->agg_states, - internal::InitKernels(agg_kernels_, ctx_, aggs_, agg_src_descrs)); - - ARROW_ASSIGN_OR_RAISE( - FieldVector agg_result_fields, - internal::ResolveKernels(aggs_, agg_kernels_, state->agg_states, ctx_, - agg_src_descrs)); - } - } - return Status::OK(); - } + {"hashsemiindexjoin"}, std::move(output_schema), */ +/*num_outputs=*//*1), +ctx_(ctx), +num_build_batches_processed_(0), +num_build_batches_total_(-1), +num_probe_batches_processed_(0), +num_probe_batches_total_(-1), +num_output_batches_processed_(0), +index_field_ids_(std::move(index_field_ids)), +output_started_(false), +build_phase_finished_(false){} + +const char* kind_name() override { return "HashSemiIndexJoinNode"; } + +private: +struct ThreadLocalState; + +public: +Status InitLocalStateIfNeeded(ThreadLocalState* state) { +// Get input schema +auto input_schema = inputs_[0]->output_schema(); + +if (!state->grouper) { +// Build vector of key field data types +std::vector key_descrs(index_field_ids_.size()); +for (size_t i = 0; i < index_field_ids_.size(); ++i) { +auto key_field_id = index_field_ids_[i]; +key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); +} - Status ProcessInputBatch(const ExecBatch& batch) { - SmallUniqueIdHolder id_holder(&local_state_id_assignment_); - int id = id_holder.get(); - ThreadLocalState* state = local_states_.get(id); - RETURN_NOT_OK(InitLocalStateIfNeeded(state)); +// Construct grouper +ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); +} - // Create a batch with key columns - std::vector keys(key_field_ids_.size()); - for (size_t i = 0; i < key_field_ids_.size(); ++i) { - keys[i] = batch.values[key_field_ids_[i]]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); +return Status::OK(); +} - // Create a batch with group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); +Status ProcessBuildSideBatch(const ExecBatch& batch) { +SmallUniqueIdHolder id_holder(&local_state_id_assignment_); +int id = id_holder.get(); +ThreadLocalState* state = local_states_.get(id); +RETURN_NOT_OK(InitLocalStateIfNeeded(state)); - // Execute aggregate kernels - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext kernel_ctx{ctx_}; - kernel_ctx.SetState(state->agg_states[i].get()); +// Create a batch with key columns +std::vector keys(key_field_ids_.size()); +for (size_t i = 0; i < key_field_ids_.size(); ++i) { +keys[i] = batch.values[key_field_ids_[i]]; +} +ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - ARROW_ASSIGN_OR_RAISE( - auto agg_batch, - ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); +// Create a batch with group ids +ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); - RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); - RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); - } +// Execute aggregate kernels +for (size_t i = 0; i < agg_kernels_.size(); ++i) { +KernelContext kernel_ctx{ctx_}; +kernel_ctx.SetState(state->agg_states[i].get()); - return Status::OK(); - } +ARROW_ASSIGN_OR_RAISE( +auto agg_batch, +ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); - Status Merge() { - int num_local_states = local_state_id_assignment_.num_ids(); - ThreadLocalState* state0 = local_states_.get(0); - for (int i = 1; i < num_local_states; ++i) { - ThreadLocalState* state = local_states_.get(i); - ARROW_DCHECK(state); - ARROW_DCHECK(state->grouper); - ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum transposition, state0->grouper->Consume(other_keys)); - state->grouper.reset(); +RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); +RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); +} - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext batch_ctx{ctx_}; - ARROW_DCHECK(state0->agg_states[i]); - batch_ctx.SetState(state0->agg_states[i].get()); +return Status::OK(); +} - RETURN_NOT_OK(agg_kernels_[i]->resize(&batch_ctx, state0->grouper->num_groups())); - RETURN_NOT_OK(agg_kernels_[i]->merge(&batch_ctx, std::move(*state->agg_states[i]), - *transposition.array())); - state->agg_states[i].reset(); - } - } - return Status::OK(); - } +// merge all other groupers to grouper[0]. nothing needs to be done on the +// early_probe_batches, because when probing everyone +Status BuildSideMerge() { +int num_local_states = local_state_id_assignment_.num_ids(); +ThreadLocalState* state0 = local_states_.get(0); +for (int i = 1; i < num_local_states; ++i) { +ThreadLocalState* state = local_states_.get(i); +ARROW_DCHECK(state); +ARROW_DCHECK(state->grouper); +ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); +ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); +state->grouper.reset(); +} +return Status::OK(); +} - Status Finalize() { - out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); - auto it = out_data_.begin(); +Status Finalize() { +out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); +auto it = out_data_.begin(); - ThreadLocalState* state = local_states_.get(0); - num_out_groups_ = state->grouper->num_groups(); +ThreadLocalState* state = local_states_.get(0); +num_out_groups_ = state->grouper->num_groups(); - // Aggregate fields come before key fields to match the behavior of GroupBy function +// Aggregate fields come before key fields to match the behavior of GroupBy function - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext batch_ctx{ctx_}; - batch_ctx.SetState(state->agg_states[i].get()); - Datum out; - RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); - *it++ = out.array(); - state->agg_states[i].reset(); - } +for (size_t i = 0; i < agg_kernels_.size(); ++i) { +KernelContext batch_ctx{ctx_}; +batch_ctx.SetState(state->agg_states[i].get()); +Datum out; +RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); +*it++ = out.array(); +state->agg_states[i].reset(); +} - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); - for (const auto& key : out_keys.values) { - *it++ = key.array(); - } - state->grouper.reset(); +ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); +for (const auto& key : out_keys.values) { +*it++ = key.array(); +} +state->grouper.reset(); - return Status::OK(); - } +return Status::OK(); +} - Status OutputNthBatch(int n) { - ARROW_DCHECK(output_started_.load()); +Status OutputNthBatch(int n) { +ARROW_DCHECK(output_started_.load()); - // Check finished flag - if (finished_.is_finished()) { - return Status::OK(); - } +// Check finished flag +if (finished_.is_finished()) { +return Status::OK(); +} - // Slice arrays - int64_t batch_size = output_batch_size(); - int64_t batch_start = n * batch_size; - int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); - std::vector output_slices(out_data_.size()); - for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { - output_slices[out_field_id] = - out_data_[out_field_id]->Slice(batch_start, batch_length); - } +// Slice arrays +int64_t batch_size = output_batch_size(); +int64_t batch_start = n * batch_size; +int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); +std::vector output_slices(out_data_.size()); +for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { +output_slices[out_field_id] = +out_data_[out_field_id]->Slice(batch_start, batch_length); +} - ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); - outputs_[0]->InputReceived(this, n, output_batch); +ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); +outputs_[0]->InputReceived(this, n, output_batch); - uint32_t num_output_batches_processed = - 1 + num_output_batches_processed_.fetch_add(1); - if (num_output_batches_processed * batch_size >= num_out_groups_) { - finished_.MarkFinished(); - } +uint32_t num_output_batches_processed = +1 + num_output_batches_processed_.fetch_add(1); +if (num_output_batches_processed * batch_size >= num_out_groups_) { +finished_.MarkFinished(); +} - return Status::OK(); - } +return Status::OK(); +} - Status OutputResult() { - bool expected = false; - if (!output_started_.compare_exchange_strong(expected, true)) { - return Status::OK(); - } +Status OutputResult() { +bool expected = false; +if (!output_started_.compare_exchange_strong(expected, true)) { +return Status::OK(); +} - RETURN_NOT_OK(Merge()); - RETURN_NOT_OK(Finalize()); +RETURN_NOT_OK(BuildSideMerge()); +RETURN_NOT_OK(Finalize()); - int batch_size = output_batch_size(); - int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; - outputs_[0]->InputFinished(this, num_result_batches); +int batch_size = output_batch_size(); +int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; +outputs_[0]->InputFinished(this, num_result_batches); - auto executor = arrow::internal::GetCpuThreadPool(); - for (int i = 0; i < num_result_batches; ++i) { - // Check finished flag - if (finished_.is_finished()) { - break; - } +auto executor = arrow::internal::GetCpuThreadPool(); +for (int i = 0; i < num_result_batches; ++i) { +// Check finished flag +if (finished_.is_finished()) { +break; +} - RETURN_NOT_OK(executor->Spawn([this, i]() { - Status status = OutputNthBatch(i); - if (!status.ok()) { - ErrorReceived(inputs_[0], status); - } - })); - } +RETURN_NOT_OK(executor->Spawn([this, i]() { +Status status = OutputNthBatch(i); +if (!status.ok()) { +ErrorReceived(inputs_[0], status); +} +})); +} - return Status::OK(); - } +return Status::OK(); +} - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - DCHECK_EQ(input, inputs_[0]); +void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { +assert(input == inputs_[0] || input == inputs_[1]); - if (finished_.is_finished()) { - return; - } +if (finished_.is_finished()) { +return; +} - ARROW_DCHECK(num_input_batches_processed_.load() != num_input_batches_total_.load()); +ARROW_DCHECK(num_build_batches_processed_.load() != num_build_batches_total_.load()); - Status status = ProcessInputBatch(batch); - if (!status.ok()) { - ErrorReceived(input, status); - return; - } +Status status = ProcessBuildSideBatch(batch); +if (!status.ok()) { +ErrorReceived(input, status); +return; +} - num_input_batches_processed_.fetch_add(1); - if (num_input_batches_processed_.load() == num_input_batches_total_.load()) { - status = OutputResult(); - if (!status.ok()) { - ErrorReceived(input, status); - return; - } - } - } +num_build_batches_processed_.fetch_add(1); +if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { +status = OutputResult(); +if (!status.ok()) { +ErrorReceived(input, status); +return; +} +} +} - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); +void ErrorReceived(ExecNode* input, Status error) override { +DCHECK_EQ(input, inputs_[0]); - outputs_[0]->ErrorReceived(this, std::move(error)); - StopProducing(); - } +outputs_[0]->ErrorReceived(this, std::move(error)); +StopProducing(); +} - void InputFinished(ExecNode* input, int seq) override { - DCHECK_EQ(input, inputs_[0]); +void InputFinished(ExecNode* input, int seq) override { +DCHECK_EQ(input, inputs_[0]); - num_input_batches_total_.store(seq); - if (num_input_batches_processed_.load() == num_input_batches_total_.load()) { - Status status = OutputResult(); +num_build_batches_total_.store(seq); +if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { +Status status = OutputResult(); - if (!status.ok()) { - ErrorReceived(input, status); - } - } - } +if (!status.ok()) { +ErrorReceived(input, status); +} +} +} - Status StartProducing() override { - finished_ = Future<>::Make(); - return Status::OK(); - } +Status StartProducing() override { +finished_ = Future<>::Make(); +return Status::OK(); +} - void PauseProducing(ExecNode* output) override {} +void PauseProducing(ExecNode* output) override {} - void ResumeProducing(ExecNode* output) override {} +void ResumeProducing(ExecNode* output) override {} - void StopProducing(ExecNode* output) override { - DCHECK_EQ(output, outputs_[0]); - inputs_[0]->StopProducing(this); +void StopProducing(ExecNode* output) override { +DCHECK_EQ(output, outputs_[0]); +inputs_[0]->StopProducing(this); - finished_.MarkFinished(); - } +finished_.MarkFinished(); +} - void StopProducing() override { StopProducing(outputs_[0]); } +void StopProducing() override { StopProducing(outputs_[0]); } - Future<> finished() override { return finished_; } +Future<> finished() override { return finished_; } - private: - int output_batch_size() const { - int result = static_cast(ctx_->exec_chunksize()); - if (result < 0) { - result = 32 * 1024; - } - return result; - } +private: +int output_batch_size() const { +int result = static_cast(ctx_->exec_chunksize()); +if (result < 0) { +result = 32 * 1024; +} +return result; +} - ExecContext* ctx_; - Future<> finished_ = Future<>::MakeFinished(); +ExecContext* ctx_; +Future<> finished_ = Future<>::MakeFinished(); - // std::atomic num_input_batches_processed_; - // std::atomic num_input_batches_total_; - // std::atomic num_output_batches_processed_; +std::atomic num_build_batches_processed_; +std::atomic num_build_batches_total_; +std::atomic num_probe_batches_processed_; +std::atomic num_probe_batches_total_; +std::atomic num_output_batches_processed_; - const std::vector index_field_ids_; +const std::vector index_field_ids_; - struct ThreadLocalState { - std::unique_ptr grouper; - std::vector> agg_states; - }; - SharedSequenceOfObjects local_states_; - SmallUniqueIdAssignment local_state_id_assignment_; - uint32_t num_out_groups_{0}; - ArrayDataVector out_data_; - std::atomic output_started_; +struct ThreadLocalState { +std::unique_ptr grouper; +std::vector early_probe_batches{}; }; +SharedSequenceOfObjects local_states_; +SmallUniqueIdAssignment local_state_id_assignment_; +uint32_t num_out_groups_{0}; +ArrayDataVector out_data_; +std::atomic output_started_, build_phase_finished_; +};*/ } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index ba5c90f15de..25adf3edcbd 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +#include + #include #include #include @@ -368,30 +370,43 @@ struct GrouperImpl : Grouper { return std::move(impl); } - Result Consume(const ExecBatch& batch) override { - std::vector offsets_batch(batch.length + 1); + Status PopulateKeyData(const ExecBatch& batch, std::vector* offsets_batch, + std::vector* key_bytes_batch, + std::vector* key_buf_ptrs) { + offsets_batch->resize(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { - encoders_[i]->AddLength(*batch[i].array(), offsets_batch.data()); + encoders_[i]->AddLength(*batch[i].array(), offsets_batch->data()); } int32_t total_length = 0; for (int64_t i = 0; i < batch.length; ++i) { auto total_length_before = total_length; - total_length += offsets_batch[i]; - offsets_batch[i] = total_length_before; + total_length += offsets_batch->at(i); + offsets_batch->at(i) = total_length_before; } - offsets_batch[batch.length] = total_length; + offsets_batch->at(batch.length) = total_length; - std::vector key_bytes_batch(total_length); - std::vector key_buf_ptrs(batch.length); + key_bytes_batch->resize(total_length); + key_buf_ptrs->resize(batch.length); for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs[i] = key_bytes_batch.data() + offsets_batch[i]; + key_buf_ptrs->at(i) = key_bytes_batch->data() + offsets_batch->at(i); } for (int i = 0; i < batch.num_values(); ++i) { - RETURN_NOT_OK(encoders_[i]->Encode(*batch[i].array(), key_buf_ptrs.data())); + RETURN_NOT_OK(encoders_[i]->Encode(*batch[i].array(), key_buf_ptrs->data())); } + return Status::OK(); + } + + Result Consume(const ExecBatch& batch) override { + std::vector offsets_batch; + std::vector key_bytes_batch; + std::vector key_buf_ptrs; + + RETURN_NOT_OK( + PopulateKeyData(batch, &offsets_batch, &key_bytes_batch, &key_buf_ptrs)); + TypedBufferBuilder group_ids_batch(ctx_->memory_pool()); RETURN_NOT_OK(group_ids_batch.Resize(batch.length)); @@ -420,6 +435,36 @@ struct GrouperImpl : Grouper { return Datum(UInt32Array(batch.length, std::move(group_ids))); } + Result Find(const ExecBatch& batch) override { + std::vector offsets_batch; + std::vector key_bytes_batch; + std::vector key_buf_ptrs; + + RETURN_NOT_OK( + PopulateKeyData(batch, &offsets_batch, &key_bytes_batch, &key_buf_ptrs)); + + UInt32Builder group_ids_batch(ctx_->memory_pool()); + RETURN_NOT_OK(group_ids_batch.Resize(batch.length)); + + for (int64_t i = 0; i < batch.length; ++i) { + int32_t key_length = offsets_batch[i + 1] - offsets_batch[i]; + std::string key( + reinterpret_cast(key_bytes_batch.data() + offsets_batch[i]), + key_length); + + auto it = map_.find(key); + // no group ID was found, null will be emitted! + if (it == map_.end()) { + group_ids_batch.UnsafeAppendNull(); + } else { + group_ids_batch.UnsafeAppend(it->second); + } + } + + ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); + return Datum(group_ids); + } + uint32_t num_groups() const override { return num_groups_; } Result GetUniques() override { @@ -621,6 +666,11 @@ struct GrouperFastImpl : Grouper { return Datum(UInt32Array(batch.length, std::move(group_ids))); } + Result Find(const ExecBatch& batch) override { + // todo impl this + return Result(); + } + uint32_t num_groups() const override { return static_cast(rows_.length()); } // Make sure padded buffers end up with the right logical size @@ -1769,9 +1819,10 @@ Result ResolveKernels( Result> Grouper::Make(const std::vector& descrs, ExecContext* ctx) { - if (GrouperFastImpl::CanUse(descrs)) { - return GrouperFastImpl::Make(descrs, ctx); - } + // TODO(niranda) re-enable this! + // if (GrouperFastImpl::CanUse(descrs)) { + // return GrouperFastImpl::Make(descrs, ctx); + // } return GrouperImpl::Make(descrs, ctx); } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index d37d8f32ac8..3b96b23b415 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -216,6 +216,16 @@ struct TestGrouper { AssertEquivalentIds(expected, ids); } + void ExpectFind(const std::string& key_json, const std::string& expected) { + ExpectFind(ExecBatchFromJSON(descrs_, key_json), ArrayFromJSON(uint32(), expected)); + } + void ExpectFind(const ExecBatch& key_batch, Datum expected) { + ASSERT_OK_AND_ASSIGN(Datum id_batch, grouper_->Find(key_batch)); + ValidateOutput(id_batch); + + AssertDatumsEqual(expected, id_batch); + } + void AssertEquivalentIds(const Datum& expected, const Datum& actual) { auto left = expected.make_array(); auto right = actual.make_array(); @@ -341,6 +351,18 @@ TEST(Grouper, NumericKey) { g.ExpectConsume("[[3], [27], [3], [27], [null], [81], [27], [81]]", "[0, 1, 0, 1, 3, 2, 1, 2]"); + + g.ExpectFind("[[3], [3]]", "[0, 0]"); + + g.ExpectFind("[[3], [3]]", "[0, 0]"); + + g.ExpectFind("[[27], [81]]", "[1, 2]"); + + g.ExpectFind("[[3], [27], [3], [27], [null], [81], [27], [81]]", + "[0, 1, 0, 1, 3, 2, 1, 2]"); + + g.ExpectFind("[[27], [3], [27], [null], [81], [1], [81]]", + "[1, 0, 1, 3, 2, null, 2]"); } } From 8ddc475b353d36c9f37780293da6f6b8974efa30 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 28 Jul 2021 14:44:56 -0400 Subject: [PATCH 03/37] incomplete --- cpp/src/arrow/compute/exec/exec_plan.cc | 454 ++++++++++++------------ 1 file changed, 234 insertions(+), 220 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 969db5db889..e2f945038bb 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -541,6 +541,8 @@ class AtomicCounter { return total; } + bool IsComplete() { return count_.load() == total_.load(); } + // return true if the counter is complete bool Increment() { DCHECK_NE(count_.load(), total_.load()); @@ -1308,278 +1310,290 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /*null_count=*/0); } -/*struct HashSemiIndexJoinNode : ExecNode { - HashSemiIndexJoinNode(ExecNode* left_input, ExecNode* right_input, std::string label, +struct HashSemiIndexJoinNode : ExecNode { + HashSemiIndexJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, std::shared_ptr output_schema, ExecContext* ctx, const std::vector&& index_field_ids) - : ExecNode(left_input->plan(), std::move(label), {left_input, right_input}, - {"hashsemiindexjoin"}, std::move(output_schema), */ -/*num_outputs=*//*1), -ctx_(ctx), -num_build_batches_processed_(0), -num_build_batches_total_(-1), -num_probe_batches_processed_(0), -num_probe_batches_total_(-1), -num_output_batches_processed_(0), -index_field_ids_(std::move(index_field_ids)), -output_started_(false), -build_phase_finished_(false){} - -const char* kind_name() override { return "HashSemiIndexJoinNode"; } - -private: -struct ThreadLocalState; - -public: -Status InitLocalStateIfNeeded(ThreadLocalState* state) { -// Get input schema -auto input_schema = inputs_[0]->output_schema(); - -if (!state->grouper) { -// Build vector of key field data types -std::vector key_descrs(index_field_ids_.size()); -for (size_t i = 0; i < index_field_ids_.size(); ++i) { -auto key_field_id = index_field_ids_[i]; -key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); -} + : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, + {"hash_join_build", "hash_join_probe"}, std::move(output_schema), + /*num_outputs=*/1), + ctx_(ctx), + index_field_ids_(std::move(index_field_ids)) {} -// Construct grouper -ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); -} + const char* kind_name() override { return "HashSemiIndexJoinNode"; } -return Status::OK(); -} + public: + Status InitLocalStateIfNeeded(ThreadLocalState* state) { + // Get input schema + auto input_schema = inputs_[0]->output_schema(); -Status ProcessBuildSideBatch(const ExecBatch& batch) { -SmallUniqueIdHolder id_holder(&local_state_id_assignment_); -int id = id_holder.get(); -ThreadLocalState* state = local_states_.get(id); -RETURN_NOT_OK(InitLocalStateIfNeeded(state)); + if (!state->grouper) { + // Build vector of key field data types + std::vector key_descrs(index_field_ids_.size()); + for (size_t i = 0; i < index_field_ids_.size(); ++i) { + auto key_field_id = index_field_ids_[i]; + key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); + } -// Create a batch with key columns -std::vector keys(key_field_ids_.size()); -for (size_t i = 0; i < key_field_ids_.size(); ++i) { -keys[i] = batch.values[key_field_ids_[i]]; -} -ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + // Construct grouper + ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + } -// Create a batch with group ids -ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); + return Status::OK(); + } -// Execute aggregate kernels -for (size_t i = 0; i < agg_kernels_.size(); ++i) { -KernelContext kernel_ctx{ctx_}; -kernel_ctx.SetState(state->agg_states[i].get()); + Status ProcessBuildSideBatch(const ExecBatch& batch) { + SmallUniqueIdHolder id_holder(&local_state_id_assignment_); + int id = id_holder.get(); + ThreadLocalState* state = local_states_.get(id); + RETURN_NOT_OK(InitLocalStateIfNeeded(state)); -ARROW_ASSIGN_OR_RAISE( -auto agg_batch, -ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); + // Create a batch with key columns + std::vector keys(key_field_ids_.size()); + for (size_t i = 0; i < key_field_ids_.size(); ++i) { + keys[i] = batch.values[key_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); -RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); -RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); -} + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); -return Status::OK(); -} + // Execute aggregate kernels + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext kernel_ctx{ctx_}; + kernel_ctx.SetState(state->agg_states[i].get()); -// merge all other groupers to grouper[0]. nothing needs to be done on the -// early_probe_batches, because when probing everyone -Status BuildSideMerge() { -int num_local_states = local_state_id_assignment_.num_ids(); -ThreadLocalState* state0 = local_states_.get(0); -for (int i = 1; i < num_local_states; ++i) { -ThreadLocalState* state = local_states_.get(i); -ARROW_DCHECK(state); -ARROW_DCHECK(state->grouper); -ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); -ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); -state->grouper.reset(); -} -return Status::OK(); -} + ARROW_ASSIGN_OR_RAISE( + auto agg_batch, + ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); -Status Finalize() { -out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); -auto it = out_data_.begin(); + RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); + RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + } -ThreadLocalState* state = local_states_.get(0); -num_out_groups_ = state->grouper->num_groups(); + return Status::OK(); + } -// Aggregate fields come before key fields to match the behavior of GroupBy function + // merge all other groupers to grouper[0]. nothing needs to be done on the + // early_probe_batches, because when probing everyone + Status BuildSideMerge() { + int num_local_states = local_state_id_assignment_.num_ids(); + ThreadLocalState* state0 = local_states_.get(0); + for (int i = 1; i < num_local_states; ++i) { + ThreadLocalState* state = local_states_.get(i); + ARROW_DCHECK(state); + ARROW_DCHECK(state->grouper); + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); + state->grouper.reset(); + } + return Status::OK(); + } -for (size_t i = 0; i < agg_kernels_.size(); ++i) { -KernelContext batch_ctx{ctx_}; -batch_ctx.SetState(state->agg_states[i].get()); -Datum out; -RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); -*it++ = out.array(); -state->agg_states[i].reset(); -} + Status Finalize() { + out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); + auto it = out_data_.begin(); -ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); -for (const auto& key : out_keys.values) { -*it++ = key.array(); -} -state->grouper.reset(); + ThreadLocalState* state = local_states_.get(0); + num_out_groups_ = state->grouper->num_groups(); -return Status::OK(); -} + // Aggregate fields come before key fields to match the behavior of GroupBy function -Status OutputNthBatch(int n) { -ARROW_DCHECK(output_started_.load()); + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + batch_ctx.SetState(state->agg_states[i].get()); + Datum out; + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); + *it++ = out.array(); + state->agg_states[i].reset(); + } -// Check finished flag -if (finished_.is_finished()) { -return Status::OK(); -} + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } + state->grouper.reset(); -// Slice arrays -int64_t batch_size = output_batch_size(); -int64_t batch_start = n * batch_size; -int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); -std::vector output_slices(out_data_.size()); -for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { -output_slices[out_field_id] = -out_data_[out_field_id]->Slice(batch_start, batch_length); -} + return Status::OK(); + } -ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); -outputs_[0]->InputReceived(this, n, output_batch); + Status OutputNthBatch(int n) { + ARROW_DCHECK(output_started_.load()); -uint32_t num_output_batches_processed = -1 + num_output_batches_processed_.fetch_add(1); -if (num_output_batches_processed * batch_size >= num_out_groups_) { -finished_.MarkFinished(); -} + // Check finished flag + if (finished_.is_finished()) { + return Status::OK(); + } -return Status::OK(); -} + // Slice arrays + int64_t batch_size = output_batch_size(); + int64_t batch_start = n * batch_size; + int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); + std::vector output_slices(out_data_.size()); + for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { + output_slices[out_field_id] = + out_data_[out_field_id]->Slice(batch_start, batch_length); + } -Status OutputResult() { -bool expected = false; -if (!output_started_.compare_exchange_strong(expected, true)) { -return Status::OK(); -} + ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); + outputs_[0]->InputReceived(this, n, output_batch); -RETURN_NOT_OK(BuildSideMerge()); -RETURN_NOT_OK(Finalize()); + uint32_t num_output_batches_processed = + 1 + num_output_batches_processed_.fetch_add(1); + if (num_output_batches_processed * batch_size >= num_out_groups_) { + finished_.MarkFinished(); + } -int batch_size = output_batch_size(); -int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; -outputs_[0]->InputFinished(this, num_result_batches); + return Status::OK(); + } -auto executor = arrow::internal::GetCpuThreadPool(); -for (int i = 0; i < num_result_batches; ++i) { -// Check finished flag -if (finished_.is_finished()) { -break; -} + Status OutputResult() { + bool expected = false; + if (!output_started_.compare_exchange_strong(expected, true)) { + return Status::OK(); + } -RETURN_NOT_OK(executor->Spawn([this, i]() { -Status status = OutputNthBatch(i); -if (!status.ok()) { -ErrorReceived(inputs_[0], status); -} -})); -} + RETURN_NOT_OK(BuildSideMerge()); + RETURN_NOT_OK(Finalize()); -return Status::OK(); -} + int batch_size = output_batch_size(); + int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; + outputs_[0]->InputFinished(this, num_result_batches); -void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { -assert(input == inputs_[0] || input == inputs_[1]); + auto executor = arrow::internal::GetCpuThreadPool(); + for (int i = 0; i < num_result_batches; ++i) { + // Check finished flag + if (finished_.is_finished()) { + break; + } -if (finished_.is_finished()) { -return; -} + RETURN_NOT_OK(executor->Spawn([this, i]() { + Status status = OutputNthBatch(i); + if (!status.ok()) { + ErrorReceived(inputs_[0], status); + } + })); + } -ARROW_DCHECK(num_build_batches_processed_.load() != num_build_batches_total_.load()); + return Status::OK(); + } -Status status = ProcessBuildSideBatch(batch); -if (!status.ok()) { -ErrorReceived(input, status); -return; -} + Status ProcessBuildBatch(const ExecBatch& batch) { + // if build side is still going on + return Status::OK(); + } -num_build_batches_processed_.fetch_add(1); -if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { -status = OutputResult(); -if (!status.ok()) { -ErrorReceived(input, status); -return; -} -} -} + Status ProcessCachedProbeBatches() { return Status::OK(); } -void ErrorReceived(ExecNode* input, Status error) override { -DCHECK_EQ(input, inputs_[0]); + Status ProcessProbeBatch(const ExecBatch& batch) { return Status::OK(); } -outputs_[0]->ErrorReceived(this, std::move(error)); -StopProducing(); -} + Status CacheProbeBatch(const ExecBatch& batch) { return Status::OK(); } -void InputFinished(ExecNode* input, int seq) override { -DCHECK_EQ(input, inputs_[0]); + // If all build side batches received? continue streaming using probing + // else cache the batches in thread-local state + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); -num_build_batches_total_.store(seq); -if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { -Status status = OutputResult(); + size_t thread_index = get_thread_index_(); + ARROW_DCHECK(thread_index < local_states_.size()); -if (!status.ok()) { -ErrorReceived(input, status); -} -} -} + if (finished_.is_finished()) { + return; + } -Status StartProducing() override { -finished_ = Future<>::Make(); -return Status::OK(); -} + if (build_counter_.IsComplete()) { // build side complete! + ARROW_DCHECK(input != inputs_[0]); // if a build batch is received, error! + if (ErrorIfNotOk(ProcessProbeBatch(batch))) return; + } else { // build side is still processing! + if (input == inputs_[1]) { // if a probe batch is received, cache it! + if (ErrorIfNotOk(CacheProbeBatch(batch))) return; + } else { // else process build batch + if (ErrorIfNotOk(ProcessBuildSideBatch(batch))) return; + } + } -void PauseProducing(ExecNode* output) override {} + ARROW_DCHECK(num_build_batches_processed_.load() != num_build_batches_total_.load()); -void ResumeProducing(ExecNode* output) override {} + Status status = ProcessBuildSideBatch(batch); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } -void StopProducing(ExecNode* output) override { -DCHECK_EQ(output, outputs_[0]); -inputs_[0]->StopProducing(this); + num_build_batches_processed_.fetch_add(1); + if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { + status = OutputResult(); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + } + } -finished_.MarkFinished(); -} + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); -void StopProducing() override { StopProducing(outputs_[0]); } + outputs_[0]->ErrorReceived(this, std::move(error)); + StopProducing(); + } -Future<> finished() override { return finished_; } + void InputFinished(ExecNode* input, int seq) override { + DCHECK_EQ(input, inputs_[0]); -private: -int output_batch_size() const { -int result = static_cast(ctx_->exec_chunksize()); -if (result < 0) { -result = 32 * 1024; -} -return result; -} + num_build_batches_total_.store(seq); + if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { + Status status = OutputResult(); -ExecContext* ctx_; -Future<> finished_ = Future<>::MakeFinished(); + if (!status.ok()) { + ErrorReceived(input, status); + } + } + } -std::atomic num_build_batches_processed_; -std::atomic num_build_batches_total_; -std::atomic num_probe_batches_processed_; -std::atomic num_probe_batches_total_; -std::atomic num_output_batches_processed_; + Status StartProducing() override { + finished_ = Future<>::Make(); + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + DCHECK_EQ(output, outputs_[0]); + inputs_[0]->StopProducing(this); + + finished_.MarkFinished(); + } -const std::vector index_field_ids_; + void StopProducing() override { StopProducing(outputs_[0]); } + + Future<> finished() override { return finished_; } -struct ThreadLocalState { -std::unique_ptr grouper; -std::vector early_probe_batches{}; + private: + int output_batch_size() const { + int result = static_cast(ctx_->exec_chunksize()); + if (result < 0) { + result = 32 * 1024; + } + return result; + } + + struct ThreadLocalState { + std::unique_ptr grouper; + std::vector early_probe_batches{}; + }; + + ExecContext* ctx_; + Future<> finished_ = Future<>::MakeFinished(); + + ThreadIndexer get_thread_index_; + const std::vector index_field_ids_; + + AtomicCounter build_counter_, probe_counter_, out_counter_; + std::vector local_states_; + ExecBatch out_data_; }; -SharedSequenceOfObjects local_states_; -SmallUniqueIdAssignment local_state_id_assignment_; -uint32_t num_out_groups_{0}; -ArrayDataVector out_data_; -std::atomic output_started_, build_phase_finished_; -};*/ + } // namespace compute } // namespace arrow From 2f7cb0f3e4b2bf8c4c4c43dc64a7c45d45123abc Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 29 Jul 2021 15:29:35 -0400 Subject: [PATCH 04/37] mid way --- cpp/src/arrow/compute/api_aggregate.h | 2 +- cpp/src/arrow/compute/exec/exec_plan.cc | 246 +++++++++++------- .../arrow/compute/kernels/hash_aggregate.cc | 6 +- 3 files changed, 152 insertions(+), 102 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 195e76fe9f1..506398cb23c 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -352,7 +352,7 @@ class ARROW_EXPORT Grouper { /// Finds/ queries the group IDs for the given ExecBatch for every index. Returns the /// group IDs as an integer array. If a group ID not found, a UINT32_MAX will be /// added to that index. This is a thread-safe lookup. - virtual Result Find(const ExecBatch& batch) = 0; + virtual Result Find(const ExecBatch& batch) const = 0; /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index e2f945038bb..550deddb88a 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -541,7 +541,7 @@ class AtomicCounter { return total; } - bool IsComplete() { return count_.load() == total_.load(); } + bool TotalReached() { return complete_.load(); } // return true if the counter is complete bool Increment() { @@ -1318,7 +1318,11 @@ struct HashSemiIndexJoinNode : ExecNode { {"hash_join_build", "hash_join_probe"}, std::move(output_schema), /*num_outputs=*/1), ctx_(ctx), - index_field_ids_(std::move(index_field_ids)) {} + index_field_ids_(index_field_ids), + build_side_complete_(false) {} + + private: + struct ThreadLocalState; const char* kind_name() override { return "HashSemiIndexJoinNode"; } @@ -1327,94 +1331,92 @@ struct HashSemiIndexJoinNode : ExecNode { // Get input schema auto input_schema = inputs_[0]->output_schema(); - if (!state->grouper) { - // Build vector of key field data types - std::vector key_descrs(index_field_ids_.size()); - for (size_t i = 0; i < index_field_ids_.size(); ++i) { - auto key_field_id = index_field_ids_[i]; - key_descrs[i] = ValueDescr(input_schema->field(key_field_id)->type()); - } + if (state->grouper != nullptr) return Status::OK(); - // Construct grouper - ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + // Build vector of key field data types + std::vector key_descrs(index_field_ids_.size()); + for (size_t i = 0; i < index_field_ids_.size(); ++i) { + auto idx_field_id = index_field_ids_[i]; + key_descrs[i] = ValueDescr(input_schema->field(idx_field_id)->type()); } + // Construct grouper + ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + return Status::OK(); } - Status ProcessBuildSideBatch(const ExecBatch& batch) { - SmallUniqueIdHolder id_holder(&local_state_id_assignment_); - int id = id_holder.get(); - ThreadLocalState* state = local_states_.get(id); + // merge all other groupers to grouper[0]. nothing needs to be done on the + // cached_probe_batches, because when probing everyone + Status BuildSideMerge() { + ThreadLocalState* state0 = &local_states_[0]; + for (int i = 1; i < local_states_.size(); ++i) { + ThreadLocalState* state = &local_states_[i]; + ARROW_DCHECK(state); + ARROW_DCHECK(state->grouper); + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); + state->grouper.reset(); + } + return Status::OK(); + } + + // consumes a build batch and increments the build_batches count. if the build batches + // total reached at the end of consumption, all the local states will be merged, before + // incrementing the total batches + Status ConsumeBuildBatch(const size_t thread_index, ExecBatch batch) { + auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); // Create a batch with key columns - std::vector keys(key_field_ids_.size()); - for (size_t i = 0; i < key_field_ids_.size(); ++i) { - keys[i] = batch.values[key_field_ids_[i]]; + std::vector keys(index_field_ids_.size()); + for (size_t i = 0; i < index_field_ids_.size(); ++i) { + keys[i] = batch.values[index_field_ids_[i]]; } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); // Create a batch with group ids ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); - // Execute aggregate kernels - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext kernel_ctx{ctx_}; - kernel_ctx.SetState(state->agg_states[i].get()); + if (build_counter_.Increment()) { + // while incrementing, if the total is reached, merge all the groupers to 0'th one + RETURN_NOT_OK(BuildSideMerge()); - ARROW_ASSIGN_OR_RAISE( - auto agg_batch, - ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); + // enable flag that build side is completed + build_side_complete_.store(true); - RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); - RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + // since the build side is completed, consume cached probe batches + RETURN_NOT_OK(ConsumeCachedProbeBatches(thread_index)); } return Status::OK(); } - // merge all other groupers to grouper[0]. nothing needs to be done on the - // early_probe_batches, because when probing everyone - Status BuildSideMerge() { - int num_local_states = local_state_id_assignment_.num_ids(); - ThreadLocalState* state0 = local_states_.get(0); - for (int i = 1; i < num_local_states; ++i) { - ThreadLocalState* state = local_states_.get(i); - ARROW_DCHECK(state); - ARROW_DCHECK(state->grouper); - ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); - state->grouper.reset(); - } - return Status::OK(); - } - Status Finalize() { - out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); - auto it = out_data_.begin(); + ThreadLocalState* state = &local_states_[0]; - ThreadLocalState* state = local_states_.get(0); - num_out_groups_ = state->grouper->num_groups(); + ExecBatch out_data{{}, state->grouper->num_groups()}; + out_data.values.resize(agg_kernels_.size() + key_field_ids_.size()); // Aggregate fields come before key fields to match the behavior of GroupBy function - for (size_t i = 0; i < agg_kernels_.size(); ++i) { KernelContext batch_ctx{ctx_}; batch_ctx.SetState(state->agg_states[i].get()); - Datum out; - RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); - *it++ = out.array(); + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out_data.values[i])); state->agg_states[i].reset(); } ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); - for (const auto& key : out_keys.values) { - *it++ = key.array(); - } + std::move(out_keys.values.begin(), out_keys.values.end(), + out_data.values.begin() + agg_kernels_.size()); state->grouper.reset(); - return Status::OK(); + if (output_counter_.SetTotal( + static_cast(BitUtil::CeilDiv(out_data.length, output_batch_size())))) { + // this will be hit if out_data.length == 0 + finished_.MarkFinished(); + } + return out_data; } Status OutputNthBatch(int n) { @@ -1453,7 +1455,6 @@ struct HashSemiIndexJoinNode : ExecNode { return Status::OK(); } - RETURN_NOT_OK(BuildSideMerge()); RETURN_NOT_OK(Finalize()); int batch_size = output_batch_size(); @@ -1478,16 +1479,53 @@ struct HashSemiIndexJoinNode : ExecNode { return Status::OK(); } - Status ProcessBuildBatch(const ExecBatch& batch) { - // if build side is still going on + Status ConsumeCachedProbeBatches(const size_t thread_index) { + ThreadLocalState* state = &local_states_[thread_index]; + + // TODO (niranda) check if this is the best way to move batches + for (ExecBatch batch : state->cached_probe_batches) { + RETURN_NOT_OK(ConsumeProbeBatch(std::move(batch))); + } + state->cached_probe_batches.clear(); + return Status::OK(); } - Status ProcessCachedProbeBatches() { return Status::OK(); } + // consumes a probe batch and increment probe batches count. Probing would query the + // grouper[0] which have been merged with all others. + Status ConsumeProbeBatch(ExecBatch batch) { + auto* grouper = local_states_[0].grouper.get(); + + // Create a batch with key columns + std::vector keys(index_field_ids_.size()); + for (size_t i = 0; i < index_field_ids_.size(); ++i) { + keys[i] = batch.values[index_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + + // Query the grouper with key_batch. If no match was found, returning group_ids would + // have null. + ARROW_ASSIGN_OR_RAISE(Datum group_ids, grouper->Find(key_batch)); + auto group_ids_data = *group_ids.array(); - Status ProcessProbeBatch(const ExecBatch& batch) { return Status::OK(); } + auto filter_arr = + std::make_shared(group_ids_data.length, group_ids_data.buffers[0], + /*null_bitmap=*/nullptr, /*null_count=*/0, + /*offset=*/group_ids_data.offset); + Filter(); - Status CacheProbeBatch(const ExecBatch& batch) { return Status::OK(); } + probe_counter_.Increment(); + return Status::OK(); + } + + Status CacheProbeBatch(const size_t thread_index, ExecBatch batch) { + ThreadLocalState* state = &local_states_[thread_index]; + state->cached_probe_batches.push_back(std::move(batch)); + return Status::OK(); + } + + inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } + inline bool IsProbeInput(ExecNode* input) { return input == inputs_[1]; } // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state @@ -1501,31 +1539,16 @@ struct HashSemiIndexJoinNode : ExecNode { return; } - if (build_counter_.IsComplete()) { // build side complete! - ARROW_DCHECK(input != inputs_[0]); // if a build batch is received, error! - if (ErrorIfNotOk(ProcessProbeBatch(batch))) return; - } else { // build side is still processing! - if (input == inputs_[1]) { // if a probe batch is received, cache it! - if (ErrorIfNotOk(CacheProbeBatch(batch))) return; - } else { // else process build batch - if (ErrorIfNotOk(ProcessBuildSideBatch(batch))) return; - } - } - - ARROW_DCHECK(num_build_batches_processed_.load() != num_build_batches_total_.load()); - - Status status = ProcessBuildSideBatch(batch); - if (!status.ok()) { - ErrorReceived(input, status); - return; - } + if (IsBuildInput(input)) { // build input batch is received + // if a build input is received when build side is completed, something's wrong! + ARROW_DCHECK(!build_side_complete_.load()); - num_build_batches_processed_.fetch_add(1); - if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { - status = OutputResult(); - if (!status.ok()) { - ErrorReceived(input, status); - return; + if (ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch)))) return; + } else { // probe input batch is received + if (build_side_complete_.load()) { // build side done, continue with probing + if (ErrorIfNotOk(ConsumeProbeBatch(std::move(batch)))) return; + } else { // build side not completed. Cache this batch! + if (ErrorIfNotOk(CacheProbeBatch(thread_index, std::move(batch)))) return; } } } @@ -1537,21 +1560,31 @@ struct HashSemiIndexJoinNode : ExecNode { StopProducing(); } - void InputFinished(ExecNode* input, int seq) override { - DCHECK_EQ(input, inputs_[0]); + void InputFinished(ExecNode* input, int num_total) override { + // bail if StopProducing was called + if (finished_.is_finished()) return; - num_build_batches_total_.store(seq); - if (num_build_batches_processed_.load() == num_build_batches_total_.load()) { - Status status = OutputResult(); + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - if (!status.ok()) { - ErrorReceived(input, status); - } + // set total for build input + if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { + // only build side has completed! so process cached probe batches (of this thread) + ErrorIfNotOk(ConsumeCachedProbeBatches(get_thread_index_())); + return; } + + // set total for probe input. If it returns that probe side has completed, nothing to + // do, because probing inputs will be streamed to the output + probe_counter_.SetTotal(num_total); + + // output will be streamed from the probe side. So, they will have the same total. + out_counter_.SetTotal(num_total); } Status StartProducing() override { finished_ = Future<>::Make(); + + local_states_.resize(ThreadIndexer::Capacity()); return Status::OK(); } @@ -1560,13 +1593,23 @@ struct HashSemiIndexJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - DCHECK_EQ(output, outputs_[0]); - inputs_[0]->StopProducing(this); + // DCHECK_EQ(output, outputs_[0]); - finished_.MarkFinished(); + if (build_counter_.Cancel() || probe_counter_.Cancel() || out_counter_.Cancel()) { + finished_.MarkFinished(); + } + + for (auto&& input : inputs_) { + input->StopProducing(this); + } } - void StopProducing() override { StopProducing(outputs_[0]); } + // TODO(niranda) couldn't there be multiple outputs for a Node? + void StopProducing() override { + for (auto&& output : outputs_) { + StopProducing(output); + } + } Future<> finished() override { return finished_; } @@ -1581,7 +1624,7 @@ struct HashSemiIndexJoinNode : ExecNode { struct ThreadLocalState { std::unique_ptr grouper; - std::vector early_probe_batches{}; + std::vector cached_probe_batches{}; }; ExecContext* ctx_; @@ -1592,6 +1635,13 @@ struct HashSemiIndexJoinNode : ExecNode { AtomicCounter build_counter_, probe_counter_, out_counter_; std::vector local_states_; + + // need a separate atomic bool to track if the build side complete. Can't use the flag + // inside the AtomicCounter, because we need to merge the build groupers once we receive + // all the build batches. So, while merging, we need to prevent probe batches, being + // consumed. + std::atomic build_side_complete_; + ExecBatch out_data_; }; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 25adf3edcbd..93c9d01ffae 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -372,7 +372,7 @@ struct GrouperImpl : Grouper { Status PopulateKeyData(const ExecBatch& batch, std::vector* offsets_batch, std::vector* key_bytes_batch, - std::vector* key_buf_ptrs) { + std::vector* key_buf_ptrs) const { offsets_batch->resize(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { encoders_[i]->AddLength(*batch[i].array(), offsets_batch->data()); @@ -435,7 +435,7 @@ struct GrouperImpl : Grouper { return Datum(UInt32Array(batch.length, std::move(group_ids))); } - Result Find(const ExecBatch& batch) override { + Result Find(const ExecBatch& batch) const override { std::vector offsets_batch; std::vector key_bytes_batch; std::vector key_buf_ptrs; @@ -666,7 +666,7 @@ struct GrouperFastImpl : Grouper { return Datum(UInt32Array(batch.length, std::move(group_ids))); } - Result Find(const ExecBatch& batch) override { + Result Find(const ExecBatch& batch) const override { // todo impl this return Result(); } From 51ef659ef52b29a975e02c633efe2b555f495d3e Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 29 Jul 2021 23:02:48 -0400 Subject: [PATCH 05/37] untested --- cpp/src/arrow/compute/exec/exec_plan.cc | 161 ++++++------------------ 1 file changed, 39 insertions(+), 122 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 550deddb88a..7b3fed95f46 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1310,10 +1310,10 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /*null_count=*/0); } -struct HashSemiIndexJoinNode : ExecNode { - HashSemiIndexJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, - std::shared_ptr output_schema, ExecContext* ctx, - const std::vector&& index_field_ids) +struct HashSemiJoinNode : ExecNode { + HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, + std::shared_ptr output_schema, ExecContext* ctx, + const std::vector&& index_field_ids) : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, {"hash_join_build", "hash_join_probe"}, std::move(output_schema), /*num_outputs=*/1), @@ -1324,9 +1324,9 @@ struct HashSemiIndexJoinNode : ExecNode { private: struct ThreadLocalState; - const char* kind_name() override { return "HashSemiIndexJoinNode"; } - public: + const char* kind_name() override { return "HashSemiJoinNode"; } + Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema auto input_schema = inputs_[0]->output_schema(); @@ -1392,99 +1392,12 @@ struct HashSemiIndexJoinNode : ExecNode { return Status::OK(); } - Status Finalize() { - ThreadLocalState* state = &local_states_[0]; - - ExecBatch out_data{{}, state->grouper->num_groups()}; - out_data.values.resize(agg_kernels_.size() + key_field_ids_.size()); - - // Aggregate fields come before key fields to match the behavior of GroupBy function - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext batch_ctx{ctx_}; - batch_ctx.SetState(state->agg_states[i].get()); - RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out_data.values[i])); - state->agg_states[i].reset(); - } - - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); - std::move(out_keys.values.begin(), out_keys.values.end(), - out_data.values.begin() + agg_kernels_.size()); - state->grouper.reset(); - - if (output_counter_.SetTotal( - static_cast(BitUtil::CeilDiv(out_data.length, output_batch_size())))) { - // this will be hit if out_data.length == 0 - finished_.MarkFinished(); - } - return out_data; - } - - Status OutputNthBatch(int n) { - ARROW_DCHECK(output_started_.load()); - - // Check finished flag - if (finished_.is_finished()) { - return Status::OK(); - } - - // Slice arrays - int64_t batch_size = output_batch_size(); - int64_t batch_start = n * batch_size; - int64_t batch_length = std::min(batch_size, num_out_groups_ - batch_start); - std::vector output_slices(out_data_.size()); - for (size_t out_field_id = 0; out_field_id < out_data_.size(); ++out_field_id) { - output_slices[out_field_id] = - out_data_[out_field_id]->Slice(batch_start, batch_length); - } - - ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); - outputs_[0]->InputReceived(this, n, output_batch); - - uint32_t num_output_batches_processed = - 1 + num_output_batches_processed_.fetch_add(1); - if (num_output_batches_processed * batch_size >= num_out_groups_) { - finished_.MarkFinished(); - } - - return Status::OK(); - } - - Status OutputResult() { - bool expected = false; - if (!output_started_.compare_exchange_strong(expected, true)) { - return Status::OK(); - } - - RETURN_NOT_OK(Finalize()); - - int batch_size = output_batch_size(); - int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; - outputs_[0]->InputFinished(this, num_result_batches); - - auto executor = arrow::internal::GetCpuThreadPool(); - for (int i = 0; i < num_result_batches; ++i) { - // Check finished flag - if (finished_.is_finished()) { - break; - } - - RETURN_NOT_OK(executor->Spawn([this, i]() { - Status status = OutputNthBatch(i); - if (!status.ok()) { - ErrorReceived(inputs_[0], status); - } - })); - } - - return Status::OK(); - } - Status ConsumeCachedProbeBatches(const size_t thread_index) { ThreadLocalState* state = &local_states_[thread_index]; // TODO (niranda) check if this is the best way to move batches - for (ExecBatch batch : state->cached_probe_batches) { - RETURN_NOT_OK(ConsumeProbeBatch(std::move(batch))); + for (auto cached : state->cached_probe_batches) { + RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); } state->cached_probe_batches.clear(); @@ -1493,7 +1406,7 @@ struct HashSemiIndexJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[0] which have been merged with all others. - Status ConsumeProbeBatch(ExecBatch batch) { + Status ConsumeProbeBatch(int seq, ExecBatch batch) { auto* grouper = local_states_[0].grouper.get(); // Create a batch with key columns @@ -1508,24 +1421,34 @@ struct HashSemiIndexJoinNode : ExecNode { ARROW_ASSIGN_OR_RAISE(Datum group_ids, grouper->Find(key_batch)); auto group_ids_data = *group_ids.array(); - auto filter_arr = - std::make_shared(group_ids_data.length, group_ids_data.buffers[0], - /*null_bitmap=*/nullptr, /*null_count=*/0, - /*offset=*/group_ids_data.offset); - Filter(); + if (group_ids_data.MayHaveNulls()) { // values need to be filtered + auto filter_arr = + std::make_shared(group_ids_data.length, group_ids_data.buffers[0], + /*null_bitmap=*/nullptr, /*null_count=*/0, + /*offset=*/group_ids_data.offset); + ARROW_ASSIGN_OR_RAISE(auto rec_batch, + batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); + ARROW_ASSIGN_OR_RAISE( + auto filtered, + Filter(rec_batch, filter_arr, + /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); + auto out_batch = ExecBatch(*filtered.record_batch()); + outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + } else { // all values are valid for output + outputs_[0]->InputReceived(this, seq, std::move(batch)); + } - probe_counter_.Increment(); + out_counter_.Increment(); return Status::OK(); } - Status CacheProbeBatch(const size_t thread_index, ExecBatch batch) { + Status CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { ThreadLocalState* state = &local_states_[thread_index]; - state->cached_probe_batches.push_back(std::move(batch)); + state->cached_probe_batches.emplace_back(seq_num, std::move(batch)); return Status::OK(); } inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } - inline bool IsProbeInput(ExecNode* input) { return input == inputs_[1]; } // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state @@ -1546,9 +1469,9 @@ struct HashSemiIndexJoinNode : ExecNode { if (ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch)))) return; } else { // probe input batch is received if (build_side_complete_.load()) { // build side done, continue with probing - if (ErrorIfNotOk(ConsumeProbeBatch(std::move(batch)))) return; + if (ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch)))) return; } else { // build side not completed. Cache this batch! - if (ErrorIfNotOk(CacheProbeBatch(thread_index, std::move(batch)))) return; + if (ErrorIfNotOk(CacheProbeBatch(thread_index, seq, std::move(batch)))) return; } } } @@ -1575,10 +1498,13 @@ struct HashSemiIndexJoinNode : ExecNode { // set total for probe input. If it returns that probe side has completed, nothing to // do, because probing inputs will be streamed to the output - probe_counter_.SetTotal(num_total); + // probe_counter_.SetTotal(num_total); // output will be streamed from the probe side. So, they will have the same total. - out_counter_.SetTotal(num_total); + if (out_counter_.SetTotal(num_total)) { + // if out_counter has completed, the future is finished! + finished_.MarkFinished(); + } } Status StartProducing() override { @@ -1593,9 +1519,9 @@ struct HashSemiIndexJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - // DCHECK_EQ(output, outputs_[0]); + DCHECK_EQ(output, outputs_[0]); - if (build_counter_.Cancel() || probe_counter_.Cancel() || out_counter_.Cancel()) { + if (build_counter_.Cancel() || /*probe_counter_.Cancel() ||*/ out_counter_.Cancel()) { finished_.MarkFinished(); } @@ -1614,17 +1540,10 @@ struct HashSemiIndexJoinNode : ExecNode { Future<> finished() override { return finished_; } private: - int output_batch_size() const { - int result = static_cast(ctx_->exec_chunksize()); - if (result < 0) { - result = 32 * 1024; - } - return result; - } struct ThreadLocalState { std::unique_ptr grouper; - std::vector cached_probe_batches{}; + std::vector> cached_probe_batches{}; }; ExecContext* ctx_; @@ -1633,7 +1552,7 @@ struct HashSemiIndexJoinNode : ExecNode { ThreadIndexer get_thread_index_; const std::vector index_field_ids_; - AtomicCounter build_counter_, probe_counter_, out_counter_; + AtomicCounter build_counter_, /*probe_counter_,*/ out_counter_; std::vector local_states_; // need a separate atomic bool to track if the build side complete. Can't use the flag @@ -1641,8 +1560,6 @@ struct HashSemiIndexJoinNode : ExecNode { // all the build batches. So, while merging, we need to prevent probe batches, being // consumed. std::atomic build_side_complete_; - - ExecBatch out_data_; }; } // namespace compute From db68536c5780a72625a511b91b541b0571c242a2 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Fri, 30 Jul 2021 15:46:38 -0400 Subject: [PATCH 06/37] code complete --- cpp/src/arrow/compute/exec/exec_plan.cc | 112 ++++++++++++++++++++---- cpp/src/arrow/compute/exec/exec_plan.h | 10 +++ 2 files changed, 103 insertions(+), 19 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 7b3fed95f46..95bf0493f68 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1312,13 +1312,14 @@ Result GroupByUsingExecPlan(const std::vector& arguments, struct HashSemiJoinNode : ExecNode { HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, - std::shared_ptr output_schema, ExecContext* ctx, - const std::vector&& index_field_ids) + ExecContext* ctx, const std::vector&& build_index_field_ids, + const std::vector&& probe_index_field_ids) : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, - {"hash_join_build", "hash_join_probe"}, std::move(output_schema), + {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), /*num_outputs=*/1), ctx_(ctx), - index_field_ids_(index_field_ids), + build_index_field_ids_(build_index_field_ids), + probe_index_field_ids_(probe_index_field_ids), build_side_complete_(false) {} private: @@ -1329,15 +1330,15 @@ struct HashSemiJoinNode : ExecNode { Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema - auto input_schema = inputs_[0]->output_schema(); + auto build_schema = inputs_[0]->output_schema(); if (state->grouper != nullptr) return Status::OK(); // Build vector of key field data types - std::vector key_descrs(index_field_ids_.size()); - for (size_t i = 0; i < index_field_ids_.size(); ++i) { - auto idx_field_id = index_field_ids_[i]; - key_descrs[i] = ValueDescr(input_schema->field(idx_field_id)->type()); + std::vector key_descrs(build_index_field_ids_.size()); + for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { + auto build_type = build_schema->field(build_index_field_ids_[i])->type(); + key_descrs[i] = ValueDescr(build_type); } // Construct grouper @@ -1369,9 +1370,9 @@ struct HashSemiJoinNode : ExecNode { RETURN_NOT_OK(InitLocalStateIfNeeded(state)); // Create a batch with key columns - std::vector keys(index_field_ids_.size()); - for (size_t i = 0; i < index_field_ids_.size(); ++i) { - keys[i] = batch.values[index_field_ids_[i]]; + std::vector keys(build_index_field_ids_.size()); + for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { + keys[i] = batch.values[build_index_field_ids_[i]]; } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); @@ -1410,9 +1411,9 @@ struct HashSemiJoinNode : ExecNode { auto* grouper = local_states_[0].grouper.get(); // Create a batch with key columns - std::vector keys(index_field_ids_.size()); - for (size_t i = 0; i < index_field_ids_.size(); ++i) { - keys[i] = batch.values[index_field_ids_[i]]; + std::vector keys(probe_index_field_ids_.size()); + for (size_t i = 0; i < probe_index_field_ids_.size(); ++i) { + keys[i] = batch.values[probe_index_field_ids_[i]]; } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); @@ -1466,12 +1467,12 @@ struct HashSemiJoinNode : ExecNode { // if a build input is received when build side is completed, something's wrong! ARROW_DCHECK(!build_side_complete_.load()); - if (ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch)))) return; + ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch))); } else { // probe input batch is received if (build_side_complete_.load()) { // build side done, continue with probing - if (ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch)))) return; + ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); } else { // build side not completed. Cache this batch! - if (ErrorIfNotOk(CacheProbeBatch(thread_index, seq, std::move(batch)))) return; + ErrorIfNotOk(CacheProbeBatch(thread_index, seq, std::move(batch))); } } } @@ -1550,7 +1551,7 @@ struct HashSemiJoinNode : ExecNode { Future<> finished_ = Future<>::MakeFinished(); ThreadIndexer get_thread_index_; - const std::vector index_field_ids_; + const std::vector build_index_field_ids_, probe_index_field_ids_; AtomicCounter build_counter_, /*probe_counter_,*/ out_counter_; std::vector local_states_; @@ -1562,5 +1563,78 @@ struct HashSemiJoinNode : ExecNode { std::atomic build_side_complete_; }; +Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, + const std::vector& left_keys, + const std::vector& right_keys) { + if (left_keys.size() != right_keys.size()) { + return Status::Invalid("left and right key sizes do not match"); + } + + const auto& l_schema = left_input->output_schema(); + const auto& r_schema = right_input->output_schema(); + for (size_t i = 0; i < left_keys.size(); i++) { + auto l_type = l_schema->GetFieldByName(left_keys[i])->type(); + auto r_type = r_schema->GetFieldByName(right_keys[i])->type(); + + if (!l_type->Equals(r_type)) { + return Status::Invalid("build and probe types do not match: " + l_type->ToString() + + "!=" + r_type->ToString()); + } + } + + return Status::OK(); +} + +Result> PopulateKeys(const Schema& schema, + const std::vector& keys) { + std::vector key_field_ids(keys.size()); + // Find input field indices for left key fields + for (size_t i = 0; i < keys.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(schema)); + key_field_ids[i] = match[0]; + } + + return key_field_ids; +} + +Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, + std::string label, + const std::vector& build_keys, + const std::vector& probe_keys) { + RETURN_NOT_OK(ValidateJoinInputs(build_input, probe_input, build_keys, probe_keys)); + + auto build_schema = build_input->output_schema(); + auto probe_schema = probe_input->output_schema(); + + ARROW_ASSIGN_OR_RAISE(auto build_key_ids, PopulateKeys(*build_schema, build_keys)); + ARROW_ASSIGN_OR_RAISE(auto probe_key_ids, PopulateKeys(*probe_schema, probe_keys)); + + // output schema will be probe schema + auto ctx = build_input->plan()->exec_context(); + ExecPlan* plan = build_input->plan(); + + return plan->EmplaceNode(build_input, probe_input, std::move(label), + ctx, std::move(build_key_ids), + std::move(probe_key_ids)); +} + +Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + const std::vector& left_keys, + const std::vector& right_keys) { + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, + left_keys); +} + +Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + const std::vector& left_keys, + const std::vector& right_keys) { + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, + right_keys); +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index fc3af92af4a..1957c857371 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -303,5 +303,15 @@ Result GroupByUsingExecPlan(const std::vector& arguments, const std::vector& aggregates, bool use_threads, ExecContext* ctx); +/// \brief +Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + std::vector left_keys, + std::vector right_keys); +ARROW_EXPORT +Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + const std::vector& left_keys, + const std::vector& right_keys); } // namespace compute } // namespace arrow From 7e06f56679190343e73de4b8cbbd83025cabe52d Mon Sep 17 00:00:00 2001 From: niranda perera Date: Fri, 30 Jul 2021 16:04:53 -0400 Subject: [PATCH 07/37] adding test case dummy --- cpp/src/arrow/compute/exec/exec_plan.cc | 2 +- cpp/src/arrow/compute/exec/plan_test.cc | 28 +++++++++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 95bf0493f68..c2007b2e942 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1351,7 +1351,7 @@ struct HashSemiJoinNode : ExecNode { // cached_probe_batches, because when probing everyone Status BuildSideMerge() { ThreadLocalState* state0 = &local_states_[0]; - for (int i = 1; i < local_states_.size(); ++i) { + for (size_t i = 1; i < local_states_.size(); ++i) { ThreadLocalState* state = &local_states_[i]; ARROW_DCHECK(state); ARROW_DCHECK(state->grouper); diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 101257f5de8..f85b15106f2 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -581,5 +581,33 @@ TEST(ExecPlanExecution, ScalarSourceScalarAggSink) { })))); } +TEST(ExecPlanExecution, SourceHashLeftSemiJoin) { + // TODO (Niranda) add this! + /* for (bool parallel : {false, true}) { + SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); + + auto input = MakeGroupableBatches(*/ + /*multiplicity=*/ /*parallel ? 100 : 1); + +ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + +ASSERT_OK_AND_ASSIGN(auto source, + MakeTestSourceNode(plan.get(), "source", input, + */ + /*parallel=*//*parallel, */ /*slow=*//*false)); +ASSERT_OK_AND_ASSIGN( +auto gby, MakeGroupByNode(source, "gby", */ + /*keys=*//*{"str"}, */ /*targets=*/ /*{"i32"}, +{{"hash_sum", nullptr}})); +auto sink_gen = MakeSinkNode(gby, "sink"); + +ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), +Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( +{int64(), utf8()}, +parallel ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" +: R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); +}*/ +} + } // namespace compute } // namespace arrow From d5f4f4af64513d4e0cc7730f80338d9c3c9177ae Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 3 Aug 2021 00:42:01 -0400 Subject: [PATCH 08/37] adding PR comments --- cpp/src/arrow/compute/api_aggregate.h | 4 +-- cpp/src/arrow/compute/exec/exec_plan.cc | 40 +++++++++++++++---------- cpp/src/arrow/compute/exec/exec_plan.h | 4 +-- 3 files changed, 29 insertions(+), 19 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 506398cb23c..742b013b1b6 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -350,8 +350,8 @@ class ARROW_EXPORT Grouper { virtual Result Consume(const ExecBatch& batch) = 0; /// Finds/ queries the group IDs for the given ExecBatch for every index. Returns the - /// group IDs as an integer array. If a group ID not found, a UINT32_MAX will be - /// added to that index. This is a thread-safe lookup. + /// group IDs as an integer array. If a group ID not found, a null will be added to that + /// index. This is a thread-safe lookup. virtual Result Find(const ExecBatch& batch) const = 0; /// Get current unique keys. May be called multiple times. diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index c2007b2e942..0409e2537ce 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1320,7 +1320,7 @@ struct HashSemiJoinNode : ExecNode { ctx_(ctx), build_index_field_ids_(build_index_field_ids), probe_index_field_ids_(probe_index_field_ids), - build_side_complete_(false) {} + hash_table_built_(false) {} private: struct ThreadLocalState; @@ -1348,7 +1348,8 @@ struct HashSemiJoinNode : ExecNode { } // merge all other groupers to grouper[0]. nothing needs to be done on the - // cached_probe_batches, because when probing everyone + // cached_probe_batches, because when probing everyone. Note: Only one thread + // should execute this out of the pool! Status BuildSideMerge() { ThreadLocalState* state0 = &local_states_[0]; for (size_t i = 1; i < local_states_.size(); ++i) { @@ -1380,11 +1381,13 @@ struct HashSemiJoinNode : ExecNode { ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); if (build_counter_.Increment()) { + // only a single thread would come inside this if-block! + // while incrementing, if the total is reached, merge all the groupers to 0'th one RETURN_NOT_OK(BuildSideMerge()); // enable flag that build side is completed - build_side_complete_.store(true); + hash_table_built_.store(true); // since the build side is completed, consume cached probe batches RETURN_NOT_OK(ConsumeCachedProbeBatches(thread_index)); @@ -1397,10 +1400,12 @@ struct HashSemiJoinNode : ExecNode { ThreadLocalState* state = &local_states_[thread_index]; // TODO (niranda) check if this is the best way to move batches - for (auto cached : state->cached_probe_batches) { - RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + if (!state->cached_probe_batches.empty()) { + for (auto cached : state->cached_probe_batches) { + RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + } + state->cached_probe_batches.clear(); } - state->cached_probe_batches.clear(); return Status::OK(); } @@ -1443,10 +1448,9 @@ struct HashSemiJoinNode : ExecNode { return Status::OK(); } - Status CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { + void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { ThreadLocalState* state = &local_states_[thread_index]; state->cached_probe_batches.emplace_back(seq_num, std::move(batch)); - return Status::OK(); } inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } @@ -1465,14 +1469,18 @@ struct HashSemiJoinNode : ExecNode { if (IsBuildInput(input)) { // build input batch is received // if a build input is received when build side is completed, something's wrong! - ARROW_DCHECK(!build_side_complete_.load()); + ARROW_DCHECK(!hash_table_built_.load()); ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch))); - } else { // probe input batch is received - if (build_side_complete_.load()) { // build side done, continue with probing + } else { // probe input batch is received + if (hash_table_built_.load()) { // build side done, continue with probing + // consume cachedProbeBatches if available (for this thread) + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + + // consume this probe batch ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); } else { // build side not completed. Cache this batch! - ErrorIfNotOk(CacheProbeBatch(thread_index, seq, std::move(batch))); + CacheProbeBatch(thread_index, seq, std::move(batch)); } } } @@ -1522,7 +1530,9 @@ struct HashSemiJoinNode : ExecNode { void StopProducing(ExecNode* output) override { DCHECK_EQ(output, outputs_[0]); - if (build_counter_.Cancel() || /*probe_counter_.Cancel() ||*/ out_counter_.Cancel()) { + if (build_counter_.Cancel()) { + finished_.MarkFinished(); + } else if (out_counter_.Cancel()) { finished_.MarkFinished(); } @@ -1553,14 +1563,14 @@ struct HashSemiJoinNode : ExecNode { ThreadIndexer get_thread_index_; const std::vector build_index_field_ids_, probe_index_field_ids_; - AtomicCounter build_counter_, /*probe_counter_,*/ out_counter_; + AtomicCounter build_counter_, out_counter_; std::vector local_states_; // need a separate atomic bool to track if the build side complete. Can't use the flag // inside the AtomicCounter, because we need to merge the build groupers once we receive // all the build batches. So, while merging, we need to prevent probe batches, being // consumed. - std::atomic build_side_complete_; + std::atomic hash_table_built_; }; Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 1957c857371..829eaf204bd 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -306,8 +306,8 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /// \brief Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, std::string label, - std::vector left_keys, - std::vector right_keys); + const std::vector& left_keys, + const std::vector& right_keys); ARROW_EXPORT Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, std::string label, From cdf277119b07458d26d18bd7ec59556e3d2f6723 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 3 Aug 2021 00:42:11 -0400 Subject: [PATCH 09/37] adding serial test case --- cpp/src/arrow/compute/exec/plan_test.cc | 75 ++++++++++++++++--------- 1 file changed, 50 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index f85b15106f2..fcb84a158db 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -581,32 +581,57 @@ TEST(ExecPlanExecution, ScalarSourceScalarAggSink) { })))); } +void GenerateBatchesFromString(const std::shared_ptr& schema, + const std::vector& json_strings, + BatchesWithSchema* out_batches) { + std::vector descrs; + for (auto&& field : schema->fields()) { + descrs.emplace_back(field->type()); + } + + for (auto&& s : json_strings) { + out_batches->batches.push_back(ExecBatchFromJSON(descrs, s)); + } + + out_batches->schema = schema; +} + TEST(ExecPlanExecution, SourceHashLeftSemiJoin) { - // TODO (Niranda) add this! - /* for (bool parallel : {false, true}) { - SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); - - auto input = MakeGroupableBatches(*/ - /*multiplicity=*/ /*parallel ? 100 : 1); - -ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - -ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", input, - */ - /*parallel=*//*parallel, */ /*slow=*//*false)); -ASSERT_OK_AND_ASSIGN( -auto gby, MakeGroupByNode(source, "gby", */ - /*keys=*//*{"str"}, */ /*targets=*/ /*{"i32"}, -{{"hash_sum", nullptr}})); -auto sink_gen = MakeSinkNode(gby, "sink"); - -ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), -Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( -{int64(), utf8()}, -parallel ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" -: R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); -}*/ + BatchesWithSchema l_batches, r_batches; + + GenerateBatchesFromString(schema({field("l_i32", int32()), field("l_str", utf8())}), + {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", + R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &l_batches); + + GenerateBatchesFromString( + schema({field("r_str", utf8()), field("r_i32", int32())}), + {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, + &r_batches); + + SCOPED_TRACE("serial"); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + ASSERT_OK_AND_ASSIGN(auto l_source, + MakeTestSourceNode(plan.get(), "l_source", l_batches, + /*parallel=*/false, + /*slow=*/false)); + ASSERT_OK_AND_ASSIGN(auto r_source, + MakeTestSourceNode(plan.get(), "r_source", r_batches, + /*parallel=*/false, + /*slow=*/false)); + + ASSERT_OK_AND_ASSIGN( + auto semi_join, + MakeHashLeftSemiJoinNode(l_source, r_source, "l_semi_join", + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); + auto sink_gen = MakeSinkNode(semi_join, "sink"); + + ASSERT_THAT( + StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( + {int64(), utf8()}, R"([[1,"b"], [5,"b"], [6,"c"], [7,"e"], [8,"e"]])")})))); } } // namespace compute From 72c672d1eec2fc1acb8a93a17533c193c6404aa5 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 3 Aug 2021 01:32:54 -0400 Subject: [PATCH 10/37] passing test --- cpp/src/arrow/compute/exec/exec_plan.cc | 45 ++++++++++++++++++++++--- cpp/src/arrow/compute/exec/plan_test.cc | 12 ++++--- 2 files changed, 48 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 0409e2537ce..6ca2566375b 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -17,6 +17,7 @@ #include "arrow/compute/exec/exec_plan.h" +#include #include #include #include @@ -1329,6 +1330,8 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { + std::cout << "init" << "\n"; + // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -1351,11 +1354,15 @@ struct HashSemiJoinNode : ExecNode { // cached_probe_batches, because when probing everyone. Note: Only one thread // should execute this out of the pool! Status BuildSideMerge() { + std::cout << "build side merge" << "\n"; + ThreadLocalState* state0 = &local_states_[0]; for (size_t i = 1; i < local_states_.size(); ++i) { ThreadLocalState* state = &local_states_[i]; ARROW_DCHECK(state); - ARROW_DCHECK(state->grouper); + if (!state->grouper) { + continue; + } ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); state->grouper.reset(); @@ -1367,6 +1374,8 @@ struct HashSemiJoinNode : ExecNode { // total reached at the end of consumption, all the local states will be merged, before // incrementing the total batches Status ConsumeBuildBatch(const size_t thread_index, ExecBatch batch) { + std::cout << "ConsumeBuildBatch " << thread_index << " " << batch.length << "\n"; + auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -1397,6 +1406,8 @@ struct HashSemiJoinNode : ExecNode { } Status ConsumeCachedProbeBatches(const size_t thread_index) { + std::cout << "ConsumeCachedProbeBatches " << thread_index << "\n"; + ThreadLocalState* state = &local_states_[thread_index]; // TODO (niranda) check if this is the best way to move batches @@ -1413,6 +1424,8 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[0] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { + std::cout << "ConsumeProbeBatch " << seq << "\n"; + auto* grouper = local_states_[0].grouper.get(); // Create a batch with key columns @@ -1439,8 +1452,10 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); + std::cout << "output " << seq << " " << out_batch.ToString() << "\n"; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // all values are valid for output + std::cout << "output " << seq << " " << batch.ToString() << "\n"; outputs_[0]->InputReceived(this, seq, std::move(batch)); } @@ -1458,6 +1473,9 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + std::cout << "input received " << IsBuildInput(input) << " " << seq << " " + << batch.length << "\n"; + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); size_t thread_index = get_thread_index_(); @@ -1486,6 +1504,7 @@ struct HashSemiJoinNode : ExecNode { } void ErrorReceived(ExecNode* input, Status error) override { + std::cout << "error received " << error.ToString() << "\n"; DCHECK_EQ(input, inputs_[0]); outputs_[0]->ErrorReceived(this, std::move(error)); @@ -1493,15 +1512,25 @@ struct HashSemiJoinNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { + std::cout << "input finished " << IsBuildInput(input) << " " << num_total << "\n"; + // bail if StopProducing was called if (finished_.is_finished()) return; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); + size_t thread_index = get_thread_index_(); + // set total for build input if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { + // while incrementing, if the total is reached, merge all the groupers to 0'th one + ErrorIfNotOk(BuildSideMerge()); + + // enable flag that build side is completed + hash_table_built_.store(true); + // only build side has completed! so process cached probe batches (of this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches(get_thread_index_())); + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); return; } @@ -1512,11 +1541,14 @@ struct HashSemiJoinNode : ExecNode { // output will be streamed from the probe side. So, they will have the same total. if (out_counter_.SetTotal(num_total)) { // if out_counter has completed, the future is finished! + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); finished_.MarkFinished(); } + outputs_[0]->InputFinished(this, num_total); } Status StartProducing() override { + std::cout << "start prod \n"; finished_ = Future<>::Make(); local_states_.resize(ThreadIndexer::Capacity()); @@ -1528,6 +1560,8 @@ struct HashSemiJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { + std::cout << "stop prod from node\n"; + DCHECK_EQ(output, outputs_[0]); if (build_counter_.Cancel()) { @@ -1543,15 +1577,18 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { + std::cout << "stop prod \n"; for (auto&& output : outputs_) { StopProducing(output); } } - Future<> finished() override { return finished_; } + Future<> finished() override { + std::cout << "finished? " << finished_.is_finished() << "\n"; + return finished_; + } private: - struct ThreadLocalState { std::unique_ptr grouper; std::vector> cached_probe_batches{}; diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index fcb84a158db..d012a5cf771 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -597,7 +597,7 @@ void GenerateBatchesFromString(const std::shared_ptr& schema, } TEST(ExecPlanExecution, SourceHashLeftSemiJoin) { - BatchesWithSchema l_batches, r_batches; + BatchesWithSchema l_batches, r_batches, exp_batches; GenerateBatchesFromString(schema({field("l_i32", int32()), field("l_str", utf8())}), {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", @@ -628,10 +628,12 @@ TEST(ExecPlanExecution, SourceHashLeftSemiJoin) { /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); auto sink_gen = MakeSinkNode(semi_join, "sink"); - ASSERT_THAT( - StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( - {int64(), utf8()}, R"([[1,"b"], [5,"b"], [6,"c"], [7,"e"], [8,"e"]])")})))); + GenerateBatchesFromString( + schema({field("l_i32", int32()), field("l_str", utf8())}), + {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, &exp_batches); + + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); } } // namespace compute From 8816d9a1c8d61bf72ae8816d0d127c2d35a48e51 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 3 Aug 2021 16:52:46 -0400 Subject: [PATCH 11/37] refactoring files --- cpp/src/arrow/CMakeLists.txt | 3 +- cpp/src/arrow/compute/exec/CMakeLists.txt | 1 + cpp/src/arrow/compute/exec/exec_plan.cc | 444 +----------------- cpp/src/arrow/compute/exec/exec_plan.h | 26 +- cpp/src/arrow/compute/exec/exec_utils.cc | 78 +++ .../exec/{hash_join.h => exec_utils.h} | 47 +- cpp/src/arrow/compute/exec/hash_join.cc | 412 +++++++++++++++- cpp/src/arrow/compute/exec/hash_join_test.cc | 88 ++++ cpp/src/arrow/compute/exec/plan_test.cc | 142 ------ cpp/src/arrow/compute/exec/test_util.cc | 75 +++ cpp/src/arrow/compute/exec/test_util.h | 22 + 11 files changed, 737 insertions(+), 601 deletions(-) create mode 100644 cpp/src/arrow/compute/exec/exec_utils.cc rename cpp/src/arrow/compute/exec/{hash_join.h => exec_utils.h} (50%) create mode 100644 cpp/src/arrow/compute/exec/hash_join_test.cc diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 8f898caa7ed..f9bb36f4a6f 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -371,7 +371,8 @@ if(ARROW_COMPUTE) compute/cast.cc compute/exec.cc compute/exec/exec_plan.cc - compute/exec/expression.cc + compute/exec/exec_utils.cc + compute/exec/expression.cc compute/function.cc compute/function_internal.cc compute/kernel.cc diff --git a/cpp/src/arrow/compute/exec/CMakeLists.txt b/cpp/src/arrow/compute/exec/CMakeLists.txt index 2ed8b1c9480..281154e3518 100644 --- a/cpp/src/arrow/compute/exec/CMakeLists.txt +++ b/cpp/src/arrow/compute/exec/CMakeLists.txt @@ -25,5 +25,6 @@ add_arrow_compute_test(expression_test subtree_test.cc) add_arrow_compute_test(plan_test PREFIX "arrow-compute") +add_arrow_compute_test(hash_join_test PREFIX "arrow-compute") add_arrow_benchmark(expression_benchmark PREFIX "arrow-compute") diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 6ca2566375b..85992f3e87a 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -27,6 +27,7 @@ #include "arrow/array/util.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" +#include "arrow/compute/exec/exec_utils.h" #include "arrow/compute/exec/expression.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/registry.h" @@ -530,49 +531,6 @@ Result MakeProjectNode(ExecNode* input, std::string label, input, std::move(label), schema(std::move(fields)), std::move(exprs)); } -class AtomicCounter { - public: - AtomicCounter() = default; - - int count() const { return count_.load(); } - - util::optional total() const { - int total = total_.load(); - if (total == -1) return {}; - return total; - } - - bool TotalReached() { return complete_.load(); } - - // return true if the counter is complete - bool Increment() { - DCHECK_NE(count_.load(), total_.load()); - int count = count_.fetch_add(1) + 1; - if (count != total_.load()) return false; - return DoneOnce(); - } - - // return true if the counter is complete - bool SetTotal(int total) { - total_.store(total); - if (count_.load() != total) return false; - return DoneOnce(); - } - - // return true if the counter has not already been completed - bool Cancel() { return DoneOnce(); } - - private: - // ensure there is only one true return from Increment(), SetTotal(), or Cancel() - bool DoneOnce() { - bool expected = false; - return complete_.compare_exchange_strong(expected, true); - } - - std::atomic count_{0}, total_{-1}; - std::atomic complete_{false}; -}; - struct SinkNode : ExecNode { SinkNode(ExecNode* input, std::string label, AsyncGenerator>* generator) @@ -690,34 +648,6 @@ std::shared_ptr MakeGeneratorReader( return out; } -class ThreadIndexer { - public: - size_t operator()() { - auto id = std::this_thread::get_id(); - - std::unique_lock lock(mutex_); - const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; - - return Check(id_index.second); - } - - static size_t Capacity() { - static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); - return max_size; - } - - private: - size_t Check(size_t thread_index) { - DCHECK_LT(thread_index, Capacity()) << "thread index " << thread_index - << " is out of range [0, " << Capacity() << ")"; - - return thread_index; - } - - std::mutex mutex_; - std::unordered_map id_to_index_; -}; - struct ScalarAggregateNode : ExecNode { ScalarAggregateNode(ExecNode* input, std::string label, std::shared_ptr output_schema, @@ -1311,377 +1241,5 @@ Result GroupByUsingExecPlan(const std::vector& arguments, /*null_count=*/0); } -struct HashSemiJoinNode : ExecNode { - HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, - ExecContext* ctx, const std::vector&& build_index_field_ids, - const std::vector&& probe_index_field_ids) - : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, - {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), - /*num_outputs=*/1), - ctx_(ctx), - build_index_field_ids_(build_index_field_ids), - probe_index_field_ids_(probe_index_field_ids), - hash_table_built_(false) {} - - private: - struct ThreadLocalState; - - public: - const char* kind_name() override { return "HashSemiJoinNode"; } - - Status InitLocalStateIfNeeded(ThreadLocalState* state) { - std::cout << "init" << "\n"; - - // Get input schema - auto build_schema = inputs_[0]->output_schema(); - - if (state->grouper != nullptr) return Status::OK(); - - // Build vector of key field data types - std::vector key_descrs(build_index_field_ids_.size()); - for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { - auto build_type = build_schema->field(build_index_field_ids_[i])->type(); - key_descrs[i] = ValueDescr(build_type); - } - - // Construct grouper - ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); - - return Status::OK(); - } - - // merge all other groupers to grouper[0]. nothing needs to be done on the - // cached_probe_batches, because when probing everyone. Note: Only one thread - // should execute this out of the pool! - Status BuildSideMerge() { - std::cout << "build side merge" << "\n"; - - ThreadLocalState* state0 = &local_states_[0]; - for (size_t i = 1; i < local_states_.size(); ++i) { - ThreadLocalState* state = &local_states_[i]; - ARROW_DCHECK(state); - if (!state->grouper) { - continue; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); - state->grouper.reset(); - } - return Status::OK(); - } - - // consumes a build batch and increments the build_batches count. if the build batches - // total reached at the end of consumption, all the local states will be merged, before - // incrementing the total batches - Status ConsumeBuildBatch(const size_t thread_index, ExecBatch batch) { - std::cout << "ConsumeBuildBatch " << thread_index << " " << batch.length << "\n"; - - auto state = &local_states_[thread_index]; - RETURN_NOT_OK(InitLocalStateIfNeeded(state)); - - // Create a batch with key columns - std::vector keys(build_index_field_ids_.size()); - for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { - keys[i] = batch.values[build_index_field_ids_[i]]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - - // Create a batch with group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); - - if (build_counter_.Increment()) { - // only a single thread would come inside this if-block! - - // while incrementing, if the total is reached, merge all the groupers to 0'th one - RETURN_NOT_OK(BuildSideMerge()); - - // enable flag that build side is completed - hash_table_built_.store(true); - - // since the build side is completed, consume cached probe batches - RETURN_NOT_OK(ConsumeCachedProbeBatches(thread_index)); - } - - return Status::OK(); - } - - Status ConsumeCachedProbeBatches(const size_t thread_index) { - std::cout << "ConsumeCachedProbeBatches " << thread_index << "\n"; - - ThreadLocalState* state = &local_states_[thread_index]; - - // TODO (niranda) check if this is the best way to move batches - if (!state->cached_probe_batches.empty()) { - for (auto cached : state->cached_probe_batches) { - RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); - } - state->cached_probe_batches.clear(); - } - - return Status::OK(); - } - - // consumes a probe batch and increment probe batches count. Probing would query the - // grouper[0] which have been merged with all others. - Status ConsumeProbeBatch(int seq, ExecBatch batch) { - std::cout << "ConsumeProbeBatch " << seq << "\n"; - - auto* grouper = local_states_[0].grouper.get(); - - // Create a batch with key columns - std::vector keys(probe_index_field_ids_.size()); - for (size_t i = 0; i < probe_index_field_ids_.size(); ++i) { - keys[i] = batch.values[probe_index_field_ids_[i]]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - - // Query the grouper with key_batch. If no match was found, returning group_ids would - // have null. - ARROW_ASSIGN_OR_RAISE(Datum group_ids, grouper->Find(key_batch)); - auto group_ids_data = *group_ids.array(); - - if (group_ids_data.MayHaveNulls()) { // values need to be filtered - auto filter_arr = - std::make_shared(group_ids_data.length, group_ids_data.buffers[0], - /*null_bitmap=*/nullptr, /*null_count=*/0, - /*offset=*/group_ids_data.offset); - ARROW_ASSIGN_OR_RAISE(auto rec_batch, - batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); - ARROW_ASSIGN_OR_RAISE( - auto filtered, - Filter(rec_batch, filter_arr, - /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); - auto out_batch = ExecBatch(*filtered.record_batch()); - std::cout << "output " << seq << " " << out_batch.ToString() << "\n"; - outputs_[0]->InputReceived(this, seq, std::move(out_batch)); - } else { // all values are valid for output - std::cout << "output " << seq << " " << batch.ToString() << "\n"; - outputs_[0]->InputReceived(this, seq, std::move(batch)); - } - - out_counter_.Increment(); - return Status::OK(); - } - - void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { - ThreadLocalState* state = &local_states_[thread_index]; - state->cached_probe_batches.emplace_back(seq_num, std::move(batch)); - } - - inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } - - // If all build side batches received? continue streaming using probing - // else cache the batches in thread-local state - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - std::cout << "input received " << IsBuildInput(input) << " " << seq << " " - << batch.length << "\n"; - - ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - - size_t thread_index = get_thread_index_(); - ARROW_DCHECK(thread_index < local_states_.size()); - - if (finished_.is_finished()) { - return; - } - - if (IsBuildInput(input)) { // build input batch is received - // if a build input is received when build side is completed, something's wrong! - ARROW_DCHECK(!hash_table_built_.load()); - - ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch))); - } else { // probe input batch is received - if (hash_table_built_.load()) { // build side done, continue with probing - // consume cachedProbeBatches if available (for this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); - - // consume this probe batch - ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); - } else { // build side not completed. Cache this batch! - CacheProbeBatch(thread_index, seq, std::move(batch)); - } - } - } - - void ErrorReceived(ExecNode* input, Status error) override { - std::cout << "error received " << error.ToString() << "\n"; - DCHECK_EQ(input, inputs_[0]); - - outputs_[0]->ErrorReceived(this, std::move(error)); - StopProducing(); - } - - void InputFinished(ExecNode* input, int num_total) override { - std::cout << "input finished " << IsBuildInput(input) << " " << num_total << "\n"; - - // bail if StopProducing was called - if (finished_.is_finished()) return; - - ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - - size_t thread_index = get_thread_index_(); - - // set total for build input - if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { - // while incrementing, if the total is reached, merge all the groupers to 0'th one - ErrorIfNotOk(BuildSideMerge()); - - // enable flag that build side is completed - hash_table_built_.store(true); - - // only build side has completed! so process cached probe batches (of this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); - return; - } - - // set total for probe input. If it returns that probe side has completed, nothing to - // do, because probing inputs will be streamed to the output - // probe_counter_.SetTotal(num_total); - - // output will be streamed from the probe side. So, they will have the same total. - if (out_counter_.SetTotal(num_total)) { - // if out_counter has completed, the future is finished! - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); - finished_.MarkFinished(); - } - outputs_[0]->InputFinished(this, num_total); - } - - Status StartProducing() override { - std::cout << "start prod \n"; - finished_ = Future<>::Make(); - - local_states_.resize(ThreadIndexer::Capacity()); - return Status::OK(); - } - - void PauseProducing(ExecNode* output) override {} - - void ResumeProducing(ExecNode* output) override {} - - void StopProducing(ExecNode* output) override { - std::cout << "stop prod from node\n"; - - DCHECK_EQ(output, outputs_[0]); - - if (build_counter_.Cancel()) { - finished_.MarkFinished(); - } else if (out_counter_.Cancel()) { - finished_.MarkFinished(); - } - - for (auto&& input : inputs_) { - input->StopProducing(this); - } - } - - // TODO(niranda) couldn't there be multiple outputs for a Node? - void StopProducing() override { - std::cout << "stop prod \n"; - for (auto&& output : outputs_) { - StopProducing(output); - } - } - - Future<> finished() override { - std::cout << "finished? " << finished_.is_finished() << "\n"; - return finished_; - } - - private: - struct ThreadLocalState { - std::unique_ptr grouper; - std::vector> cached_probe_batches{}; - }; - - ExecContext* ctx_; - Future<> finished_ = Future<>::MakeFinished(); - - ThreadIndexer get_thread_index_; - const std::vector build_index_field_ids_, probe_index_field_ids_; - - AtomicCounter build_counter_, out_counter_; - std::vector local_states_; - - // need a separate atomic bool to track if the build side complete. Can't use the flag - // inside the AtomicCounter, because we need to merge the build groupers once we receive - // all the build batches. So, while merging, we need to prevent probe batches, being - // consumed. - std::atomic hash_table_built_; -}; - -Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, - const std::vector& left_keys, - const std::vector& right_keys) { - if (left_keys.size() != right_keys.size()) { - return Status::Invalid("left and right key sizes do not match"); - } - - const auto& l_schema = left_input->output_schema(); - const auto& r_schema = right_input->output_schema(); - for (size_t i = 0; i < left_keys.size(); i++) { - auto l_type = l_schema->GetFieldByName(left_keys[i])->type(); - auto r_type = r_schema->GetFieldByName(right_keys[i])->type(); - - if (!l_type->Equals(r_type)) { - return Status::Invalid("build and probe types do not match: " + l_type->ToString() + - "!=" + r_type->ToString()); - } - } - - return Status::OK(); -} - -Result> PopulateKeys(const Schema& schema, - const std::vector& keys) { - std::vector key_field_ids(keys.size()); - // Find input field indices for left key fields - for (size_t i = 0; i < keys.size(); ++i) { - ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(schema)); - key_field_ids[i] = match[0]; - } - - return key_field_ids; -} - -Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, - std::string label, - const std::vector& build_keys, - const std::vector& probe_keys) { - RETURN_NOT_OK(ValidateJoinInputs(build_input, probe_input, build_keys, probe_keys)); - - auto build_schema = build_input->output_schema(); - auto probe_schema = probe_input->output_schema(); - - ARROW_ASSIGN_OR_RAISE(auto build_key_ids, PopulateKeys(*build_schema, build_keys)); - ARROW_ASSIGN_OR_RAISE(auto probe_key_ids, PopulateKeys(*probe_schema, probe_keys)); - - // output schema will be probe schema - auto ctx = build_input->plan()->exec_context(); - ExecPlan* plan = build_input->plan(); - - return plan->EmplaceNode(build_input, probe_input, std::move(label), - ctx, std::move(build_key_ids), - std::move(probe_key_ids)); -} - -Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys) { - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, - left_keys); -} - -Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys) { - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, - right_keys); -} - } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 829eaf204bd..ec62ca45964 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -303,15 +303,23 @@ Result GroupByUsingExecPlan(const std::vector& arguments, const std::vector& aggregates, bool use_threads, ExecContext* ctx); -/// \brief -Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys); +/// \brief Make a node which joins batches from two other nodes based on key fields +enum JoinType { + LEFT_SEMI, + RIGHT_SEMI, + LEFT_ANTI, + RIGHT_ANTI, + INNER, // Not Implemented + LEFT_OUTER, // Not Implemented + RIGHT_OUTER, // Not Implemented + FULL_OUTER // Not Implemented +}; + ARROW_EXPORT -Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys); +Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, + ExecNode* right_input, std::string label, + const std::vector& left_keys, + const std::vector& right_keys); + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_utils.cc b/cpp/src/arrow/compute/exec/exec_utils.cc new file mode 100644 index 00000000000..f1a96ac0812 --- /dev/null +++ b/cpp/src/arrow/compute/exec/exec_utils.cc @@ -0,0 +1,78 @@ +// 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/exec_utils.h" + +#include + +namespace arrow { +namespace compute { + +size_t ThreadIndexer::operator()() { + auto id = std::this_thread::get_id(); + + std::unique_lock lock(mutex_); + const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; + + return Check(id_index.second); +} + +size_t ThreadIndexer::Capacity() { + static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); + return max_size; +} + +size_t ThreadIndexer::Check(size_t thread_index) { + DCHECK_LT(thread_index, Capacity()) + << "thread index " << thread_index << " is out of range [0, " << Capacity() << ")"; + + return thread_index; +} + +int AtomicCounter::count() const { return count_.load(); } + +util::optional AtomicCounter::total() const { + int total = total_.load(); + if (total == -1) return {}; + return total; +} + +bool AtomicCounter::Increment() { + DCHECK_NE(count_.load(), total_.load()); + int count = count_.fetch_add(1) + 1; + if (count != total_.load()) return false; + return DoneOnce(); +} + +// return true if the counter is complete +bool AtomicCounter::SetTotal(int total) { + total_.store(total); + if (count_.load() != total) return false; + return DoneOnce(); +} + +// return true if the counter has not already been completed +bool AtomicCounter::Cancel() { return DoneOnce(); } + +// ensure there is only one true return from Increment(), SetTotal(), or Cancel() +bool AtomicCounter::DoneOnce() { + bool expected = false; + return complete_.compare_exchange_strong(expected, true); +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/compute/exec/exec_utils.h similarity index 50% rename from cpp/src/arrow/compute/exec/hash_join.h rename to cpp/src/arrow/compute/exec/exec_utils.h index 492cf0a0a49..c899e3b3058 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/compute/exec/exec_utils.h @@ -15,14 +15,51 @@ // specific language governing permissions and limitations // under the License. +#include +#include +#include + +#include "arrow/util/thread_pool.h" + namespace arrow { namespace compute { -enum JoinType { - LEFT_SEMI_JOIN, - RIGHT_SEMI_JOIN, - LEFT_ANTI_SEMI_JOIN, - RIGHT_ANTI_SEMI_JOIN +class ThreadIndexer { + public: + size_t operator()(); + + static size_t Capacity(); + + private: + static size_t Check(size_t thread_index); + + std::mutex mutex_; + std::unordered_map id_to_index_; +}; + +class AtomicCounter { + public: + AtomicCounter() = default; + + int count() const; + + util::optional total() const; + + // return true if the counter is complete + bool Increment(); + + // return true if the counter is complete + bool SetTotal(int total); + + // return true if the counter has not already been completed + bool Cancel(); + + private: + // ensure there is only one true return from Increment(), SetTotal(), or Cancel() + bool DoneOnce(); + + std::atomic count_{0}, total_{-1}; + std::atomic complete_{false}; }; } // namespace compute diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 10b676be892..ba7a7646b6f 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -15,11 +15,421 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/hash_join.h" +#include +#include +#include + +#include + +#include "arrow/compute/exec/exec_plan.h" +#include "arrow/compute/exec/exec_utils.h" namespace arrow { namespace compute { +struct HashSemiJoinNode : ExecNode { + HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, + ExecContext* ctx, const std::vector&& build_index_field_ids, + const std::vector&& probe_index_field_ids) + : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, + {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), + /*num_outputs=*/1), + ctx_(ctx), + build_index_field_ids_(build_index_field_ids), + probe_index_field_ids_(probe_index_field_ids), + hash_table_built_(false) {} + + private: + struct ThreadLocalState; + + public: + const char* kind_name() override { return "HashSemiJoinNode"; } + + Status InitLocalStateIfNeeded(ThreadLocalState* state) { + std::cout << "init" + << "\n"; + + // Get input schema + auto build_schema = inputs_[0]->output_schema(); + + if (state->grouper != nullptr) return Status::OK(); + + // Build vector of key field data types + std::vector key_descrs(build_index_field_ids_.size()); + for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { + auto build_type = build_schema->field(build_index_field_ids_[i])->type(); + key_descrs[i] = ValueDescr(build_type); + } + + // Construct grouper + ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + + return Status::OK(); + } + + // merge all other groupers to grouper[0]. nothing needs to be done on the + // cached_probe_batches, because when probing everyone. Note: Only one thread + // should execute this out of the pool! + Status BuildSideMerge() { + std::cout << "build side merge" + << "\n"; + + ThreadLocalState* state0 = &local_states_[0]; + for (size_t i = 1; i < local_states_.size(); ++i) { + ThreadLocalState* state = &local_states_[i]; + ARROW_DCHECK(state); + if (!state->grouper) { + continue; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); + state->grouper.reset(); + } + return Status::OK(); + } + + // consumes a build batch and increments the build_batches count. if the build batches + // total reached at the end of consumption, all the local states will be merged, before + // incrementing the total batches + Status ConsumeBuildBatch(const size_t thread_index, ExecBatch batch) { + std::cout << "ConsumeBuildBatch " << thread_index << " " << batch.length << "\n"; + + auto state = &local_states_[thread_index]; + RETURN_NOT_OK(InitLocalStateIfNeeded(state)); + + // Create a batch with key columns + std::vector keys(build_index_field_ids_.size()); + for (size_t i = 0; i < build_index_field_ids_.size(); ++i) { + keys[i] = batch.values[build_index_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); + + if (build_counter_.Increment()) { + // only a single thread would come inside this if-block! + + // while incrementing, if the total is reached, merge all the groupers to 0'th one + RETURN_NOT_OK(BuildSideMerge()); + + // enable flag that build side is completed + hash_table_built_.store(true); + + // since the build side is completed, consume cached probe batches + RETURN_NOT_OK(ConsumeCachedProbeBatches(thread_index)); + } + + return Status::OK(); + } + + Status ConsumeCachedProbeBatches(const size_t thread_index) { + std::cout << "ConsumeCachedProbeBatches " << thread_index << "\n"; + + ThreadLocalState* state = &local_states_[thread_index]; + + // TODO (niranda) check if this is the best way to move batches + if (!state->cached_probe_batches.empty()) { + for (auto cached : state->cached_probe_batches) { + RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + } + state->cached_probe_batches.clear(); + } + + return Status::OK(); + } + + // consumes a probe batch and increment probe batches count. Probing would query the + // grouper[0] which have been merged with all others. + Status ConsumeProbeBatch(int seq, ExecBatch batch) { + std::cout << "ConsumeProbeBatch " << seq << "\n"; + + auto* grouper = local_states_[0].grouper.get(); + + // Create a batch with key columns + std::vector keys(probe_index_field_ids_.size()); + for (size_t i = 0; i < probe_index_field_ids_.size(); ++i) { + keys[i] = batch.values[probe_index_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + + // Query the grouper with key_batch. If no match was found, returning group_ids would + // have null. + ARROW_ASSIGN_OR_RAISE(Datum group_ids, grouper->Find(key_batch)); + auto group_ids_data = *group_ids.array(); + + if (group_ids_data.MayHaveNulls()) { // values need to be filtered + auto filter_arr = + std::make_shared(group_ids_data.length, group_ids_data.buffers[0], + /*null_bitmap=*/nullptr, /*null_count=*/0, + /*offset=*/group_ids_data.offset); + ARROW_ASSIGN_OR_RAISE(auto rec_batch, + batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); + ARROW_ASSIGN_OR_RAISE( + auto filtered, + Filter(rec_batch, filter_arr, + /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); + auto out_batch = ExecBatch(*filtered.record_batch()); + std::cout << "output " << seq << " " << out_batch.ToString() << "\n"; + outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + } else { // all values are valid for output + std::cout << "output " << seq << " " << batch.ToString() << "\n"; + outputs_[0]->InputReceived(this, seq, std::move(batch)); + } + + out_counter_.Increment(); + return Status::OK(); + } + + void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { + ThreadLocalState* state = &local_states_[thread_index]; + state->cached_probe_batches.emplace_back(seq_num, std::move(batch)); + } + + inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } + + // If all build side batches received? continue streaming using probing + // else cache the batches in thread-local state + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + std::cout << "input received " << IsBuildInput(input) << " " << seq << " " + << batch.length << "\n"; + + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); + + size_t thread_index = get_thread_index_(); + ARROW_DCHECK(thread_index < local_states_.size()); + + if (finished_.is_finished()) { + return; + } + + if (IsBuildInput(input)) { // build input batch is received + // if a build input is received when build side is completed, something's wrong! + ARROW_DCHECK(!hash_table_built_.load()); + + ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch))); + } else { // probe input batch is received + if (hash_table_built_.load()) { // build side done, continue with probing + // consume cachedProbeBatches if available (for this thread) + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + + // consume this probe batch + ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); + } else { // build side not completed. Cache this batch! + CacheProbeBatch(thread_index, seq, std::move(batch)); + } + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + std::cout << "error received " << error.ToString() << "\n"; + DCHECK_EQ(input, inputs_[0]); + + outputs_[0]->ErrorReceived(this, std::move(error)); + StopProducing(); + } + + void InputFinished(ExecNode* input, int num_total) override { + std::cout << "input finished " << IsBuildInput(input) << " " << num_total << "\n"; + + // bail if StopProducing was called + if (finished_.is_finished()) return; + + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); + + size_t thread_index = get_thread_index_(); + + // set total for build input + if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { + // while incrementing, if the total is reached, merge all the groupers to 0'th one + ErrorIfNotOk(BuildSideMerge()); + + // enable flag that build side is completed + hash_table_built_.store(true); + + // only build side has completed! so process cached probe batches (of this thread) + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + return; + } + + // set total for probe input. If it returns that probe side has completed, nothing to + // do, because probing inputs will be streamed to the output + // probe_counter_.SetTotal(num_total); + + // output will be streamed from the probe side. So, they will have the same total. + if (out_counter_.SetTotal(num_total)) { + // if out_counter has completed, the future is finished! + ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + finished_.MarkFinished(); + } + outputs_[0]->InputFinished(this, num_total); + } + + Status StartProducing() override { + std::cout << "start prod \n"; + finished_ = Future<>::Make(); + + local_states_.resize(ThreadIndexer::Capacity()); + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + std::cout << "stop prod from node\n"; + + DCHECK_EQ(output, outputs_[0]); + + if (build_counter_.Cancel()) { + finished_.MarkFinished(); + } else if (out_counter_.Cancel()) { + finished_.MarkFinished(); + } + + for (auto&& input : inputs_) { + input->StopProducing(this); + } + } + + // TODO(niranda) couldn't there be multiple outputs for a Node? + void StopProducing() override { + std::cout << "stop prod \n"; + for (auto&& output : outputs_) { + StopProducing(output); + } + } + + Future<> finished() override { + std::cout << "finished? " << finished_.is_finished() << "\n"; + return finished_; + } + + private: + struct ThreadLocalState { + std::unique_ptr grouper; + std::vector> cached_probe_batches{}; + }; + + ExecContext* ctx_; + Future<> finished_ = Future<>::MakeFinished(); + + ThreadIndexer get_thread_index_; + const std::vector build_index_field_ids_, probe_index_field_ids_; + + AtomicCounter build_counter_, out_counter_; + std::vector local_states_; + + // need a separate atomic bool to track if the build side complete. Can't use the flag + // inside the AtomicCounter, because we need to merge the build groupers once we receive + // all the build batches. So, while merging, we need to prevent probe batches, being + // consumed. + std::atomic hash_table_built_; +}; + +Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, + const std::vector& left_keys, + const std::vector& right_keys) { + if (left_keys.size() != right_keys.size()) { + return Status::Invalid("left and right key sizes do not match"); + } + + const auto& l_schema = left_input->output_schema(); + const auto& r_schema = right_input->output_schema(); + for (size_t i = 0; i < left_keys.size(); i++) { + auto l_type = l_schema->GetFieldByName(left_keys[i])->type(); + auto r_type = r_schema->GetFieldByName(right_keys[i])->type(); + + if (!l_type->Equals(r_type)) { + return Status::Invalid("build and probe types do not match: " + l_type->ToString() + + "!=" + r_type->ToString()); + } + } + + return Status::OK(); +} + +Result> PopulateKeys(const Schema& schema, + const std::vector& keys) { + std::vector key_field_ids(keys.size()); + // Find input field indices for left key fields + for (size_t i = 0; i < keys.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(schema)); + key_field_ids[i] = match[0]; + } + + return key_field_ids; +} + +Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, + std::string label, + const std::vector& build_keys, + const std::vector& probe_keys) { + RETURN_NOT_OK(ValidateJoinInputs(build_input, probe_input, build_keys, probe_keys)); + + auto build_schema = build_input->output_schema(); + auto probe_schema = probe_input->output_schema(); + + ARROW_ASSIGN_OR_RAISE(auto build_key_ids, PopulateKeys(*build_schema, build_keys)); + ARROW_ASSIGN_OR_RAISE(auto probe_key_ids, PopulateKeys(*probe_schema, probe_keys)); + + // output schema will be probe schema + auto ctx = build_input->plan()->exec_context(); + ExecPlan* plan = build_input->plan(); + + return plan->EmplaceNode(build_input, probe_input, std::move(label), + ctx, std::move(build_key_ids), + std::move(probe_key_ids)); +} + +Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + const std::vector& left_keys, + const std::vector& right_keys) { + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, + left_keys); +} + +Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, + std::string label, + const std::vector& left_keys, + const std::vector& right_keys) { + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, + right_keys); +} + +static std::string JoinTypeToString[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", + "RIGHT_ANTI", "INNER", "LEFT_OUTER", + "RIGHT_OUTER", "FULL_OUTER"}; + +Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, + ExecNode* right_input, std::string label, + const std::vector& left_keys, + const std::vector& right_keys) { + switch (join_type) { + case LEFT_SEMI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, + left_keys); + case RIGHT_SEMI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, + right_keys); + case LEFT_ANTI: + case RIGHT_ANTI: + case INNER: + case LEFT_OUTER: + case RIGHT_OUTER: + case FULL_OUTER: + return Status::NotImplemented(JoinTypeToString[join_type] + + " joins not implemented!"); + default: + return Status::Invalid("invalid join type"); + } +} } // namespace compute } // namespace arrow \ No newline at end of file diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_test.cc new file mode 100644 index 00000000000..601e7fee8fa --- /dev/null +++ b/cpp/src/arrow/compute/exec/hash_join_test.cc @@ -0,0 +1,88 @@ +// 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 + +#include "arrow/api.h" +#include "arrow/compute/exec/test_util.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/matchers.h" + +using testing::UnorderedElementsAreArray; + +namespace arrow { +namespace compute { + +void GenerateBatchesFromString(const std::shared_ptr& schema, + const std::vector& json_strings, + BatchesWithSchema* out_batches) { + std::vector descrs; + for (auto&& field : schema->fields()) { + descrs.emplace_back(field->type()); + } + + for (auto&& s : json_strings) { + out_batches->batches.push_back(ExecBatchFromJSON(descrs, s)); + } + + out_batches->schema = schema; +} + +TEST(HashJoin, LeftSemi) { + auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); + auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); + BatchesWithSchema l_batches, r_batches, exp_batches; + + GenerateBatchesFromString(l_schema, + {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", + R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &l_batches); + + GenerateBatchesFromString( + r_schema, + {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, + &r_batches); + + SCOPED_TRACE("serial"); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + ASSERT_OK_AND_ASSIGN(auto l_source, + MakeTestSourceNode(plan.get(), "l_source", l_batches, + /*parallel=*/false, + /*slow=*/false)); + ASSERT_OK_AND_ASSIGN(auto r_source, + MakeTestSourceNode(plan.get(), "r_source", r_batches, + /*parallel=*/false, + /*slow=*/false)); + + ASSERT_OK_AND_ASSIGN( + auto semi_join, + MakeHashJoinNode(JoinType::LEFT_SEMI, l_source, r_source, "l_semi_join", + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); + auto sink_gen = MakeSinkNode(semi_join, "sink"); + + GenerateBatchesFromString( + l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &exp_batches); + + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); +} + +} // namespace compute +} // namespace arrow \ No newline at end of file diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index d012a5cf771..5f3809db6c7 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -18,21 +18,16 @@ #include #include -#include #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/expression.h" #include "arrow/compute/exec/test_util.h" #include "arrow/record_batch.h" -#include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" -#include "arrow/testing/random.h" #include "arrow/util/async_generator.h" #include "arrow/util/logging.h" -#include "arrow/util/thread_pool.h" -#include "arrow/util/vector.h" using testing::ElementsAre; using testing::HasSubstr; @@ -194,88 +189,6 @@ TEST(ExecPlan, DummyStartProducingError) { ASSERT_THAT(t.stopped, ElementsAre("process2", "process3", "sink")); } -namespace { - -struct BatchesWithSchema { - std::vector batches; - std::shared_ptr schema; -}; - -Result MakeTestSourceNode(ExecPlan* plan, std::string label, - BatchesWithSchema batches_with_schema, bool parallel, - bool slow) { - DCHECK_GT(batches_with_schema.batches.size(), 0); - - auto opt_batches = ::arrow::internal::MapVector( - [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, - std::move(batches_with_schema.batches)); - - AsyncGenerator> gen; - - if (parallel) { - // emulate batches completing initial decode-after-scan on a cpu thread - ARROW_ASSIGN_OR_RAISE( - gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - ::arrow::internal::GetCpuThreadPool())); - - // ensure that callbacks are not executed immediately on a background thread - gen = MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); - } else { - gen = MakeVectorGenerator(std::move(opt_batches)); - } - - if (slow) { - gen = MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { - SleepABit(); - return batch; - }); - } - - return MakeSourceNode(plan, label, std::move(batches_with_schema.schema), - std::move(gen)); -} - -Future> StartAndCollect( - ExecPlan* plan, AsyncGenerator> gen) { - RETURN_NOT_OK(plan->Validate()); - RETURN_NOT_OK(plan->StartProducing()); - - auto collected_fut = CollectAsyncGenerator(gen); - - return AllComplete({plan->finished(), Future<>(collected_fut)}) - .Then([collected_fut]() -> Result> { - ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); - return ::arrow::internal::MapVector( - [](util::optional batch) { return std::move(*batch); }, - std::move(collected)); - }); -} - -BatchesWithSchema MakeBasicBatches() { - BatchesWithSchema out; - out.batches = { - ExecBatchFromJSON({int32(), boolean()}, "[[null, true], [4, false]]"), - ExecBatchFromJSON({int32(), boolean()}, "[[5, null], [6, false], [7, false]]")}; - out.schema = schema({field("i32", int32()), field("bool", boolean())}); - return out; -} - -BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, - int num_batches = 10, int batch_size = 4) { - BatchesWithSchema out; - - random::RandomArrayGenerator rng(42); - out.batches.resize(num_batches); - - for (int i = 0; i < num_batches; ++i) { - out.batches[i] = ExecBatch(*rng.BatchOf(schema->fields(), batch_size)); - // add a tag scalar to ensure the batches are unique - out.batches[i].values.emplace_back(i); - } - return out; -} -} // namespace - TEST(ExecPlanExecution, SourceSink) { for (bool slow : {false, true}) { SCOPED_TRACE(slow ? "slowed" : "unslowed"); @@ -581,60 +494,5 @@ TEST(ExecPlanExecution, ScalarSourceScalarAggSink) { })))); } -void GenerateBatchesFromString(const std::shared_ptr& schema, - const std::vector& json_strings, - BatchesWithSchema* out_batches) { - std::vector descrs; - for (auto&& field : schema->fields()) { - descrs.emplace_back(field->type()); - } - - for (auto&& s : json_strings) { - out_batches->batches.push_back(ExecBatchFromJSON(descrs, s)); - } - - out_batches->schema = schema; -} - -TEST(ExecPlanExecution, SourceHashLeftSemiJoin) { - BatchesWithSchema l_batches, r_batches, exp_batches; - - GenerateBatchesFromString(schema({field("l_i32", int32()), field("l_str", utf8())}), - {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", - R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &l_batches); - - GenerateBatchesFromString( - schema({field("r_str", utf8()), field("r_i32", int32())}), - {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, - &r_batches); - - SCOPED_TRACE("serial"); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - - ASSERT_OK_AND_ASSIGN(auto l_source, - MakeTestSourceNode(plan.get(), "l_source", l_batches, - /*parallel=*/false, - /*slow=*/false)); - ASSERT_OK_AND_ASSIGN(auto r_source, - MakeTestSourceNode(plan.get(), "r_source", r_batches, - /*parallel=*/false, - /*slow=*/false)); - - ASSERT_OK_AND_ASSIGN( - auto semi_join, - MakeHashLeftSemiJoinNode(l_source, r_source, "l_semi_join", - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); - auto sink_gen = MakeSinkNode(semi_join, "sink"); - - GenerateBatchesFromString( - schema({field("l_i32", int32()), field("l_str", utf8())}), - {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, &exp_batches); - - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); -} - } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index b47d6087c0b..e52e2834988 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -35,6 +35,7 @@ #include "arrow/datum.h" #include "arrow/record_batch.h" #include "arrow/testing/gtest_util.h" +#include "arrow/testing/random.h" #include "arrow/type.h" #include "arrow/util/async_generator.h" #include "arrow/util/iterator.h" @@ -154,5 +155,79 @@ ExecBatch ExecBatchFromJSON(const std::vector& descrs, return batch; } +Result MakeTestSourceNode(ExecPlan* plan, std::string label, + BatchesWithSchema batches_with_schema, bool parallel, + bool slow) { + DCHECK_GT(batches_with_schema.batches.size(), 0); + + auto opt_batches = ::arrow::internal::MapVector( + [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, + std::move(batches_with_schema.batches)); + + AsyncGenerator> gen; + + if (parallel) { + // emulate batches completing initial decode-after-scan on a cpu thread + ARROW_ASSIGN_OR_RAISE( + gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), + ::arrow::internal::GetCpuThreadPool())); + + // ensure that callbacks are not executed immediately on a background thread + gen = MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); + } else { + gen = MakeVectorGenerator(std::move(opt_batches)); + } + + if (slow) { + gen = MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { + SleepABit(); + return batch; + }); + } + + return MakeSourceNode(plan, std::move(label), std::move(batches_with_schema.schema), + std::move(gen)); +} + +Future> StartAndCollect( + ExecPlan* plan, AsyncGenerator> gen) { + RETURN_NOT_OK(plan->Validate()); + RETURN_NOT_OK(plan->StartProducing()); + + auto collected_fut = CollectAsyncGenerator(gen); + + return AllComplete({plan->finished(), Future<>(collected_fut)}) + .Then([collected_fut]() -> Result> { + ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); + return ::arrow::internal::MapVector( + [](util::optional batch) { return std::move(*batch); }, + std::move(collected)); + }); +} + +BatchesWithSchema MakeBasicBatches() { + BatchesWithSchema out; + out.batches = { + ExecBatchFromJSON({int32(), boolean()}, "[[null, true], [4, false]]"), + ExecBatchFromJSON({int32(), boolean()}, "[[5, null], [6, false], [7, false]]")}; + out.schema = schema({field("i32", int32()), field("bool", boolean())}); + return out; +} + +BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, + int num_batches, int batch_size) { + BatchesWithSchema out; + + random::RandomArrayGenerator rng(42); + out.batches.resize(num_batches); + + for (int i = 0; i < num_batches; ++i) { + out.batches[i] = ExecBatch(*rng.BatchOf(schema->fields(), batch_size)); + // add a tag scalar to ensure the batches are unique + out.batches[i].values.emplace_back(i); + } + return out; +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/compute/exec/test_util.h index faa395bab78..3ef1333ea42 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/compute/exec/test_util.h @@ -24,6 +24,7 @@ #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/testing/visibility.h" +#include "arrow/util/async_generator.h" #include "arrow/util/string_view.h" namespace arrow { @@ -41,5 +42,26 @@ ARROW_TESTING_EXPORT ExecBatch ExecBatchFromJSON(const std::vector& descrs, util::string_view json); +struct BatchesWithSchema { + std::vector batches; + std::shared_ptr schema; +}; + +ARROW_TESTING_EXPORT +Result MakeTestSourceNode(ExecPlan* plan, std::string label, + BatchesWithSchema batches_with_schema, bool parallel, + bool slow); + +ARROW_TESTING_EXPORT +Future> StartAndCollect( + ExecPlan* plan, AsyncGenerator> gen); + +ARROW_TESTING_EXPORT +BatchesWithSchema MakeBasicBatches(); + +ARROW_TESTING_EXPORT +BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, + int num_batches = 10, int batch_size = 4); + } // namespace compute } // namespace arrow From 2d4b15f95828a05eda9a4ed34948cb3ce1f69987 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 16:23:19 -0400 Subject: [PATCH 12/37] adding right semi join test --- cpp/src/arrow/compute/exec/exec_utils.h | 2 +- cpp/src/arrow/compute/exec/hash_join.cc | 196 +++++++++++-------- cpp/src/arrow/compute/exec/hash_join_test.cc | 47 +++-- 3 files changed, 153 insertions(+), 92 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_utils.h b/cpp/src/arrow/compute/exec/exec_utils.h index c899e3b3058..65dd93150e1 100644 --- a/cpp/src/arrow/compute/exec/exec_utils.h +++ b/cpp/src/arrow/compute/exec/exec_utils.h @@ -63,4 +63,4 @@ class AtomicCounter { }; } // namespace compute -} // namespace arrow \ No newline at end of file +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index ba7a7646b6f..789a71973ee 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -37,6 +37,7 @@ struct HashSemiJoinNode : ExecNode { ctx_(ctx), build_index_field_ids_(build_index_field_ids), probe_index_field_ids_(probe_index_field_ids), + build_result_index(-1), hash_table_built_(false) {} private: @@ -46,8 +47,7 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { - std::cout << "init" - << "\n"; + // std::cout << "init \n"; // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -67,32 +67,62 @@ struct HashSemiJoinNode : ExecNode { return Status::OK(); } - // merge all other groupers to grouper[0]. nothing needs to be done on the - // cached_probe_batches, because when probing everyone. Note: Only one thread - // should execute this out of the pool! - Status BuildSideMerge() { - std::cout << "build side merge" - << "\n"; + // Finds an appropriate index which could accumulate all build indices (i.e. the grouper + // which has the highest # of groups) + void CalculateBuildResultIndex() { + uint32_t curr_max = 0; + for (int i = 0; i < static_cast(local_states_.size()); i++) { + auto* state = &local_states_[i]; + ARROW_DCHECK(state); + if (state->grouper && curr_max < state->grouper->num_groups()) { + curr_max = state->grouper->num_groups(); + build_result_index = i; + } + } + ARROW_DCHECK(build_result_index > -1); + // std::cout << "build_result_index " << build_result_index << "\n"; + } - ThreadLocalState* state0 = &local_states_[0]; - for (size_t i = 1; i < local_states_.size(); ++i) { + // Performs the housekeeping work after the build-side is completed. Note: this method + // should be called ONLY ONCE! + Status BuildSideCompleted() { + // std::cout << "build side merge \n"; + + CalculateBuildResultIndex(); + + // merge every group into the build_result_index grouper + ThreadLocalState* result_state = &local_states_[build_result_index]; + for (int i = 0; i < static_cast(local_states_.size()); ++i) { ThreadLocalState* state = &local_states_[i]; ARROW_DCHECK(state); - if (!state->grouper) { + if (i == build_result_index || !state->grouper) { continue; } ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum _, state0->grouper->Consume(other_keys)); + + // TODO(niranda) replace with void consume method + ARROW_ASSIGN_OR_RAISE(Datum _, result_state->grouper->Consume(other_keys)); state->grouper.reset(); } + + // enable flag that build side is completed + hash_table_built_ = true; + + // since the build side is completed, consume cached probe batches + RETURN_NOT_OK(ConsumeCachedProbeBatches()); + return Status::OK(); } // consumes a build batch and increments the build_batches count. if the build batches // total reached at the end of consumption, all the local states will be merged, before // incrementing the total batches - Status ConsumeBuildBatch(const size_t thread_index, ExecBatch batch) { - std::cout << "ConsumeBuildBatch " << thread_index << " " << batch.length << "\n"; + Status ConsumeBuildBatch(ExecBatch batch) { + // std::cout << "ConsumeBuildBatch tid:" << thread_index << " len:" << batch.length + // << "\n"; + + size_t thread_index = get_thread_index_(); + ARROW_DCHECK(thread_index < local_states_.size()); auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -104,47 +134,51 @@ struct HashSemiJoinNode : ExecNode { } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - // Create a batch with group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); + // Create a batch with group ids TODO(niranda) replace with void consume method + ARROW_ASSIGN_OR_RAISE(Datum _, state->grouper->Consume(key_batch)); if (build_counter_.Increment()) { - // only a single thread would come inside this if-block! - - // while incrementing, if the total is reached, merge all the groupers to 0'th one - RETURN_NOT_OK(BuildSideMerge()); - - // enable flag that build side is completed - hash_table_built_.store(true); - - // since the build side is completed, consume cached probe batches - RETURN_NOT_OK(ConsumeCachedProbeBatches(thread_index)); + // while incrementing, if the total is reached, call BuildSideCompleted + RETURN_NOT_OK(BuildSideCompleted()); } return Status::OK(); } - Status ConsumeCachedProbeBatches(const size_t thread_index) { - std::cout << "ConsumeCachedProbeBatches " << thread_index << "\n"; - - ThreadLocalState* state = &local_states_[thread_index]; - - // TODO (niranda) check if this is the best way to move batches - if (!state->cached_probe_batches.empty()) { - for (auto cached : state->cached_probe_batches) { - RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + // consumes cached probe batches by invoking executor::Spawn. This should be called by a + // single thread. Note: this method should be called ONLY ONCE! + Status ConsumeCachedProbeBatches() { + // std::cout << "ConsumeCachedProbeBatches tid:" << thread_index + // << " len:" << cached_probe_batches.size() << "\n"; + + if (!cached_probe_batches.empty()) { + auto executor = ctx_->executor(); + for (auto&& cached : cached_probe_batches) { + if (executor) { + Status lambda_status; + RETURN_NOT_OK(executor->Spawn([&] { + lambda_status = ConsumeProbeBatch(cached.first, std::move(cached.second)); + })); + + // if the lambda execution failed internally, return status + RETURN_NOT_OK(lambda_status); + } else { + RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + } } - state->cached_probe_batches.clear(); + // cached vector will be cleared. exec batches are expected to be moved to the + // lambdas + cached_probe_batches.clear(); } - return Status::OK(); } // consumes a probe batch and increment probe batches count. Probing would query the - // grouper[0] which have been merged with all others. + // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { - std::cout << "ConsumeProbeBatch " << seq << "\n"; + // std::cout << "ConsumeProbeBatch seq:" << seq << "\n"; - auto* grouper = local_states_[0].grouper.get(); + auto& final_grouper = *local_states_[build_result_index].grouper; // Create a batch with key columns std::vector keys(probe_index_field_ids_.size()); @@ -155,7 +189,7 @@ struct HashSemiJoinNode : ExecNode { // Query the grouper with key_batch. If no match was found, returning group_ids would // have null. - ARROW_ASSIGN_OR_RAISE(Datum group_ids, grouper->Find(key_batch)); + ARROW_ASSIGN_OR_RAISE(Datum group_ids, final_grouper.Find(key_batch)); auto group_ids_data = *group_ids.array(); if (group_ids_data.MayHaveNulls()) { // values need to be filtered @@ -170,20 +204,25 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - std::cout << "output " << seq << " " << out_batch.ToString() << "\n"; + // std::cout << "output seq:" << seq << " " << out_batch.length << "\n"; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // all values are valid for output - std::cout << "output " << seq << " " << batch.ToString() << "\n"; + // std::cout << "output seq:" << seq << " " << batch.length << "\n"; outputs_[0]->InputReceived(this, seq, std::move(batch)); } - out_counter_.Increment(); + if (out_counter_.Increment()) { + finished_.MarkFinished(); + } return Status::OK(); } - void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { - ThreadLocalState* state = &local_states_[thread_index]; - state->cached_probe_batches.emplace_back(seq_num, std::move(batch)); + // void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { + void CacheProbeBatch(int seq_num, ExecBatch batch) { + // std::cout << "cache tid:" << thread_index << " seq:" << seq_num + // << " len:" << batch.length << "\n"; + std::lock_guard lck(cached_probe_batches_mutex); + cached_probe_batches.emplace_back(seq_num, std::move(batch)); } inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } @@ -191,38 +230,35 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - std::cout << "input received " << IsBuildInput(input) << " " << seq << " " - << batch.length << "\n"; + // std::cout << "input received input:" << (IsBuildInput(input) ? "b" : "p") + // << " seq:" << seq << " len:" << batch.length << "\n"; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - size_t thread_index = get_thread_index_(); - ARROW_DCHECK(thread_index < local_states_.size()); - if (finished_.is_finished()) { return; } if (IsBuildInput(input)) { // build input batch is received // if a build input is received when build side is completed, something's wrong! - ARROW_DCHECK(!hash_table_built_.load()); + ARROW_DCHECK(!hash_table_built_); - ErrorIfNotOk(ConsumeBuildBatch(thread_index, std::move(batch))); - } else { // probe input batch is received - if (hash_table_built_.load()) { // build side done, continue with probing + ErrorIfNotOk(ConsumeBuildBatch(std::move(batch))); + } else { // probe input batch is received + if (hash_table_built_) { // build side done, continue with probing // consume cachedProbeBatches if available (for this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + ErrorIfNotOk(ConsumeCachedProbeBatches()); // consume this probe batch ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); } else { // build side not completed. Cache this batch! - CacheProbeBatch(thread_index, seq, std::move(batch)); + CacheProbeBatch(seq, std::move(batch)); } } } void ErrorReceived(ExecNode* input, Status error) override { - std::cout << "error received " << error.ToString() << "\n"; + // std::cout << "error received " << error.ToString() << "\n"; DCHECK_EQ(input, inputs_[0]); outputs_[0]->ErrorReceived(this, std::move(error)); @@ -230,25 +266,18 @@ struct HashSemiJoinNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { - std::cout << "input finished " << IsBuildInput(input) << " " << num_total << "\n"; + // std::cout << "input finished input:" << (IsBuildInput(input) ? "b" : "p") + // << " tot:" << num_total << "\n"; // bail if StopProducing was called if (finished_.is_finished()) return; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - size_t thread_index = get_thread_index_(); - // set total for build input if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { - // while incrementing, if the total is reached, merge all the groupers to 0'th one - ErrorIfNotOk(BuildSideMerge()); - - // enable flag that build side is completed - hash_table_built_.store(true); - - // only build side has completed! so process cached probe batches (of this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + // while incrementing, if the total is reached, call BuildSideCompleted() + ErrorIfNotOk(BuildSideCompleted()); return; } @@ -259,14 +288,16 @@ struct HashSemiJoinNode : ExecNode { // output will be streamed from the probe side. So, they will have the same total. if (out_counter_.SetTotal(num_total)) { // if out_counter has completed, the future is finished! - ErrorIfNotOk(ConsumeCachedProbeBatches(thread_index)); + ErrorIfNotOk(ConsumeCachedProbeBatches()); + outputs_[0]->InputFinished(this, num_total); finished_.MarkFinished(); } outputs_[0]->InputFinished(this, num_total); + // std::cout << "output set:" << num_total << "\n"; } Status StartProducing() override { - std::cout << "start prod \n"; + // std::cout << "start prod \n"; finished_ = Future<>::Make(); local_states_.resize(ThreadIndexer::Capacity()); @@ -278,7 +309,7 @@ struct HashSemiJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - std::cout << "stop prod from node\n"; + // std::cout << "stop prod from node\n"; DCHECK_EQ(output, outputs_[0]); @@ -295,21 +326,20 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { - std::cout << "stop prod \n"; + // std::cout << "stop prod \n"; for (auto&& output : outputs_) { StopProducing(output); } } Future<> finished() override { - std::cout << "finished? " << finished_.is_finished() << "\n"; + // std::cout << "finished? " << finished_.is_finished() << "\n"; return finished_; } private: struct ThreadLocalState { std::unique_ptr grouper; - std::vector> cached_probe_batches{}; }; ExecContext* ctx_; @@ -321,11 +351,19 @@ struct HashSemiJoinNode : ExecNode { AtomicCounter build_counter_, out_counter_; std::vector local_states_; - // need a separate atomic bool to track if the build side complete. Can't use the flag + // we have no guarantee which threads would be coming from the build side. so, out of + // the thread local states, we need to find an appropriate index which could accumulate + // all build indices (ideally, the grouper which has the highest # of elems) + int32_t build_result_index; + + // need a separate bool to track if the build side complete. Can't use the flag // inside the AtomicCounter, because we need to merge the build groupers once we receive // all the build batches. So, while merging, we need to prevent probe batches, being // consumed. - std::atomic hash_table_built_; + bool hash_table_built_; + + std::mutex cached_probe_batches_mutex; + std::vector> cached_probe_batches{}; }; Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, @@ -432,4 +470,4 @@ Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, } } // namespace compute -} // namespace arrow \ No newline at end of file +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_test.cc index 601e7fee8fa..27b51c5cb30 100644 --- a/cpp/src/arrow/compute/exec/hash_join_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_test.cc @@ -29,7 +29,7 @@ namespace compute { void GenerateBatchesFromString(const std::shared_ptr& schema, const std::vector& json_strings, - BatchesWithSchema* out_batches) { + BatchesWithSchema* out_batches, int multiplicity = 1) { std::vector descrs; for (auto&& field : schema->fields()) { descrs.emplace_back(field->type()); @@ -39,10 +39,17 @@ void GenerateBatchesFromString(const std::shared_ptr& schema, out_batches->batches.push_back(ExecBatchFromJSON(descrs, s)); } + size_t batch_count = out_batches->batches.size(); + for (int repeat = 1; repeat < multiplicity; ++repeat) { + for (size_t i = 0; i < batch_count; ++i) { + out_batches->batches.push_back(out_batches->batches[i]); + } + } + out_batches->schema = schema; } -TEST(HashJoin, LeftSemi) { +void RunTest(JoinType type, bool parallel) { auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); BatchesWithSchema l_batches, r_batches, exp_batches; @@ -50,12 +57,12 @@ TEST(HashJoin, LeftSemi) { GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &l_batches); + &l_batches, /*multiplicity=*/parallel ? 100 : 1); GenerateBatchesFromString( r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, - &r_batches); + &r_batches, /*multiplicity=*/parallel ? 100 : 1); SCOPED_TRACE("serial"); @@ -63,26 +70,42 @@ TEST(HashJoin, LeftSemi) { ASSERT_OK_AND_ASSIGN(auto l_source, MakeTestSourceNode(plan.get(), "l_source", l_batches, - /*parallel=*/false, + /*parallel=*/parallel, /*slow=*/false)); ASSERT_OK_AND_ASSIGN(auto r_source, MakeTestSourceNode(plan.get(), "r_source", r_batches, - /*parallel=*/false, + /*parallel=*/parallel, /*slow=*/false)); ASSERT_OK_AND_ASSIGN( auto semi_join, - MakeHashJoinNode(JoinType::LEFT_SEMI, l_source, r_source, "l_semi_join", + MakeHashJoinNode(type, l_source, r_source, "l_semi_join", /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); auto sink_gen = MakeSinkNode(semi_join, "sink"); - GenerateBatchesFromString( - l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &exp_batches); - + if (type == JoinType::LEFT_SEMI) { + GenerateBatchesFromString( + l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &exp_batches, /*multiplicity=*/parallel ? 100 : 1); + } else if (type == JoinType::RIGHT_SEMI) { + GenerateBatchesFromString( + r_schema, {R"([["b", 1], ["b", 2]])", R"([["c", 3]])", R"([["e", 5]])"}, + &exp_batches, /*multiplicity=*/parallel ? 100 : 1); + } ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); } +class HashJoinTest : public testing::TestWithParam> {}; + +INSTANTIATE_TEST_SUITE_P(HashJoinTest, HashJoinTest, + ::testing::Combine(::testing::Values(JoinType::LEFT_SEMI, + JoinType::RIGHT_SEMI), + ::testing::Values(false, true))); + +TEST_P(HashJoinTest, TestSemiJoins) { + RunTest(std::get<0>(GetParam()), std::get<1>(GetParam())); +} + } // namespace compute -} // namespace arrow \ No newline at end of file +} // namespace arrow From 0f9904a63e54f0211f3b406a4a2712f37b82b2a1 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 16:49:57 -0400 Subject: [PATCH 13/37] using log instead of cout --- cpp/src/arrow/compute/exec/hash_join.cc | 49 ++++++++++++------------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 789a71973ee..4a0d154f33d 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -19,10 +19,9 @@ #include #include -#include - #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/exec_utils.h" +#include "arrow/util/logging.h" namespace arrow { namespace compute { @@ -47,7 +46,7 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { - // std::cout << "init \n"; + ARROW_LOG(DEBUG) << "init "; // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -80,13 +79,13 @@ struct HashSemiJoinNode : ExecNode { } } ARROW_DCHECK(build_result_index > -1); - // std::cout << "build_result_index " << build_result_index << "\n"; + ARROW_LOG(DEBUG) << "build_result_index " << build_result_index; } // Performs the housekeeping work after the build-side is completed. Note: this method // should be called ONLY ONCE! Status BuildSideCompleted() { - // std::cout << "build side merge \n"; + ARROW_LOG(DEBUG) << "build side merge"; CalculateBuildResultIndex(); @@ -118,12 +117,12 @@ struct HashSemiJoinNode : ExecNode { // total reached at the end of consumption, all the local states will be merged, before // incrementing the total batches Status ConsumeBuildBatch(ExecBatch batch) { - // std::cout << "ConsumeBuildBatch tid:" << thread_index << " len:" << batch.length - // << "\n"; - size_t thread_index = get_thread_index_(); ARROW_DCHECK(thread_index < local_states_.size()); + ARROW_LOG(DEBUG) << "ConsumeBuildBatch tid:" << thread_index + << " len:" << batch.length; + auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -148,8 +147,8 @@ struct HashSemiJoinNode : ExecNode { // consumes cached probe batches by invoking executor::Spawn. This should be called by a // single thread. Note: this method should be called ONLY ONCE! Status ConsumeCachedProbeBatches() { - // std::cout << "ConsumeCachedProbeBatches tid:" << thread_index - // << " len:" << cached_probe_batches.size() << "\n"; + ARROW_LOG(DEBUG) << "ConsumeCachedProbeBatches tid:" << get_thread_index_() + << " len:" << cached_probe_batches.size(); if (!cached_probe_batches.empty()) { auto executor = ctx_->executor(); @@ -176,7 +175,7 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { - // std::cout << "ConsumeProbeBatch seq:" << seq << "\n"; + ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:" << seq; auto& final_grouper = *local_states_[build_result_index].grouper; @@ -204,10 +203,10 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - // std::cout << "output seq:" << seq << " " << out_batch.length << "\n"; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // all values are valid for output - // std::cout << "output seq:" << seq << " " << batch.length << "\n"; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; outputs_[0]->InputReceived(this, seq, std::move(batch)); } @@ -219,8 +218,8 @@ struct HashSemiJoinNode : ExecNode { // void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { void CacheProbeBatch(int seq_num, ExecBatch batch) { - // std::cout << "cache tid:" << thread_index << " seq:" << seq_num - // << " len:" << batch.length << "\n"; + ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " seq:" << seq_num + << " len:" << batch.length; std::lock_guard lck(cached_probe_batches_mutex); cached_probe_batches.emplace_back(seq_num, std::move(batch)); } @@ -230,8 +229,8 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - // std::cout << "input received input:" << (IsBuildInput(input) ? "b" : "p") - // << " seq:" << seq << " len:" << batch.length << "\n"; + // //std::cout << "input received input:" << (IsBuildInput(input) ? "b" : "p") + // << " seq:" << seq << " len:" << batch.length ; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); @@ -258,7 +257,7 @@ struct HashSemiJoinNode : ExecNode { } void ErrorReceived(ExecNode* input, Status error) override { - // std::cout << "error received " << error.ToString() << "\n"; + ARROW_LOG(DEBUG) << "error received " << error.ToString(); DCHECK_EQ(input, inputs_[0]); outputs_[0]->ErrorReceived(this, std::move(error)); @@ -266,8 +265,8 @@ struct HashSemiJoinNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { - // std::cout << "input finished input:" << (IsBuildInput(input) ? "b" : "p") - // << " tot:" << num_total << "\n"; + ARROW_LOG(DEBUG) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") + << " tot:" << num_total; // bail if StopProducing was called if (finished_.is_finished()) return; @@ -293,11 +292,11 @@ struct HashSemiJoinNode : ExecNode { finished_.MarkFinished(); } outputs_[0]->InputFinished(this, num_total); - // std::cout << "output set:" << num_total << "\n"; + ARROW_LOG(DEBUG) << "output set:" << num_total; } Status StartProducing() override { - // std::cout << "start prod \n"; + ARROW_LOG(DEBUG) << "start prod"; finished_ = Future<>::Make(); local_states_.resize(ThreadIndexer::Capacity()); @@ -309,7 +308,7 @@ struct HashSemiJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - // std::cout << "stop prod from node\n"; + ARROW_LOG(DEBUG) << "stop prod from node"; DCHECK_EQ(output, outputs_[0]); @@ -326,14 +325,14 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { - // std::cout << "stop prod \n"; + ARROW_LOG(DEBUG) << "stop prod "; for (auto&& output : outputs_) { StopProducing(output); } } Future<> finished() override { - // std::cout << "finished? " << finished_.is_finished() << "\n"; + ARROW_LOG(DEBUG) << "finished? " << finished_.is_finished(); return finished_; } From 93da2ae22a29ba43596748a21d2002c06ffcc59b Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 16:55:33 -0400 Subject: [PATCH 14/37] minor changes --- cpp/src/arrow/compute/exec/hash_join.cc | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 4a0d154f33d..47140e552b8 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -15,10 +15,8 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include - +#include "arrow/api.h" +#include "arrow/compute/api.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/exec_utils.h" #include "arrow/util/logging.h" @@ -229,8 +227,8 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - // //std::cout << "input received input:" << (IsBuildInput(input) ? "b" : "p") - // << " seq:" << seq << " len:" << batch.length ; + ARROW_LOG(DEBUG) << "input received input:" << (IsBuildInput(input) ? "b" : "p") + << " seq:" << seq << " len:" << batch.length; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); From 4675c4f3fa1daa41286ca04815c87f8aa4963c27 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 18:16:54 -0400 Subject: [PATCH 15/37] minor bug fix --- cpp/src/arrow/compute/exec/hash_join.cc | 79 +++++++++++++++++-------- 1 file changed, 53 insertions(+), 26 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 47140e552b8..a7fb60da43f 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -35,7 +35,8 @@ struct HashSemiJoinNode : ExecNode { build_index_field_ids_(build_index_field_ids), probe_index_field_ids_(probe_index_field_ids), build_result_index(-1), - hash_table_built_(false) {} + hash_table_built_(false), + cached_probe_batches_consumed(false) {} private: struct ThreadLocalState; @@ -44,7 +45,7 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { - ARROW_LOG(DEBUG) << "init "; + ARROW_LOG(DEBUG) << "init state"; // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -80,11 +81,15 @@ struct HashSemiJoinNode : ExecNode { ARROW_LOG(DEBUG) << "build_result_index " << build_result_index; } - // Performs the housekeeping work after the build-side is completed. Note: this method - // should be called ONLY ONCE! + // Performs the housekeeping work after the build-side is completed. + // Note: this method is not thread safe, and hence should be guaranteed that it is + // not accessed concurrently! Status BuildSideCompleted() { ARROW_LOG(DEBUG) << "build side merge"; + // if the hash table has already been built, return + if (hash_table_built_) return Status::OK(); + CalculateBuildResultIndex(); // merge every group into the build_result_index grouper @@ -131,23 +136,28 @@ struct HashSemiJoinNode : ExecNode { } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - // Create a batch with group ids TODO(niranda) replace with void consume method + // Create a batch with group ids + // TODO(niranda) replace with void consume method ARROW_ASSIGN_OR_RAISE(Datum _, state->grouper->Consume(key_batch)); if (build_counter_.Increment()) { - // while incrementing, if the total is reached, call BuildSideCompleted + // only one thread would get inside this block! + // while incrementing, if the total is reached, call BuildSideCompleted. RETURN_NOT_OK(BuildSideCompleted()); } return Status::OK(); } - // consumes cached probe batches by invoking executor::Spawn. This should be called by a - // single thread. Note: this method should be called ONLY ONCE! + // consumes cached probe batches by invoking executor::Spawn. Status ConsumeCachedProbeBatches() { ARROW_LOG(DEBUG) << "ConsumeCachedProbeBatches tid:" << get_thread_index_() << " len:" << cached_probe_batches.size(); + // acquire the mutex to access cached_probe_batches, because while consuming, other + // batches should not be cached! + std::lock_guard lck(cached_probe_batches_mutex); + if (!cached_probe_batches.empty()) { auto executor = ctx_->executor(); for (auto&& cached : cached_probe_batches) { @@ -167,6 +177,9 @@ struct HashSemiJoinNode : ExecNode { // lambdas cached_probe_batches.clear(); } + + // set flag + cached_probe_batches_consumed = true; return Status::OK(); } @@ -214,12 +227,19 @@ struct HashSemiJoinNode : ExecNode { return Status::OK(); } - // void CacheProbeBatch(const size_t thread_index, int seq_num, ExecBatch batch) { - void CacheProbeBatch(int seq_num, ExecBatch batch) { + // Attempt to cache a probe batch. If it is not cached, return false. + // if cached_probe_batches_consumed is true, by the time a thread acquires + // cached_probe_batches_mutex, it should no longer be cached! instead, it can be + // directly consumed! + bool AttemptToCacheProbeBatch(int seq_num, ExecBatch* batch) { ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " seq:" << seq_num - << " len:" << batch.length; + << " len:" << batch->length; std::lock_guard lck(cached_probe_batches_mutex); - cached_probe_batches.emplace_back(seq_num, std::move(batch)); + if (cached_probe_batches_consumed) { + return false; + } + cached_probe_batches.emplace_back(seq_num, std::move(*batch)); + return true; } inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } @@ -241,15 +261,18 @@ struct HashSemiJoinNode : ExecNode { ARROW_DCHECK(!hash_table_built_); ErrorIfNotOk(ConsumeBuildBatch(std::move(batch))); - } else { // probe input batch is received - if (hash_table_built_) { // build side done, continue with probing - // consume cachedProbeBatches if available (for this thread) - ErrorIfNotOk(ConsumeCachedProbeBatches()); + } else { // probe input batch is received + if (hash_table_built_) { + // build side done, continue with probing. when hash_table_built_ is set, it is + // guaranteed that some thread has already called the ConsumeCachedProbeBatches // consume this probe batch ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); } else { // build side not completed. Cache this batch! - CacheProbeBatch(seq, std::move(batch)); + if (!AttemptToCacheProbeBatch(seq, &batch)) { + // if the cache attempt fails, consume the batch + ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); + } } } } @@ -273,7 +296,8 @@ struct HashSemiJoinNode : ExecNode { // set total for build input if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { - // while incrementing, if the total is reached, call BuildSideCompleted() + // only one thread would get inside this block! + // while incrementing, if the total is reached, call BuildSideCompleted. ErrorIfNotOk(BuildSideCompleted()); return; } @@ -288,9 +312,9 @@ struct HashSemiJoinNode : ExecNode { ErrorIfNotOk(ConsumeCachedProbeBatches()); outputs_[0]->InputFinished(this, num_total); finished_.MarkFinished(); + } else { + outputs_[0]->InputFinished(this, num_total); } - outputs_[0]->InputFinished(this, num_total); - ARROW_LOG(DEBUG) << "output set:" << num_total; } Status StartProducing() override { @@ -324,9 +348,7 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { ARROW_LOG(DEBUG) << "stop prod "; - for (auto&& output : outputs_) { - StopProducing(output); - } + outputs_[0]->StopProducing(); } Future<> finished() override { @@ -348,9 +370,9 @@ struct HashSemiJoinNode : ExecNode { AtomicCounter build_counter_, out_counter_; std::vector local_states_; - // we have no guarantee which threads would be coming from the build side. so, out of - // the thread local states, we need to find an appropriate index which could accumulate - // all build indices (ideally, the grouper which has the highest # of elems) + // There's no guarantee on which threads would be coming from the build side. so, out of + // the thread local states, an appropriate state needs to be chosen to accumulate + // all built results (ideally, the grouper which has the highest # of elems) int32_t build_result_index; // need a separate bool to track if the build side complete. Can't use the flag @@ -361,6 +383,11 @@ struct HashSemiJoinNode : ExecNode { std::mutex cached_probe_batches_mutex; std::vector> cached_probe_batches{}; + // a flag is required to indicate if the cached probe batches have already been + // consumed! if cached_probe_batches_consumed is true, by the time a thread aquires + // cached_probe_batches_mutex, it should no longer be cached! instead, it can be + // directly consumed! + bool cached_probe_batches_consumed; }; Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, From f2efe07ae204d974659da6334e490dbf5179ae92 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 18:59:13 -0400 Subject: [PATCH 16/37] adding empty tests --- cpp/src/arrow/compute/exec/hash_join.cc | 7 +- cpp/src/arrow/compute/exec/hash_join_test.cc | 101 ++++++++++++++----- 2 files changed, 81 insertions(+), 27 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index a7fb60da43f..8077099f17c 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -68,12 +68,13 @@ struct HashSemiJoinNode : ExecNode { // Finds an appropriate index which could accumulate all build indices (i.e. the grouper // which has the highest # of groups) void CalculateBuildResultIndex() { - uint32_t curr_max = 0; + int32_t curr_max = -1; for (int i = 0; i < static_cast(local_states_.size()); i++) { auto* state = &local_states_[i]; ARROW_DCHECK(state); - if (state->grouper && curr_max < state->grouper->num_groups()) { - curr_max = state->grouper->num_groups(); + if (state->grouper && + curr_max < static_cast(state->grouper->num_groups())) { + curr_max = static_cast(state->grouper->num_groups()); build_result_index = i; } } diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_test.cc index 27b51c5cb30..a04eef60bc7 100644 --- a/cpp/src/arrow/compute/exec/hash_join_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_test.cc @@ -49,51 +49,100 @@ void GenerateBatchesFromString(const std::shared_ptr& schema, out_batches->schema = schema; } -void RunTest(JoinType type, bool parallel) { - auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); - auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); - BatchesWithSchema l_batches, r_batches, exp_batches; - - GenerateBatchesFromString(l_schema, - {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", - R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &l_batches, /*multiplicity=*/parallel ? 100 : 1); - - GenerateBatchesFromString( - r_schema, - {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, - &r_batches, /*multiplicity=*/parallel ? 100 : 1); - +void CheckRunOutput(JoinType type, BatchesWithSchema l_batches, + BatchesWithSchema r_batches, + const std::vector& left_keys, + const std::vector& right_keys, + const BatchesWithSchema& exp_batches, bool parallel = false) { SCOPED_TRACE("serial"); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); ASSERT_OK_AND_ASSIGN(auto l_source, - MakeTestSourceNode(plan.get(), "l_source", l_batches, + MakeTestSourceNode(plan.get(), "l_source", std::move(l_batches), /*parallel=*/parallel, /*slow=*/false)); ASSERT_OK_AND_ASSIGN(auto r_source, - MakeTestSourceNode(plan.get(), "r_source", r_batches, + MakeTestSourceNode(plan.get(), "r_source", std::move(r_batches), /*parallel=*/parallel, /*slow=*/false)); ASSERT_OK_AND_ASSIGN( auto semi_join, - MakeHashJoinNode(type, l_source, r_source, "l_semi_join", - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"})); + MakeHashJoinNode(type, l_source, r_source, "hash_join", left_keys, right_keys)); auto sink_gen = MakeSinkNode(semi_join, "sink"); + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); +} + +void RunNonEmptyTest(JoinType type, bool parallel) { + auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); + auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); + BatchesWithSchema l_batches, r_batches, exp_batches; + + int multiplicity = parallel ? 100 : 1; + + GenerateBatchesFromString(l_schema, + {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", + R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &l_batches, multiplicity); + + GenerateBatchesFromString( + r_schema, + {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, + &r_batches, multiplicity); + if (type == JoinType::LEFT_SEMI) { GenerateBatchesFromString( l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &exp_batches, /*multiplicity=*/parallel ? 100 : 1); + &exp_batches, multiplicity); } else if (type == JoinType::RIGHT_SEMI) { GenerateBatchesFromString( r_schema, {R"([["b", 1], ["b", 2]])", R"([["c", 3]])", R"([["e", 5]])"}, - &exp_batches, /*multiplicity=*/parallel ? 100 : 1); + &exp_batches, multiplicity); } - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); + + CheckRunOutput(type, std::move(l_batches), std::move(r_batches), + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, + parallel); +} + +void RunEmptyTest(JoinType type, bool parallel) { + auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); + auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); + BatchesWithSchema l_batches, r_batches, exp_batches; + + int multiplicity = parallel ? 100 : 1; + + if (type == JoinType::LEFT_SEMI) { + GenerateBatchesFromString(l_schema, {R"([])"}, &exp_batches, multiplicity); + } else if (type == JoinType::RIGHT_SEMI) { + GenerateBatchesFromString(r_schema, {R"([])"}, &exp_batches, multiplicity); + } + + // both empty + GenerateBatchesFromString(l_schema, {R"([])"}, &l_batches, multiplicity); + GenerateBatchesFromString(r_schema, {R"([])"}, &r_batches, multiplicity); + CheckRunOutput(type, std::move(l_batches), std::move(r_batches), + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, + parallel); + + // left empty + GenerateBatchesFromString(l_schema, {R"([])"}, &l_batches, multiplicity); + GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, &r_batches, + multiplicity); + CheckRunOutput(type, std::move(l_batches), std::move(r_batches), + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, + parallel); + + // right empty + GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])"}, &l_batches, + multiplicity); + GenerateBatchesFromString(r_schema, {R"([])"}, &r_batches, multiplicity); + CheckRunOutput(type, std::move(l_batches), std::move(r_batches), + /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, + parallel); } class HashJoinTest : public testing::TestWithParam> {}; @@ -104,7 +153,11 @@ INSTANTIATE_TEST_SUITE_P(HashJoinTest, HashJoinTest, ::testing::Values(false, true))); TEST_P(HashJoinTest, TestSemiJoins) { - RunTest(std::get<0>(GetParam()), std::get<1>(GetParam())); + RunNonEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); +} + +TEST_P(HashJoinTest, TestSemiJoinsLeftEmpty) { + RunEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } } // namespace compute From 686e08ae3a4a34440fb891d899244a88906cc183 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 4 Aug 2021 19:56:31 -0400 Subject: [PATCH 17/37] lint changes --- cpp/src/arrow/compute/exec/exec_utils.h | 2 ++ cpp/src/arrow/compute/exec/hash_join.cc | 10 +++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_utils.h b/cpp/src/arrow/compute/exec/exec_utils.h index 65dd93150e1..d6ecbda26b6 100644 --- a/cpp/src/arrow/compute/exec/exec_utils.h +++ b/cpp/src/arrow/compute/exec/exec_utils.h @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +#pragma once + #include #include #include diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 8077099f17c..f00344e13bb 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -464,14 +464,14 @@ Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* righ right_keys); } -static std::string JoinTypeToString[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", - "RIGHT_ANTI", "INNER", "LEFT_OUTER", - "RIGHT_OUTER", "FULL_OUTER"}; - Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, ExecNode* right_input, std::string label, const std::vector& left_keys, const std::vector& right_keys) { + static std::string join_type_string[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", + "RIGHT_ANTI", "INNER", "LEFT_OUTER", + "RIGHT_OUTER", "FULL_OUTER"}; + switch (join_type) { case LEFT_SEMI: // left join--> build from right and probe from left @@ -487,7 +487,7 @@ Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, case LEFT_OUTER: case RIGHT_OUTER: case FULL_OUTER: - return Status::NotImplemented(JoinTypeToString[join_type] + + return Status::NotImplemented(join_type_string[join_type] + " joins not implemented!"); default: return Status::Invalid("invalid join type"); From 8d286d2ff62ac55aa4e0bf028c164d8b2d3b14f8 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 5 Aug 2021 10:39:38 -0400 Subject: [PATCH 18/37] fixing c++/cli mutex import --- cpp/src/arrow/compute/exec/exec_utils.cc | 4 ++-- cpp/src/arrow/compute/exec/exec_utils.h | 4 ++-- cpp/src/arrow/compute/exec/hash_join.cc | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_utils.cc b/cpp/src/arrow/compute/exec/exec_utils.cc index f1a96ac0812..7026351e0b7 100644 --- a/cpp/src/arrow/compute/exec/exec_utils.cc +++ b/cpp/src/arrow/compute/exec/exec_utils.cc @@ -17,7 +17,7 @@ #include "arrow/compute/exec/exec_utils.h" -#include +#include "arrow/util/logging.h" namespace arrow { namespace compute { @@ -25,7 +25,7 @@ namespace compute { size_t ThreadIndexer::operator()() { auto id = std::this_thread::get_id(); - std::unique_lock lock(mutex_); + auto guard = mutex_.Lock(); // acquire the lock const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; return Check(id_index.second); diff --git a/cpp/src/arrow/compute/exec/exec_utils.h b/cpp/src/arrow/compute/exec/exec_utils.h index d6ecbda26b6..93cd0775098 100644 --- a/cpp/src/arrow/compute/exec/exec_utils.h +++ b/cpp/src/arrow/compute/exec/exec_utils.h @@ -17,10 +17,10 @@ #pragma once -#include #include #include +#include "arrow/util/mutex.h" #include "arrow/util/thread_pool.h" namespace arrow { @@ -35,7 +35,7 @@ class ThreadIndexer { private: static size_t Check(size_t thread_index); - std::mutex mutex_; + util::Mutex mutex_; std::unordered_map id_to_index_; }; diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index f00344e13bb..9dd3a037dcd 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +#include + #include "arrow/api.h" #include "arrow/compute/api.h" #include "arrow/compute/exec/exec_plan.h" @@ -159,7 +161,7 @@ struct HashSemiJoinNode : ExecNode { // batches should not be cached! std::lock_guard lck(cached_probe_batches_mutex); - if (!cached_probe_batches.empty()) { + if (!cached_probe_batches_consumed) { auto executor = ctx_->executor(); for (auto&& cached : cached_probe_batches) { if (executor) { From 674eb70d9d2da177ae6886a3b003d4ef89700d9c Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 5 Aug 2021 16:32:49 -0400 Subject: [PATCH 19/37] adding anti-joins --- cpp/src/arrow/compute/exec/hash_join.cc | 115 +++++++++++------- cpp/src/arrow/compute/exec/hash_join_test.cc | 117 +++++++++++++------ 2 files changed, 156 insertions(+), 76 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 9dd3a037dcd..95db8f254cc 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -21,11 +21,13 @@ #include "arrow/compute/api.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/exec_utils.h" +#include "arrow/util/bitmap_ops.h" #include "arrow/util/logging.h" namespace arrow { namespace compute { +template struct HashSemiJoinNode : ExecNode { HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, ExecContext* ctx, const std::vector&& build_index_field_ids, @@ -186,6 +188,33 @@ struct HashSemiJoinNode : ExecNode { return Status::OK(); } + Status GenerateOutput(int seq, const ArrayData& group_ids_data, ExecBatch batch) { + if (group_ids_data.GetNullCount() == batch.length) { + // All NULLS! hence, there are no valid outputs! + ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; + outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); + } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered + auto filter_arr = + std::make_shared(group_ids_data.length, group_ids_data.buffers[0], + /*null_bitmap=*/nullptr, /*null_count=*/0, + /*offset=*/group_ids_data.offset); + ARROW_ASSIGN_OR_RAISE(auto rec_batch, + batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); + ARROW_ASSIGN_OR_RAISE( + auto filtered, + Filter(rec_batch, filter_arr, + /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); + auto out_batch = ExecBatch(*filtered.record_batch()); + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; + outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + } else { // all values are valid for output + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; + outputs_[0]->InputReceived(this, seq, std::move(batch)); + } + + return Status::OK(); + } + // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { @@ -205,24 +234,7 @@ struct HashSemiJoinNode : ExecNode { ARROW_ASSIGN_OR_RAISE(Datum group_ids, final_grouper.Find(key_batch)); auto group_ids_data = *group_ids.array(); - if (group_ids_data.MayHaveNulls()) { // values need to be filtered - auto filter_arr = - std::make_shared(group_ids_data.length, group_ids_data.buffers[0], - /*null_bitmap=*/nullptr, /*null_count=*/0, - /*offset=*/group_ids_data.offset); - ARROW_ASSIGN_OR_RAISE(auto rec_batch, - batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); - ARROW_ASSIGN_OR_RAISE( - auto filtered, - Filter(rec_batch, filter_arr, - /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); - auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; - outputs_[0]->InputReceived(this, seq, std::move(out_batch)); - } else { // all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; - outputs_[0]->InputReceived(this, seq, std::move(batch)); - } + RETURN_NOT_OK(GenerateOutput(seq, group_ids_data, std::move(batch))); if (out_counter_.Increment()) { finished_.MarkFinished(); @@ -393,6 +405,42 @@ struct HashSemiJoinNode : ExecNode { bool cached_probe_batches_consumed; }; +// template specialization for anti joins. For anti joins, group_ids_data needs to be +// inverted. Output will be taken for indices which are NULL +template <> +Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_ids_data, + ExecBatch batch) { + if (group_ids_data.GetNullCount() == group_ids_data.length) { + // All NULLS! hence, all values are valid for output + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; + outputs_[0]->InputReceived(this, seq, std::move(batch)); + } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered + // invert the validity buffer + arrow::internal::InvertBitmap( + group_ids_data.buffers[0]->data(), group_ids_data.offset, group_ids_data.length, + group_ids_data.buffers[0]->mutable_data(), group_ids_data.offset); + + auto filter_arr = + std::make_shared(group_ids_data.length, group_ids_data.buffers[0], + /*null_bitmap=*/nullptr, /*null_count=*/0, + /*offset=*/group_ids_data.offset); + ARROW_ASSIGN_OR_RAISE(auto rec_batch, + batch.ToRecordBatch(output_schema_, ctx_->memory_pool())); + ARROW_ASSIGN_OR_RAISE( + auto filtered, + Filter(rec_batch, filter_arr, + /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); + auto out_batch = ExecBatch(*filtered.record_batch()); + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; + outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + } else { + // No NULLS! hence, there are no valid outputs! + ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; + outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); + } + return Status::OK(); +} + Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, const std::vector& left_keys, const std::vector& right_keys) { @@ -427,6 +475,7 @@ Result> PopulateKeys(const Schema& schema, return key_field_ids; } +template Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, const std::vector& build_keys, @@ -443,27 +492,9 @@ Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_in auto ctx = build_input->plan()->exec_context(); ExecPlan* plan = build_input->plan(); - return plan->EmplaceNode(build_input, probe_input, std::move(label), - ctx, std::move(build_key_ids), - std::move(probe_key_ids)); -} - -Result MakeHashLeftSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys) { - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, - left_keys); -} - -Result MakeHashRightSemiJoinNode(ExecNode* left_input, ExecNode* right_input, - std::string label, - const std::vector& left_keys, - const std::vector& right_keys) { - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, - right_keys); + return plan->EmplaceNode>( + build_input, probe_input, std::move(label), ctx, std::move(build_key_ids), + std::move(probe_key_ids)); } Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, @@ -484,7 +515,13 @@ Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, right_keys); case LEFT_ANTI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, std::move(label), + right_keys, left_keys); case RIGHT_ANTI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, std::move(label), + left_keys, right_keys); case INNER: case LEFT_OUTER: case RIGHT_OUTER: diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_test.cc index a04eef60bc7..816d6adaf7a 100644 --- a/cpp/src/arrow/compute/exec/hash_join_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_test.cc @@ -93,14 +93,32 @@ void RunNonEmptyTest(JoinType type, bool parallel) { {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, &r_batches, multiplicity); - if (type == JoinType::LEFT_SEMI) { - GenerateBatchesFromString( - l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &exp_batches, multiplicity); - } else if (type == JoinType::RIGHT_SEMI) { - GenerateBatchesFromString( - r_schema, {R"([["b", 1], ["b", 2]])", R"([["c", 3]])", R"([["e", 5]])"}, - &exp_batches, multiplicity); + switch (type) { + case LEFT_SEMI: + GenerateBatchesFromString( + l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + &exp_batches, multiplicity); + break; + case RIGHT_SEMI: + GenerateBatchesFromString( + r_schema, {R"([["b", 1], ["b", 2]])", R"([["c", 3]])", R"([["e", 5]])"}, + &exp_batches, multiplicity); + break; + case LEFT_ANTI: + GenerateBatchesFromString( + l_schema, {R"([[0,"d"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", R"([])"}, + &exp_batches, multiplicity); + break; + case RIGHT_ANTI: + GenerateBatchesFromString(r_schema, {R"([["f", 0]])", R"([["g", 4]])", R"([])"}, + &exp_batches, multiplicity); + break; + case INNER: + case LEFT_OUTER: + case RIGHT_OUTER: + case FULL_OUTER: + default: + FAIL() << "join type not implemented!"; } CheckRunOutput(type, std::move(l_batches), std::move(r_batches), @@ -111,46 +129,71 @@ void RunNonEmptyTest(JoinType type, bool parallel) { void RunEmptyTest(JoinType type, bool parallel) { auto l_schema = schema({field("l_i32", int32()), field("l_str", utf8())}); auto r_schema = schema({field("r_str", utf8()), field("r_i32", int32())}); - BatchesWithSchema l_batches, r_batches, exp_batches; int multiplicity = parallel ? 100 : 1; - if (type == JoinType::LEFT_SEMI) { - GenerateBatchesFromString(l_schema, {R"([])"}, &exp_batches, multiplicity); - } else if (type == JoinType::RIGHT_SEMI) { - GenerateBatchesFromString(r_schema, {R"([])"}, &exp_batches, multiplicity); - } + BatchesWithSchema l_empty, r_empty, l_n_empty, r_n_empty; - // both empty - GenerateBatchesFromString(l_schema, {R"([])"}, &l_batches, multiplicity); - GenerateBatchesFromString(r_schema, {R"([])"}, &r_batches, multiplicity); - CheckRunOutput(type, std::move(l_batches), std::move(r_batches), - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, - parallel); + GenerateBatchesFromString(l_schema, {R"([])"}, &l_empty, multiplicity); + GenerateBatchesFromString(r_schema, {R"([])"}, &r_empty, multiplicity); - // left empty - GenerateBatchesFromString(l_schema, {R"([])"}, &l_batches, multiplicity); - GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, &r_batches, + GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])"}, &l_n_empty, multiplicity); - CheckRunOutput(type, std::move(l_batches), std::move(r_batches), - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, - parallel); - - // right empty - GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])"}, &l_batches, + GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, &r_n_empty, multiplicity); - GenerateBatchesFromString(r_schema, {R"([])"}, &r_batches, multiplicity); - CheckRunOutput(type, std::move(l_batches), std::move(r_batches), - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, - parallel); + + std::vector l_keys{"l_str"}; + std::vector r_keys{"r_str"}; + + switch (type) { + case LEFT_SEMI: + // both empty + CheckRunOutput(type, l_empty, r_empty, l_keys, r_keys, l_empty, parallel); + // right empty + CheckRunOutput(type, l_n_empty, r_empty, l_keys, r_keys, l_empty, parallel); + // left empty + CheckRunOutput(type, l_empty, r_n_empty, l_keys, r_keys, l_empty, parallel); + break; + case RIGHT_SEMI: + // both empty + CheckRunOutput(type, l_empty, r_empty, l_keys, r_keys, r_empty, parallel); + // right empty + CheckRunOutput(type, l_n_empty, r_empty, l_keys, r_keys, r_empty, parallel); + // left empty + CheckRunOutput(type, l_empty, r_n_empty, l_keys, r_keys, r_empty, parallel); + break; + case LEFT_ANTI: + // both empty + CheckRunOutput(type, l_empty, r_empty, l_keys, r_keys, l_empty, parallel); + // right empty + CheckRunOutput(type, l_n_empty, r_empty, l_keys, r_keys, l_n_empty, parallel); + // left empty + CheckRunOutput(type, l_empty, r_n_empty, l_keys, r_keys, l_empty, parallel); + break; + case RIGHT_ANTI: + // both empty + CheckRunOutput(type, l_empty, r_empty, l_keys, r_keys, r_empty, parallel); + // right empty + CheckRunOutput(type, l_n_empty, r_empty, l_keys, r_keys, r_empty, parallel); + // left empty + CheckRunOutput(type, l_empty, r_n_empty, l_keys, r_keys, r_n_empty, parallel); + break; + case INNER: + case LEFT_OUTER: + case RIGHT_OUTER: + case FULL_OUTER: + default: + FAIL() << "join type not implemented!"; + } } class HashJoinTest : public testing::TestWithParam> {}; -INSTANTIATE_TEST_SUITE_P(HashJoinTest, HashJoinTest, - ::testing::Combine(::testing::Values(JoinType::LEFT_SEMI, - JoinType::RIGHT_SEMI), - ::testing::Values(false, true))); +INSTANTIATE_TEST_SUITE_P( + HashJoinTest, HashJoinTest, + ::testing::Combine(::testing::Values(JoinType::LEFT_SEMI, JoinType::RIGHT_SEMI, + JoinType::LEFT_ANTI, JoinType::RIGHT_ANTI), + ::testing::Values(false, true))); TEST_P(HashJoinTest, TestSemiJoins) { RunNonEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); From 0a3bcbf58a346b252884be0a4e06e302f86d17bb Mon Sep 17 00:00:00 2001 From: niranda perera Date: Mon, 9 Aug 2021 14:36:07 -0400 Subject: [PATCH 20/37] attempting to solve the threading issue --- cpp/src/arrow/compute/exec/hash_join.cc | 123 ++++++++++++++++-------- 1 file changed, 84 insertions(+), 39 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 95db8f254cc..154a117a1fe 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +#include + #include #include "arrow/api.h" @@ -27,6 +29,39 @@ namespace arrow { namespace compute { +namespace { + +struct FreeIndexFinder { + public: + explicit FreeIndexFinder(size_t max_indices) : indices_(max_indices) { + std::lock_guard lock(mutex_); + std::fill(indices_.begin(), indices_.end(), true); + ARROW_DCHECK_LE(indices_.size(), max_indices); + } + + /// return the first available index + size_t FindAvailableIndex() { + std::lock_guard lock(mutex_); + auto it = std::find(indices_.begin(), indices_.end(), true); + ARROW_DCHECK_NE(it, indices_.end()); + *it = false; + return std::distance(indices_.begin(), it); + } + + /// release the index + void ReleaseIndex(size_t idx) { + std::lock_guard lock(mutex_); + // check if the indices_[idx] == false + ARROW_DCHECK(idx < indices_.size() && !indices_.at(idx)); + indices_.at(idx) = true; + } + + private: + std::mutex mutex_; + std::vector indices_; +}; +} // namespace + template struct HashSemiJoinNode : ExecNode { HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, @@ -36,6 +71,7 @@ struct HashSemiJoinNode : ExecNode { {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), /*num_outputs=*/1), ctx_(ctx), + free_index_finder(nullptr), build_index_field_ids_(build_index_field_ids), probe_index_field_ids_(probe_index_field_ids), build_result_index(-1), @@ -49,7 +85,7 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { - ARROW_LOG(DEBUG) << "init state"; + ARROW_LOG(WARNING) << "init state"; // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -72,25 +108,26 @@ struct HashSemiJoinNode : ExecNode { // Finds an appropriate index which could accumulate all build indices (i.e. the grouper // which has the highest # of groups) void CalculateBuildResultIndex() { - int32_t curr_max = -1; - for (int i = 0; i < static_cast(local_states_.size()); i++) { - auto* state = &local_states_[i]; - ARROW_DCHECK(state); - if (state->grouper && - curr_max < static_cast(state->grouper->num_groups())) { - curr_max = static_cast(state->grouper->num_groups()); - build_result_index = i; - } - } - ARROW_DCHECK(build_result_index > -1); - ARROW_LOG(DEBUG) << "build_result_index " << build_result_index; + // int32_t curr_max = -1; + // for (int i = 0; i < static_cast(local_states_.size()); i++) { + // auto* state = &local_states_[i]; + // ARROW_DCHECK(state); + // if (state->grouper && + // curr_max < static_cast(state->grouper->num_groups())) { + // curr_max = static_cast(state->grouper->num_groups()); + // build_result_index = i; + // } + // } + // ARROW_DCHECK(build_result_index > -1); + // ARROW_LOG(WARNING) << "build_result_index " << build_result_index; + build_result_index = 0; } // Performs the housekeeping work after the build-side is completed. // Note: this method is not thread safe, and hence should be guaranteed that it is // not accessed concurrently! Status BuildSideCompleted() { - ARROW_LOG(DEBUG) << "build side merge"; + ARROW_LOG(WARNING) << "build side merge"; // if the hash table has already been built, return if (hash_table_built_) return Status::OK(); @@ -125,11 +162,13 @@ struct HashSemiJoinNode : ExecNode { // total reached at the end of consumption, all the local states will be merged, before // incrementing the total batches Status ConsumeBuildBatch(ExecBatch batch) { - size_t thread_index = get_thread_index_(); - ARROW_DCHECK(thread_index < local_states_.size()); + // size_t thread_index = get_thread_index_(); + // get a free index from the finder + int thread_index = free_index_finder->FindAvailableIndex(); + ARROW_DCHECK(static_cast(thread_index) < local_states_.size()); - ARROW_LOG(DEBUG) << "ConsumeBuildBatch tid:" << thread_index - << " len:" << batch.length; + ARROW_LOG(WARNING) << "ConsumeBuildBatch tid:" << thread_index + << " len:" << batch.length; auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -145,6 +184,8 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) replace with void consume method ARROW_ASSIGN_OR_RAISE(Datum _, state->grouper->Consume(key_batch)); + free_index_finder->ReleaseIndex(thread_index); + if (build_counter_.Increment()) { // only one thread would get inside this block! // while incrementing, if the total is reached, call BuildSideCompleted. @@ -156,8 +197,8 @@ struct HashSemiJoinNode : ExecNode { // consumes cached probe batches by invoking executor::Spawn. Status ConsumeCachedProbeBatches() { - ARROW_LOG(DEBUG) << "ConsumeCachedProbeBatches tid:" << get_thread_index_() - << " len:" << cached_probe_batches.size(); + ARROW_LOG(WARNING) << "ConsumeCachedProbeBatches tid:" /*<< get_thread_index_()*/ + << " len:" << cached_probe_batches.size(); // acquire the mutex to access cached_probe_batches, because while consuming, other // batches should not be cached! @@ -191,7 +232,7 @@ struct HashSemiJoinNode : ExecNode { Status GenerateOutput(int seq, const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == batch.length) { // All NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; + ARROW_LOG(WARNING) << "output seq:" << seq << " 0"; outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered auto filter_arr = @@ -205,10 +246,10 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; + ARROW_LOG(WARNING) << "output seq:" << seq << " " << out_batch.length; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; + ARROW_LOG(WARNING) << "output seq:" << seq << " " << batch.length; outputs_[0]->InputReceived(this, seq, std::move(batch)); } @@ -218,7 +259,7 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { - ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:" << seq; + ARROW_LOG(WARNING) << "ConsumeProbeBatch seq:" << seq; auto& final_grouper = *local_states_[build_result_index].grouper; @@ -247,8 +288,8 @@ struct HashSemiJoinNode : ExecNode { // cached_probe_batches_mutex, it should no longer be cached! instead, it can be // directly consumed! bool AttemptToCacheProbeBatch(int seq_num, ExecBatch* batch) { - ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " seq:" << seq_num - << " len:" << batch->length; + ARROW_LOG(WARNING) << "cache tid:" /*<< get_thread_index_() */ << " seq:" << seq_num + << " len:" << batch->length; std::lock_guard lck(cached_probe_batches_mutex); if (cached_probe_batches_consumed) { return false; @@ -262,8 +303,8 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - ARROW_LOG(DEBUG) << "input received input:" << (IsBuildInput(input) ? "b" : "p") - << " seq:" << seq << " len:" << batch.length; + ARROW_LOG(WARNING) << "input received input:" << (IsBuildInput(input) ? "b" : "p") + << " seq:" << seq << " len:" << batch.length; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); @@ -293,7 +334,7 @@ struct HashSemiJoinNode : ExecNode { } void ErrorReceived(ExecNode* input, Status error) override { - ARROW_LOG(DEBUG) << "error received " << error.ToString(); + ARROW_LOG(WARNING) << "error received " << error.ToString(); DCHECK_EQ(input, inputs_[0]); outputs_[0]->ErrorReceived(this, std::move(error)); @@ -301,8 +342,8 @@ struct HashSemiJoinNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { - ARROW_LOG(DEBUG) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") - << " tot:" << num_total; + ARROW_LOG(WARNING) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") + << " tot:" << num_total; // bail if StopProducing was called if (finished_.is_finished()) return; @@ -333,10 +374,12 @@ struct HashSemiJoinNode : ExecNode { } Status StartProducing() override { - ARROW_LOG(DEBUG) << "start prod"; + ARROW_LOG(WARNING) << "start prod"; finished_ = Future<>::Make(); local_states_.resize(ThreadIndexer::Capacity()); + free_index_finder = + arrow::internal::make_unique(ThreadIndexer::Capacity()); return Status::OK(); } @@ -345,7 +388,7 @@ struct HashSemiJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - ARROW_LOG(DEBUG) << "stop prod from node"; + ARROW_LOG(WARNING) << "stop prod from node"; DCHECK_EQ(output, outputs_[0]); @@ -362,12 +405,12 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { - ARROW_LOG(DEBUG) << "stop prod "; + ARROW_LOG(WARNING) << "stop prod "; outputs_[0]->StopProducing(); } Future<> finished() override { - ARROW_LOG(DEBUG) << "finished? " << finished_.is_finished(); + ARROW_LOG(WARNING) << "finished? " << finished_.is_finished(); return finished_; } @@ -379,10 +422,12 @@ struct HashSemiJoinNode : ExecNode { ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); - ThreadIndexer get_thread_index_; + // ThreadIndexer get_thread_index_; + std::unique_ptr free_index_finder; const std::vector build_index_field_ids_, probe_index_field_ids_; AtomicCounter build_counter_, out_counter_; + std::vector local_states_; // There's no guarantee on which threads would be coming from the build side. so, out of @@ -412,7 +457,7 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id ExecBatch batch) { if (group_ids_data.GetNullCount() == group_ids_data.length) { // All NULLS! hence, all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; + ARROW_LOG(WARNING) << "output seq:" << seq << " " << batch.length; outputs_[0]->InputReceived(this, seq, std::move(batch)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered // invert the validity buffer @@ -431,11 +476,11 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; + ARROW_LOG(WARNING) << "output seq:" << seq << " " << out_batch.length; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // No NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; + ARROW_LOG(WARNING) << "output seq:" << seq << " 0"; outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); } return Status::OK(); From f5d3c5fad42602fd4e88f848b2e3b8e4de0eee7c Mon Sep 17 00:00:00 2001 From: niranda perera Date: Mon, 9 Aug 2021 15:45:38 -0400 Subject: [PATCH 21/37] Revert "attempting to solve the threading issue" This reverts commit 0a3bcbf58a346b252884be0a4e06e302f86d17bb. --- cpp/src/arrow/compute/exec/hash_join.cc | 123 ++++++++---------------- 1 file changed, 39 insertions(+), 84 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 154a117a1fe..95db8f254cc 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -#include - #include #include "arrow/api.h" @@ -29,39 +27,6 @@ namespace arrow { namespace compute { -namespace { - -struct FreeIndexFinder { - public: - explicit FreeIndexFinder(size_t max_indices) : indices_(max_indices) { - std::lock_guard lock(mutex_); - std::fill(indices_.begin(), indices_.end(), true); - ARROW_DCHECK_LE(indices_.size(), max_indices); - } - - /// return the first available index - size_t FindAvailableIndex() { - std::lock_guard lock(mutex_); - auto it = std::find(indices_.begin(), indices_.end(), true); - ARROW_DCHECK_NE(it, indices_.end()); - *it = false; - return std::distance(indices_.begin(), it); - } - - /// release the index - void ReleaseIndex(size_t idx) { - std::lock_guard lock(mutex_); - // check if the indices_[idx] == false - ARROW_DCHECK(idx < indices_.size() && !indices_.at(idx)); - indices_.at(idx) = true; - } - - private: - std::mutex mutex_; - std::vector indices_; -}; -} // namespace - template struct HashSemiJoinNode : ExecNode { HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, @@ -71,7 +36,6 @@ struct HashSemiJoinNode : ExecNode { {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), /*num_outputs=*/1), ctx_(ctx), - free_index_finder(nullptr), build_index_field_ids_(build_index_field_ids), probe_index_field_ids_(probe_index_field_ids), build_result_index(-1), @@ -85,7 +49,7 @@ struct HashSemiJoinNode : ExecNode { const char* kind_name() override { return "HashSemiJoinNode"; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { - ARROW_LOG(WARNING) << "init state"; + ARROW_LOG(DEBUG) << "init state"; // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -108,26 +72,25 @@ struct HashSemiJoinNode : ExecNode { // Finds an appropriate index which could accumulate all build indices (i.e. the grouper // which has the highest # of groups) void CalculateBuildResultIndex() { - // int32_t curr_max = -1; - // for (int i = 0; i < static_cast(local_states_.size()); i++) { - // auto* state = &local_states_[i]; - // ARROW_DCHECK(state); - // if (state->grouper && - // curr_max < static_cast(state->grouper->num_groups())) { - // curr_max = static_cast(state->grouper->num_groups()); - // build_result_index = i; - // } - // } - // ARROW_DCHECK(build_result_index > -1); - // ARROW_LOG(WARNING) << "build_result_index " << build_result_index; - build_result_index = 0; + int32_t curr_max = -1; + for (int i = 0; i < static_cast(local_states_.size()); i++) { + auto* state = &local_states_[i]; + ARROW_DCHECK(state); + if (state->grouper && + curr_max < static_cast(state->grouper->num_groups())) { + curr_max = static_cast(state->grouper->num_groups()); + build_result_index = i; + } + } + ARROW_DCHECK(build_result_index > -1); + ARROW_LOG(DEBUG) << "build_result_index " << build_result_index; } // Performs the housekeeping work after the build-side is completed. // Note: this method is not thread safe, and hence should be guaranteed that it is // not accessed concurrently! Status BuildSideCompleted() { - ARROW_LOG(WARNING) << "build side merge"; + ARROW_LOG(DEBUG) << "build side merge"; // if the hash table has already been built, return if (hash_table_built_) return Status::OK(); @@ -162,13 +125,11 @@ struct HashSemiJoinNode : ExecNode { // total reached at the end of consumption, all the local states will be merged, before // incrementing the total batches Status ConsumeBuildBatch(ExecBatch batch) { - // size_t thread_index = get_thread_index_(); - // get a free index from the finder - int thread_index = free_index_finder->FindAvailableIndex(); - ARROW_DCHECK(static_cast(thread_index) < local_states_.size()); + size_t thread_index = get_thread_index_(); + ARROW_DCHECK(thread_index < local_states_.size()); - ARROW_LOG(WARNING) << "ConsumeBuildBatch tid:" << thread_index - << " len:" << batch.length; + ARROW_LOG(DEBUG) << "ConsumeBuildBatch tid:" << thread_index + << " len:" << batch.length; auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -184,8 +145,6 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) replace with void consume method ARROW_ASSIGN_OR_RAISE(Datum _, state->grouper->Consume(key_batch)); - free_index_finder->ReleaseIndex(thread_index); - if (build_counter_.Increment()) { // only one thread would get inside this block! // while incrementing, if the total is reached, call BuildSideCompleted. @@ -197,8 +156,8 @@ struct HashSemiJoinNode : ExecNode { // consumes cached probe batches by invoking executor::Spawn. Status ConsumeCachedProbeBatches() { - ARROW_LOG(WARNING) << "ConsumeCachedProbeBatches tid:" /*<< get_thread_index_()*/ - << " len:" << cached_probe_batches.size(); + ARROW_LOG(DEBUG) << "ConsumeCachedProbeBatches tid:" << get_thread_index_() + << " len:" << cached_probe_batches.size(); // acquire the mutex to access cached_probe_batches, because while consuming, other // batches should not be cached! @@ -232,7 +191,7 @@ struct HashSemiJoinNode : ExecNode { Status GenerateOutput(int seq, const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == batch.length) { // All NULLS! hence, there are no valid outputs! - ARROW_LOG(WARNING) << "output seq:" << seq << " 0"; + ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered auto filter_arr = @@ -246,10 +205,10 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(WARNING) << "output seq:" << seq << " " << out_batch.length; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // all values are valid for output - ARROW_LOG(WARNING) << "output seq:" << seq << " " << batch.length; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; outputs_[0]->InputReceived(this, seq, std::move(batch)); } @@ -259,7 +218,7 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(int seq, ExecBatch batch) { - ARROW_LOG(WARNING) << "ConsumeProbeBatch seq:" << seq; + ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:" << seq; auto& final_grouper = *local_states_[build_result_index].grouper; @@ -288,8 +247,8 @@ struct HashSemiJoinNode : ExecNode { // cached_probe_batches_mutex, it should no longer be cached! instead, it can be // directly consumed! bool AttemptToCacheProbeBatch(int seq_num, ExecBatch* batch) { - ARROW_LOG(WARNING) << "cache tid:" /*<< get_thread_index_() */ << " seq:" << seq_num - << " len:" << batch->length; + ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " seq:" << seq_num + << " len:" << batch->length; std::lock_guard lck(cached_probe_batches_mutex); if (cached_probe_batches_consumed) { return false; @@ -303,8 +262,8 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - ARROW_LOG(WARNING) << "input received input:" << (IsBuildInput(input) ? "b" : "p") - << " seq:" << seq << " len:" << batch.length; + ARROW_LOG(DEBUG) << "input received input:" << (IsBuildInput(input) ? "b" : "p") + << " seq:" << seq << " len:" << batch.length; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); @@ -334,7 +293,7 @@ struct HashSemiJoinNode : ExecNode { } void ErrorReceived(ExecNode* input, Status error) override { - ARROW_LOG(WARNING) << "error received " << error.ToString(); + ARROW_LOG(DEBUG) << "error received " << error.ToString(); DCHECK_EQ(input, inputs_[0]); outputs_[0]->ErrorReceived(this, std::move(error)); @@ -342,8 +301,8 @@ struct HashSemiJoinNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { - ARROW_LOG(WARNING) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") - << " tot:" << num_total; + ARROW_LOG(DEBUG) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") + << " tot:" << num_total; // bail if StopProducing was called if (finished_.is_finished()) return; @@ -374,12 +333,10 @@ struct HashSemiJoinNode : ExecNode { } Status StartProducing() override { - ARROW_LOG(WARNING) << "start prod"; + ARROW_LOG(DEBUG) << "start prod"; finished_ = Future<>::Make(); local_states_.resize(ThreadIndexer::Capacity()); - free_index_finder = - arrow::internal::make_unique(ThreadIndexer::Capacity()); return Status::OK(); } @@ -388,7 +345,7 @@ struct HashSemiJoinNode : ExecNode { void ResumeProducing(ExecNode* output) override {} void StopProducing(ExecNode* output) override { - ARROW_LOG(WARNING) << "stop prod from node"; + ARROW_LOG(DEBUG) << "stop prod from node"; DCHECK_EQ(output, outputs_[0]); @@ -405,12 +362,12 @@ struct HashSemiJoinNode : ExecNode { // TODO(niranda) couldn't there be multiple outputs for a Node? void StopProducing() override { - ARROW_LOG(WARNING) << "stop prod "; + ARROW_LOG(DEBUG) << "stop prod "; outputs_[0]->StopProducing(); } Future<> finished() override { - ARROW_LOG(WARNING) << "finished? " << finished_.is_finished(); + ARROW_LOG(DEBUG) << "finished? " << finished_.is_finished(); return finished_; } @@ -422,12 +379,10 @@ struct HashSemiJoinNode : ExecNode { ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); - // ThreadIndexer get_thread_index_; - std::unique_ptr free_index_finder; + ThreadIndexer get_thread_index_; const std::vector build_index_field_ids_, probe_index_field_ids_; AtomicCounter build_counter_, out_counter_; - std::vector local_states_; // There's no guarantee on which threads would be coming from the build side. so, out of @@ -457,7 +412,7 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id ExecBatch batch) { if (group_ids_data.GetNullCount() == group_ids_data.length) { // All NULLS! hence, all values are valid for output - ARROW_LOG(WARNING) << "output seq:" << seq << " " << batch.length; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; outputs_[0]->InputReceived(this, seq, std::move(batch)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered // invert the validity buffer @@ -476,11 +431,11 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(WARNING) << "output seq:" << seq << " " << out_batch.length; + ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; outputs_[0]->InputReceived(this, seq, std::move(out_batch)); } else { // No NULLS! hence, there are no valid outputs! - ARROW_LOG(WARNING) << "output seq:" << seq << " 0"; + ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); } return Status::OK(); From cf5113d032777efa544258d3ca2ad1b6fcf5b072 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 10 Aug 2021 15:33:53 -0400 Subject: [PATCH 22/37] refactoring to new API --- cpp/src/arrow/compute/exec/aggregate_node.cc | 28 --- cpp/src/arrow/compute/exec/exec_utils.cc | 78 --------- cpp/src/arrow/compute/exec/exec_utils.h | 68 ------- cpp/src/arrow/compute/exec/hash_join.cc | 175 ++++++++++--------- cpp/src/arrow/compute/exec/options.h | 27 +++ cpp/src/arrow/compute/exec/plan_test.cc | 80 --------- cpp/src/arrow/compute/exec/test_util.cc | 34 ---- cpp/src/arrow/compute/exec/test_util.h | 40 ++++- cpp/src/arrow/compute/exec/util.cc | 22 +++ cpp/src/arrow/compute/exec/util.h | 16 ++ 10 files changed, 195 insertions(+), 373 deletions(-) delete mode 100644 cpp/src/arrow/compute/exec/exec_utils.cc delete mode 100644 cpp/src/arrow/compute/exec/exec_utils.h diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index de9078cd07e..f910ff38b7c 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -59,34 +59,6 @@ Result ResolveKernels( namespace { -class ThreadIndexer { - public: - size_t operator()() { - auto id = std::this_thread::get_id(); - - std::unique_lock lock(mutex_); - const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; - - return Check(id_index.second); - } - - static size_t Capacity() { - static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); - return max_size; - } - - private: - size_t Check(size_t thread_index) { - DCHECK_LT(thread_index, Capacity()) << "thread index " << thread_index - << " is out of range [0, " << Capacity() << ")"; - - return thread_index; - } - - std::mutex mutex_; - std::unordered_map id_to_index_; -}; - struct ScalarAggregateNode : ExecNode { ScalarAggregateNode(ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema, diff --git a/cpp/src/arrow/compute/exec/exec_utils.cc b/cpp/src/arrow/compute/exec/exec_utils.cc deleted file mode 100644 index 7026351e0b7..00000000000 --- a/cpp/src/arrow/compute/exec/exec_utils.cc +++ /dev/null @@ -1,78 +0,0 @@ -// 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/exec_utils.h" - -#include "arrow/util/logging.h" - -namespace arrow { -namespace compute { - -size_t ThreadIndexer::operator()() { - auto id = std::this_thread::get_id(); - - auto guard = mutex_.Lock(); // acquire the lock - const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; - - return Check(id_index.second); -} - -size_t ThreadIndexer::Capacity() { - static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); - return max_size; -} - -size_t ThreadIndexer::Check(size_t thread_index) { - DCHECK_LT(thread_index, Capacity()) - << "thread index " << thread_index << " is out of range [0, " << Capacity() << ")"; - - return thread_index; -} - -int AtomicCounter::count() const { return count_.load(); } - -util::optional AtomicCounter::total() const { - int total = total_.load(); - if (total == -1) return {}; - return total; -} - -bool AtomicCounter::Increment() { - DCHECK_NE(count_.load(), total_.load()); - int count = count_.fetch_add(1) + 1; - if (count != total_.load()) return false; - return DoneOnce(); -} - -// return true if the counter is complete -bool AtomicCounter::SetTotal(int total) { - total_.store(total); - if (count_.load() != total) return false; - return DoneOnce(); -} - -// return true if the counter has not already been completed -bool AtomicCounter::Cancel() { return DoneOnce(); } - -// ensure there is only one true return from Increment(), SetTotal(), or Cancel() -bool AtomicCounter::DoneOnce() { - bool expected = false; - return complete_.compare_exchange_strong(expected, true); -} - -} // namespace compute -} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_utils.h b/cpp/src/arrow/compute/exec/exec_utils.h deleted file mode 100644 index 93cd0775098..00000000000 --- a/cpp/src/arrow/compute/exec/exec_utils.h +++ /dev/null @@ -1,68 +0,0 @@ -// 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 - -#include "arrow/util/mutex.h" -#include "arrow/util/thread_pool.h" - -namespace arrow { -namespace compute { - -class ThreadIndexer { - public: - size_t operator()(); - - static size_t Capacity(); - - private: - static size_t Check(size_t thread_index); - - util::Mutex mutex_; - std::unordered_map id_to_index_; -}; - -class AtomicCounter { - public: - AtomicCounter() = default; - - int count() const; - - util::optional total() const; - - // return true if the counter is complete - bool Increment(); - - // return true if the counter is complete - bool SetTotal(int total); - - // return true if the counter has not already been completed - bool Cancel(); - - private: - // ensure there is only one true return from Increment(), SetTotal(), or Cancel() - bool DoneOnce(); - - std::atomic count_{0}, total_{-1}; - std::atomic complete_{false}; -}; - -} // namespace compute -} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 95db8f254cc..8bffe826dad 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -20,19 +20,60 @@ #include "arrow/api.h" #include "arrow/compute/api.h" #include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/exec_utils.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" #include "arrow/util/bitmap_ops.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" #include "arrow/util/logging.h" +#include "arrow/util/thread_pool.h" namespace arrow { + +using internal::checked_cast; + namespace compute { +namespace { +Status ValidateJoinInputs(const std::shared_ptr& left_schema, + const std::shared_ptr& right_schema, + const std::vector& left_keys, + const std::vector& right_keys) { + if (left_keys.size() != right_keys.size()) { + return Status::Invalid("left and right key sizes do not match"); + } + + for (size_t i = 0; i < left_keys.size(); i++) { + auto l_type = left_schema->field(left_keys[i])->type(); + auto r_type = right_schema->field(right_keys[i])->type(); + + if (!l_type->Equals(r_type)) { + return Status::Invalid("build and probe types do not match: " + l_type->ToString() + + "!=" + r_type->ToString()); + } + } + + return Status::OK(); +} + +Result> PopulateKeys(const Schema& schema, + const std::vector& keys) { + std::vector key_field_ids(keys.size()); + // Find input field indices for left key fields + for (size_t i = 0; i < keys.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, keys[i].FindOne(schema)); + key_field_ids[i] = match[0]; + } + return key_field_ids; +} +} // namespace + template struct HashSemiJoinNode : ExecNode { - HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, std::string label, - ExecContext* ctx, const std::vector&& build_index_field_ids, + HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, ExecContext* ctx, + const std::vector&& build_index_field_ids, const std::vector&& probe_index_field_ids) - : ExecNode(build_input->plan(), std::move(label), {build_input, probe_input}, + : ExecNode(build_input->plan(), {build_input, probe_input}, {"hash_join_build", "hash_join_probe"}, probe_input->output_schema(), /*num_outputs=*/1), ctx_(ctx), @@ -441,97 +482,71 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id return Status::OK(); } -Status ValidateJoinInputs(ExecNode* left_input, ExecNode* right_input, - const std::vector& left_keys, - const std::vector& right_keys) { - if (left_keys.size() != right_keys.size()) { - return Status::Invalid("left and right key sizes do not match"); - } - - const auto& l_schema = left_input->output_schema(); - const auto& r_schema = right_input->output_schema(); - for (size_t i = 0; i < left_keys.size(); i++) { - auto l_type = l_schema->GetFieldByName(left_keys[i])->type(); - auto r_type = r_schema->GetFieldByName(right_keys[i])->type(); - - if (!l_type->Equals(r_type)) { - return Status::Invalid("build and probe types do not match: " + l_type->ToString() + - "!=" + r_type->ToString()); - } - } - - return Status::OK(); -} - -Result> PopulateKeys(const Schema& schema, - const std::vector& keys) { - std::vector key_field_ids(keys.size()); - // Find input field indices for left key fields - for (size_t i = 0; i < keys.size(); ++i) { - ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(schema)); - key_field_ids[i] = match[0]; - } - - return key_field_ids; -} - template Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, - std::string label, - const std::vector& build_keys, - const std::vector& probe_keys) { - RETURN_NOT_OK(ValidateJoinInputs(build_input, probe_input, build_keys, probe_keys)); - + const std::vector& build_keys, + const std::vector& probe_keys) { auto build_schema = build_input->output_schema(); auto probe_schema = probe_input->output_schema(); ARROW_ASSIGN_OR_RAISE(auto build_key_ids, PopulateKeys(*build_schema, build_keys)); ARROW_ASSIGN_OR_RAISE(auto probe_key_ids, PopulateKeys(*probe_schema, probe_keys)); + RETURN_NOT_OK( + ValidateJoinInputs(build_schema, probe_schema, build_key_ids, probe_key_ids)); + // output schema will be probe schema auto ctx = build_input->plan()->exec_context(); ExecPlan* plan = build_input->plan(); return plan->EmplaceNode>( - build_input, probe_input, std::move(label), ctx, std::move(build_key_ids), - std::move(probe_key_ids)); + build_input, probe_input, ctx, std::move(build_key_ids), std::move(probe_key_ids)); } -Result MakeHashJoinNode(JoinType join_type, ExecNode* left_input, - ExecNode* right_input, std::string label, - const std::vector& left_keys, - const std::vector& right_keys) { - static std::string join_type_string[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", - "RIGHT_ANTI", "INNER", "LEFT_OUTER", - "RIGHT_OUTER", "FULL_OUTER"}; - - switch (join_type) { - case LEFT_SEMI: - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, std::move(label), right_keys, - left_keys); - case RIGHT_SEMI: - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, std::move(label), left_keys, - right_keys); - case LEFT_ANTI: - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, std::move(label), - right_keys, left_keys); - case RIGHT_ANTI: - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, std::move(label), - left_keys, right_keys); - case INNER: - case LEFT_OUTER: - case RIGHT_OUTER: - case FULL_OUTER: - return Status::NotImplemented(join_type_string[join_type] + - " joins not implemented!"); - default: - return Status::Invalid("invalid join type"); - } -} +ExecFactoryRegistry::AddOnLoad kRegisterHashJoin( + "hash_join", + [](ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) -> Result { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 2, "HashJoinNode")); + + const auto& join_options = checked_cast(options); + + static std::string join_type_string[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", + "RIGHT_ANTI", "INNER", "LEFT_OUTER", + "RIGHT_OUTER", "FULL_OUTER"}; + + auto join_type = join_options.join_type; + + ExecNode* left_input = inputs[0]; + ExecNode* right_input = inputs[1]; + const auto& left_keys = join_options.left_keys; + const auto& right_keys = join_options.right_keys; + + switch (join_type) { + case LEFT_SEMI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, right_keys, left_keys); + case RIGHT_SEMI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, left_keys, right_keys); + case LEFT_ANTI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, right_keys, + left_keys); + case RIGHT_ANTI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, left_keys, + right_keys); + case INNER: + case LEFT_OUTER: + case RIGHT_OUTER: + case FULL_OUTER: + return Status::NotImplemented(join_type_string[join_type] + + " joins not implemented!"); + default: + return Status::Invalid("invalid join type"); + } + }); } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index a853a74362d..272681c0dbf 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -111,5 +111,32 @@ class ARROW_EXPORT SinkNodeOptions : public ExecNodeOptions { std::function>()>* generator; }; +enum JoinType { + LEFT_SEMI, + RIGHT_SEMI, + LEFT_ANTI, + RIGHT_ANTI, + INNER, // Not Implemented + LEFT_OUTER, // Not Implemented + RIGHT_OUTER, // Not Implemented + FULL_OUTER // Not Implemented +}; + +class ARROW_EXPORT JoinNodeOptions : public ExecNodeOptions { + public: + JoinNodeOptions(JoinType join_type, std::vector left_keys, + std::vector right_keys) + : join_type(join_type), + left_keys(std::move(left_keys)), + right_keys(std::move(right_keys)) {} + + // type of the join + JoinType join_type; + + // index keys of the join + std::vector left_keys; + std::vector right_keys; +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 1ca620304cf..d275c591a8a 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -195,86 +195,6 @@ TEST(ExecPlan, DummyStartProducingError) { ASSERT_THAT(t.stopped, ElementsAre("process2", "process3", "sink")); } -namespace { - -struct BatchesWithSchema { - std::vector batches; - std::shared_ptr schema; - - AsyncGenerator> gen(bool parallel, bool slow) const { - DCHECK_GT(batches.size(), 0); - - auto opt_batches = ::arrow::internal::MapVector( - [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, batches); - - AsyncGenerator> gen; - - if (parallel) { - // emulate batches completing initial decode-after-scan on a cpu thread - gen = MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - ::arrow::internal::GetCpuThreadPool()) - .ValueOrDie(); - - // ensure that callbacks are not executed immediately on a background thread - gen = - MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); - } else { - gen = MakeVectorGenerator(std::move(opt_batches)); - } - - if (slow) { - gen = - MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { - SleepABit(); - return batch; - }); - } - - return gen; - } -}; - -Future> StartAndCollect( - ExecPlan* plan, AsyncGenerator> gen) { - RETURN_NOT_OK(plan->Validate()); - RETURN_NOT_OK(plan->StartProducing()); - - auto collected_fut = CollectAsyncGenerator(gen); - - return AllComplete({plan->finished(), Future<>(collected_fut)}) - .Then([collected_fut]() -> Result> { - ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); - return ::arrow::internal::MapVector( - [](util::optional batch) { return std::move(*batch); }, - std::move(collected)); - }); -} - -BatchesWithSchema MakeBasicBatches() { - BatchesWithSchema out; - out.batches = { - ExecBatchFromJSON({int32(), boolean()}, "[[null, true], [4, false]]"), - ExecBatchFromJSON({int32(), boolean()}, "[[5, null], [6, false], [7, false]]")}; - out.schema = schema({field("i32", int32()), field("bool", boolean())}); - return out; -} - -BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, - int num_batches = 10, int batch_size = 4) { - BatchesWithSchema out; - - random::RandomArrayGenerator rng(42); - out.batches.resize(num_batches); - - for (int i = 0; i < num_batches; ++i) { - out.batches[i] = ExecBatch(*rng.BatchOf(schema->fields(), batch_size)); - // add a tag scalar to ensure the batches are unique - out.batches[i].values.emplace_back(i); - } - return out; -} -} // namespace - TEST(ExecPlanExecution, SourceSink) { for (bool slow : {false, true}) { SCOPED_TRACE(slow ? "slowed" : "unslowed"); diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index d1d06e773f0..91c993c101e 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -156,40 +156,6 @@ ExecBatch ExecBatchFromJSON(const std::vector& descrs, return batch; } -Result MakeTestSourceNode(ExecPlan* plan, std::string label, - BatchesWithSchema batches_with_schema, bool parallel, - bool slow) { - DCHECK_GT(batches_with_schema.batches.size(), 0); - - auto opt_batches = ::arrow::internal::MapVector( - [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, - std::move(batches_with_schema.batches)); - - AsyncGenerator> gen; - - if (parallel) { - // emulate batches completing initial decode-after-scan on a cpu thread - ARROW_ASSIGN_OR_RAISE( - gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - ::arrow::internal::GetCpuThreadPool())); - - // ensure that callbacks are not executed immediately on a background thread - gen = MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); - } else { - gen = MakeVectorGenerator(std::move(opt_batches)); - } - - if (slow) { - gen = MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { - SleepABit(); - return batch; - }); - } - - return MakeSourceNode(plan, std::move(label), std::move(batches_with_schema.schema), - std::move(gen)); -} - Future> StartAndCollect( ExecPlan* plan, AsyncGenerator> gen) { RETURN_NOT_OK(plan->Validate()); diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/compute/exec/test_util.h index 3ef1333ea42..55c971954ea 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/compute/exec/test_util.h @@ -17,6 +17,9 @@ #pragma once +#include +#include + #include #include #include @@ -45,12 +48,39 @@ ExecBatch ExecBatchFromJSON(const std::vector& descrs, struct BatchesWithSchema { std::vector batches; std::shared_ptr schema; -}; -ARROW_TESTING_EXPORT -Result MakeTestSourceNode(ExecPlan* plan, std::string label, - BatchesWithSchema batches_with_schema, bool parallel, - bool slow); + AsyncGenerator> gen(bool parallel, bool slow) const { + DCHECK_GT(batches.size(), 0); + + auto opt_batches = ::arrow::internal::MapVector( + [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, batches); + + AsyncGenerator> gen; + + if (parallel) { + // emulate batches completing initial decode-after-scan on a cpu thread + gen = MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), + ::arrow::internal::GetCpuThreadPool()) + .ValueOrDie(); + + // ensure that callbacks are not executed immediately on a background thread + gen = + MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); + } else { + gen = MakeVectorGenerator(std::move(opt_batches)); + } + + if (slow) { + gen = + MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { + SleepABit(); + return batch; + }); + } + + return gen; + } +}; ARROW_TESTING_EXPORT Future> StartAndCollect( diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index eecc617c9c0..3079315a5b4 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -20,6 +20,7 @@ #include "arrow/compute/exec/exec_plan.h" #include "arrow/util/bit_util.h" #include "arrow/util/bitmap_ops.h" +#include "arrow/util/thread_pool.h" #include "arrow/util/ubsan.h" namespace arrow { @@ -296,5 +297,26 @@ Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inpu return Status::OK(); } +size_t ThreadIndexer::operator()() { + auto id = std::this_thread::get_id(); + + auto guard = mutex_.Lock(); // acquire the lock + const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; + + return Check(id_index.second); +} + +size_t ThreadIndexer::Capacity() { + static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); + return max_size; +} + +size_t ThreadIndexer::Check(size_t thread_index) { + DCHECK_LT(thread_index, Capacity()) + << "thread index " << thread_index << " is out of range [0, " << Capacity() << ")"; + + return thread_index; +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index b7cf0aeaa5e..a67b4a4dc6c 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -19,6 +19,8 @@ #include #include +#include +#include #include #include "arrow/buffer.h" @@ -29,6 +31,7 @@ #include "arrow/util/bit_util.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" +#include "arrow/util/mutex.h" #include "arrow/util/optional.h" #if defined(__clang__) || defined(__GNUC__) @@ -229,5 +232,18 @@ class AtomicCounter { std::atomic complete_{false}; }; +class ThreadIndexer { + public: + size_t operator()(); + + static size_t Capacity(); + + private: + static size_t Check(size_t thread_index); + + util::Mutex mutex_; + std::unordered_map id_to_index_; +}; + } // namespace compute } // namespace arrow From c0ba9951b6dac65e9d0ff468ab7f70a93b4b1ac9 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 10 Aug 2021 16:02:34 -0400 Subject: [PATCH 23/37] porting tests --- cpp/src/arrow/compute/exec/hash_join_test.cc | 43 +++++++++++--------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_test.cc index 816d6adaf7a..8a1035bf4d5 100644 --- a/cpp/src/arrow/compute/exec/hash_join_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_test.cc @@ -18,6 +18,7 @@ #include #include "arrow/api.h" +#include "arrow/compute/exec/options.h" #include "arrow/compute/exec/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" @@ -49,28 +50,30 @@ void GenerateBatchesFromString(const std::shared_ptr& schema, out_batches->schema = schema; } -void CheckRunOutput(JoinType type, BatchesWithSchema l_batches, - BatchesWithSchema r_batches, - const std::vector& left_keys, - const std::vector& right_keys, +void CheckRunOutput(JoinType type, const BatchesWithSchema& l_batches, + const BatchesWithSchema& r_batches, + const std::vector& left_keys, + const std::vector& right_keys, const BatchesWithSchema& exp_batches, bool parallel = false) { SCOPED_TRACE("serial"); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ASSERT_OK_AND_ASSIGN(auto l_source, - MakeTestSourceNode(plan.get(), "l_source", std::move(l_batches), - /*parallel=*/parallel, - /*slow=*/false)); - ASSERT_OK_AND_ASSIGN(auto r_source, - MakeTestSourceNode(plan.get(), "r_source", std::move(r_batches), - /*parallel=*/parallel, - /*slow=*/false)); + JoinNodeOptions join_options{type, left_keys, right_keys}; + Declaration join{"hash_join", join_options}; - ASSERT_OK_AND_ASSIGN( - auto semi_join, - MakeHashJoinNode(type, l_source, r_source, "hash_join", left_keys, right_keys)); - auto sink_gen = MakeSinkNode(semi_join, "sink"); + // add left source + join.inputs.emplace_back(Declaration{ + "source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, + /*slow=*/false)}}); + // add right source + join.inputs.emplace_back(Declaration{ + "source", SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, + /*slow=*/false)}}); + AsyncGenerator> sink_gen; + + ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}}) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray(exp_batches.batches)))); @@ -121,8 +124,8 @@ void RunNonEmptyTest(JoinType type, bool parallel) { FAIL() << "join type not implemented!"; } - CheckRunOutput(type, std::move(l_batches), std::move(r_batches), - /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, exp_batches, + CheckRunOutput(type, l_batches, r_batches, + /*left_keys=*/{{"l_str"}}, /*right_keys=*/{{"r_str"}}, exp_batches, parallel); } @@ -142,8 +145,8 @@ void RunEmptyTest(JoinType type, bool parallel) { GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, &r_n_empty, multiplicity); - std::vector l_keys{"l_str"}; - std::vector r_keys{"r_str"}; + std::vector l_keys{{"l_str"}}; + std::vector r_keys{{"r_str"}}; switch (type) { case LEFT_SEMI: From d943b8bf22149775c84b5afbda5c34cb9a19d93c Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 11 Aug 2021 00:21:33 -0400 Subject: [PATCH 24/37] extending the test cases --- cpp/src/arrow/CMakeLists.txt | 2 +- cpp/src/arrow/compute/exec/CMakeLists.txt | 3 +- .../exec/{hash_join.cc => hash_join_node.cc} | 0 .../compute/exec/hash_join_node_benchmark.cc | 19 +++++++ ...sh_join_test.cc => hash_join_node_test.cc} | 55 ++++++++++++++++++- cpp/src/arrow/compute/exec/test_util.cc | 2 + 6 files changed, 78 insertions(+), 3 deletions(-) rename cpp/src/arrow/compute/exec/{hash_join.cc => hash_join_node.cc} (100%) create mode 100644 cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc rename cpp/src/arrow/compute/exec/{hash_join_test.cc => hash_join_node_test.cc} (77%) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 0145ec83472..e329a1274fa 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -411,7 +411,7 @@ if(ARROW_COMPUTE) compute/kernels/vector_replace.cc compute/kernels/vector_selection.cc compute/kernels/vector_sort.cc - compute/exec/hash_join.cc + compute/exec/hash_join_node.cc compute/exec/key_hash.cc compute/exec/key_map.cc compute/exec/key_compare.cc diff --git a/cpp/src/arrow/compute/exec/CMakeLists.txt b/cpp/src/arrow/compute/exec/CMakeLists.txt index 281154e3518..030685c68b6 100644 --- a/cpp/src/arrow/compute/exec/CMakeLists.txt +++ b/cpp/src/arrow/compute/exec/CMakeLists.txt @@ -25,6 +25,7 @@ add_arrow_compute_test(expression_test subtree_test.cc) add_arrow_compute_test(plan_test PREFIX "arrow-compute") -add_arrow_compute_test(hash_join_test PREFIX "arrow-compute") +add_arrow_compute_test(hash_join_node_test PREFIX "arrow-compute") add_arrow_benchmark(expression_benchmark PREFIX "arrow-compute") +add_arrow_benchmark(hash_join_node_benchmark PREFIX "arrow-compute") diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc similarity index 100% rename from cpp/src/arrow/compute/exec/hash_join.cc rename to cpp/src/arrow/compute/exec/hash_join_node.cc diff --git a/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc new file mode 100644 index 00000000000..1f109ed3664 --- /dev/null +++ b/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc @@ -0,0 +1,19 @@ +// 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 "benchmark/benchmark.h" + diff --git a/cpp/src/arrow/compute/exec/hash_join_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc similarity index 77% rename from cpp/src/arrow/compute/exec/hash_join_test.cc rename to cpp/src/arrow/compute/exec/hash_join_node_test.cc index 8a1035bf4d5..7e058a6864d 100644 --- a/cpp/src/arrow/compute/exec/hash_join_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -202,9 +202,62 @@ TEST_P(HashJoinTest, TestSemiJoins) { RunNonEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } -TEST_P(HashJoinTest, TestSemiJoinsLeftEmpty) { +TEST_P(HashJoinTest, TestSemiJoinstEmpty) { RunEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } +void TestJoinRandom(const std::shared_ptr& data_type, JoinType type, + bool parallel, int num_batches, int batch_size) { + auto l_schema = schema({field("l0", data_type), field("l1", data_type)}); + auto r_schema = schema({field("r0", data_type), field("r1", data_type)}); + + // generate data + auto l_batches = MakeRandomBatches(l_schema, num_batches, batch_size); + auto r_batches = MakeRandomBatches(r_schema, num_batches, batch_size); + + std::vector left_keys{{"l0"}}; + std::vector right_keys{{"r1"}}; + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + JoinNodeOptions join_options{type, left_keys, right_keys}; + Declaration join{"hash_join", join_options}; + + // add left source + join.inputs.emplace_back(Declaration{ + "source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, + /*slow=*/false)}}); + // add right source + join.inputs.emplace_back(Declaration{ + "source", SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, + /*slow=*/false)}}); + AsyncGenerator> sink_gen; + + ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}}) + .AddToPlan(plan.get())); + + ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); + + // TODO(niranda) add a verification step for res +} + +class HashJoinTestRand : public testing::TestWithParam< + std::tuple, JoinType, bool>> {}; + +static constexpr int kNumBatches = 1000; +static constexpr int kBatchSize = 100; + +INSTANTIATE_TEST_SUITE_P( + HashJoinTestRand, HashJoinTestRand, + ::testing::Combine(::testing::Values(int8(), int32(), int64(), float32(), float64()), + ::testing::Values(JoinType::LEFT_SEMI, JoinType::RIGHT_SEMI, + JoinType::LEFT_ANTI, JoinType::RIGHT_ANTI), + ::testing::Values(false, true))); + +TEST_P(HashJoinTestRand, TestingTypes) { + TestJoinRandom(std::get<0>(GetParam()), std::get<1>(GetParam()), + std::get<2>(GetParam()), kNumBatches, kBatchSize); +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index 91c993c101e..46905f25ac8 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -193,6 +193,8 @@ BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, // add a tag scalar to ensure the batches are unique out.batches[i].values.emplace_back(i); } + + out.schema = schema; return out; } From efe0b4d45e94821cd7b572b3572df20879885bfe Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 20 Aug 2021 08:59:45 -1000 Subject: [PATCH 25/37] ARROW-13268: The source node was not transferring executors when marking the input finished. --- cpp/src/arrow/compute/exec/source_node.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/source_node.cc b/cpp/src/arrow/compute/exec/source_node.cc index 46f753bb12a..78d50f3ed1b 100644 --- a/cpp/src/arrow/compute/exec/source_node.cc +++ b/cpp/src/arrow/compute/exec/source_node.cc @@ -111,7 +111,9 @@ struct SourceNode : ExecNode { return Break(seq); }, options); - }).Then([&](int seq) { outputs_[0]->InputFinished(this, seq); }); + }) + .Then([&](int seq) { outputs_[0]->InputFinished(this, seq); }, + [](const Status& err) { return err; }, options); return Status::OK(); } From 523b618d4e7317bc8a09ca7025ae4688c07b0bdc Mon Sep 17 00:00:00 2001 From: niranda perera Date: Sat, 21 Aug 2021 09:30:28 -0400 Subject: [PATCH 26/37] removing seq_number --- cpp/src/arrow/compute/exec/hash_join_node.cc | 62 ++++++++++---------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 8bffe826dad..c0202f207eb 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -209,14 +209,13 @@ struct HashSemiJoinNode : ExecNode { for (auto&& cached : cached_probe_batches) { if (executor) { Status lambda_status; - RETURN_NOT_OK(executor->Spawn([&] { - lambda_status = ConsumeProbeBatch(cached.first, std::move(cached.second)); - })); + RETURN_NOT_OK(executor->Spawn( + [&] { lambda_status = ConsumeProbeBatch(std::move(cached)); })); // if the lambda execution failed internally, return status RETURN_NOT_OK(lambda_status); } else { - RETURN_NOT_OK(ConsumeProbeBatch(cached.first, std::move(cached.second))); + RETURN_NOT_OK(ConsumeProbeBatch(std::move(cached))); } } // cached vector will be cleared. exec batches are expected to be moved to the @@ -229,11 +228,11 @@ struct HashSemiJoinNode : ExecNode { return Status::OK(); } - Status GenerateOutput(int seq, const ArrayData& group_ids_data, ExecBatch batch) { + Status GenerateOutput(const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == batch.length) { // All NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; - outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); + ARROW_LOG(DEBUG) << "output seq:"; + outputs_[0]->InputReceived(this, batch.Slice(0, 0)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered auto filter_arr = std::make_shared(group_ids_data.length, group_ids_data.buffers[0], @@ -246,11 +245,13 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; - outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + ARROW_LOG(DEBUG) << "output seq:" + << " " << out_batch.length; + outputs_[0]->InputReceived(this, std::move(out_batch)); } else { // all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; - outputs_[0]->InputReceived(this, seq, std::move(batch)); + ARROW_LOG(DEBUG) << "output seq:" + << " " << batch.length; + outputs_[0]->InputReceived(this, std::move(batch)); } return Status::OK(); @@ -258,8 +259,8 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. - Status ConsumeProbeBatch(int seq, ExecBatch batch) { - ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:" << seq; + Status ConsumeProbeBatch(ExecBatch batch) { + ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:"; auto& final_grouper = *local_states_[build_result_index].grouper; @@ -275,7 +276,7 @@ struct HashSemiJoinNode : ExecNode { ARROW_ASSIGN_OR_RAISE(Datum group_ids, final_grouper.Find(key_batch)); auto group_ids_data = *group_ids.array(); - RETURN_NOT_OK(GenerateOutput(seq, group_ids_data, std::move(batch))); + RETURN_NOT_OK(GenerateOutput(group_ids_data, std::move(batch))); if (out_counter_.Increment()) { finished_.MarkFinished(); @@ -287,14 +288,13 @@ struct HashSemiJoinNode : ExecNode { // if cached_probe_batches_consumed is true, by the time a thread acquires // cached_probe_batches_mutex, it should no longer be cached! instead, it can be // directly consumed! - bool AttemptToCacheProbeBatch(int seq_num, ExecBatch* batch) { - ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " seq:" << seq_num - << " len:" << batch->length; + bool AttemptToCacheProbeBatch(ExecBatch* batch) { + ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " len:" << batch->length; std::lock_guard lck(cached_probe_batches_mutex); if (cached_probe_batches_consumed) { return false; } - cached_probe_batches.emplace_back(seq_num, std::move(*batch)); + cached_probe_batches.push_back(std::move(*batch)); return true; } @@ -302,9 +302,9 @@ struct HashSemiJoinNode : ExecNode { // If all build side batches received? continue streaming using probing // else cache the batches in thread-local state - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + void InputReceived(ExecNode* input, ExecBatch batch) override { ARROW_LOG(DEBUG) << "input received input:" << (IsBuildInput(input) ? "b" : "p") - << " seq:" << seq << " len:" << batch.length; + << " seq:" << 0 << " len:" << batch.length; ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); @@ -323,11 +323,11 @@ struct HashSemiJoinNode : ExecNode { // guaranteed that some thread has already called the ConsumeCachedProbeBatches // consume this probe batch - ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); + ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); } else { // build side not completed. Cache this batch! - if (!AttemptToCacheProbeBatch(seq, &batch)) { + if (!AttemptToCacheProbeBatch(&batch)) { // if the cache attempt fails, consume the batch - ErrorIfNotOk(ConsumeProbeBatch(seq, std::move(batch))); + ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); } } } @@ -438,7 +438,7 @@ struct HashSemiJoinNode : ExecNode { bool hash_table_built_; std::mutex cached_probe_batches_mutex; - std::vector> cached_probe_batches{}; + std::vector cached_probe_batches{}; // a flag is required to indicate if the cached probe batches have already been // consumed! if cached_probe_batches_consumed is true, by the time a thread aquires // cached_probe_batches_mutex, it should no longer be cached! instead, it can be @@ -449,12 +449,12 @@ struct HashSemiJoinNode : ExecNode { // template specialization for anti joins. For anti joins, group_ids_data needs to be // inverted. Output will be taken for indices which are NULL template <> -Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_ids_data, +Status HashSemiJoinNode::GenerateOutput(const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == group_ids_data.length) { // All NULLS! hence, all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << batch.length; - outputs_[0]->InputReceived(this, seq, std::move(batch)); + ARROW_LOG(DEBUG) << "output seq: " << batch.length; + outputs_[0]->InputReceived(this, std::move(batch)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered // invert the validity buffer arrow::internal::InvertBitmap( @@ -472,12 +472,12 @@ Status HashSemiJoinNode::GenerateOutput(int seq, const ArrayData& group_id Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << seq << " " << out_batch.length; - outputs_[0]->InputReceived(this, seq, std::move(out_batch)); + ARROW_LOG(DEBUG) << "output seq:" << out_batch.length; + outputs_[0]->InputReceived(this, std::move(out_batch)); } else { // No NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:" << seq << " 0"; - outputs_[0]->InputReceived(this, seq, batch.Slice(0, 0)); + ARROW_LOG(DEBUG) << "output seq:"; + outputs_[0]->InputReceived(this, batch.Slice(0, 0)); } return Status::OK(); } From d5887843160953325c5817bb7699a5c2742a1108 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 24 Aug 2021 20:34:47 -0400 Subject: [PATCH 27/37] attempting to fix the threading issue --- cpp/src/arrow/compute/exec/hash_join_node.cc | 21 +++++++++++-------- .../compute/exec/hash_join_node_benchmark.cc | 1 - .../arrow/compute/exec/hash_join_node_test.cc | 12 ++++++++--- 3 files changed, 21 insertions(+), 13 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index c0202f207eb..d232a4da28c 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -206,21 +206,24 @@ struct HashSemiJoinNode : ExecNode { if (!cached_probe_batches_consumed) { auto executor = ctx_->executor(); - for (auto&& cached : cached_probe_batches) { + + while (!cached_probe_batches.empty()) { + ExecBatch cached = std::move(cached_probe_batches.back()); + cached_probe_batches.pop_back(); + if (executor) { - Status lambda_status; RETURN_NOT_OK(executor->Spawn( - [&] { lambda_status = ConsumeProbeBatch(std::move(cached)); })); - - // if the lambda execution failed internally, return status - RETURN_NOT_OK(lambda_status); + // since cached will be going out-of-scope, it needs to be copied into the + // capture list + [&, cached]() mutable { + // since batch consumption is done asynchronously, a failed status would + // have to be propagated then and there! + ErrorIfNotOk(ConsumeProbeBatch(std::move(cached))); + })); } else { RETURN_NOT_OK(ConsumeProbeBatch(std::move(cached))); } } - // cached vector will be cleared. exec batches are expected to be moved to the - // lambdas - cached_probe_batches.clear(); } // set flag diff --git a/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc index 1f109ed3664..91daa7d5295 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc @@ -16,4 +16,3 @@ // under the License. #include "benchmark/benchmark.h" - diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 7e058a6864d..274d55ebcc8 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -22,6 +22,8 @@ #include "arrow/compute/exec/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" +#include "arrow/util/make_unique.h" +#include "arrow/util/thread_pool.h" using testing::UnorderedElementsAreArray; @@ -55,9 +57,10 @@ void CheckRunOutput(JoinType type, const BatchesWithSchema& l_batches, const std::vector& left_keys, const std::vector& right_keys, const BatchesWithSchema& exp_batches, bool parallel = false) { - SCOPED_TRACE("serial"); + auto exec_ctx = arrow::internal::make_unique( + default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); JoinNodeOptions join_options{type, left_keys, right_keys}; Declaration join{"hash_join", join_options}; @@ -218,7 +221,10 @@ void TestJoinRandom(const std::shared_ptr& data_type, JoinType type, std::vector left_keys{{"l0"}}; std::vector right_keys{{"r1"}}; - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + auto exec_ctx = arrow::internal::make_unique( + default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); JoinNodeOptions join_options{type, left_keys, right_keys}; Declaration join{"hash_join", join_options}; From f340c42a4725b32fb3c0955b5cd70659331cd610 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Fri, 27 Aug 2021 12:21:26 -0400 Subject: [PATCH 28/37] incorporating PR comments --- cpp/src/arrow/compute/exec/hash_join_node.cc | 263 ++++++++---------- .../arrow/compute/exec/hash_join_node_test.cc | 57 ++-- 2 files changed, 149 insertions(+), 171 deletions(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index d232a4da28c..2f821599bba 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -59,17 +59,35 @@ Status ValidateJoinInputs(const std::shared_ptr& left_schema, Result> PopulateKeys(const Schema& schema, const std::vector& keys) { std::vector key_field_ids(keys.size()); - // Find input field indices for left key fields + // Find input field indices for key fields for (size_t i = 0; i < keys.size(); ++i) { ARROW_ASSIGN_OR_RAISE(auto match, keys[i].FindOne(schema)); key_field_ids[i] = match[0]; } return key_field_ids; } + +Result MakeEmptyExecBatch(const std::shared_ptr& schema, + MemoryPool* pool) { + std::vector values; + values.reserve(schema->num_fields()); + + for (const auto& field : schema->fields()) { + ARROW_ASSIGN_OR_RAISE(auto arr, MakeArrayOfNull(field->type(), 0, pool)); + values.emplace_back(arr); + } + + return ExecBatch{std::move(values), 0}; +} + } // namespace template -struct HashSemiJoinNode : ExecNode { +class HashSemiJoinNode : public ExecNode { + private: + struct ThreadLocalState; + + public: HashSemiJoinNode(ExecNode* build_input, ExecNode* probe_input, ExecContext* ctx, const std::vector&& build_index_field_ids, const std::vector&& probe_index_field_ids) @@ -83,15 +101,101 @@ struct HashSemiJoinNode : ExecNode { hash_table_built_(false), cached_probe_batches_consumed(false) {} - private: - struct ThreadLocalState; - - public: const char* kind_name() override { return "HashSemiJoinNode"; } - Status InitLocalStateIfNeeded(ThreadLocalState* state) { - ARROW_LOG(DEBUG) << "init state"; + // If all build side batches received, continue streaming using probing + // else cache the batches in thread-local state + void InputReceived(ExecNode* input, ExecBatch batch) override { + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); + + if (finished_.is_finished()) { + return; + } + + if (IsBuildInput(input)) { // build input batch is received + // if a build input is received when build side is completed, something's wrong! + ARROW_DCHECK(!hash_table_built_); + + ErrorIfNotOk(ConsumeBuildBatch(std::move(batch))); + } else { // probe input batch is received + if (hash_table_built_) { + // build side done, continue with probing. when hash_table_built_ is set, it is + // guaranteed that some thread has already called the ConsumeCachedProbeBatches + + // consume this probe batch + ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); + } else { // build side not completed. Cache this batch! + if (!AttemptToCacheProbeBatch(&batch)) { + // if the cache attempt fails, consume the batch + ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); + } + } + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + + outputs_[0]->ErrorReceived(this, std::move(error)); + StopProducing(); + } + + void InputFinished(ExecNode* input, int num_total) override { + // bail if StopProducing was called + if (finished_.is_finished()) return; + + ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); + + // set total for build input + if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { + // only one thread would get inside this block! + // while incrementing, if the total is reached, call BuildSideCompleted. + ErrorIfNotOk(BuildSideCompleted()); + return; + } + + // output will be streamed from the probe side. So, they will have the same total. + if (out_counter_.SetTotal(num_total)) { + // if out_counter has completed, the future is finished! + ErrorIfNotOk(ConsumeCachedProbeBatches()); + outputs_[0]->InputFinished(this, num_total); + finished_.MarkFinished(); + } else { + outputs_[0]->InputFinished(this, num_total); + } + } + + Status StartProducing() override { + finished_ = Future<>::Make(); + + local_states_.resize(ThreadIndexer::Capacity()); + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + DCHECK_EQ(output, outputs_[0]); + + if (build_counter_.Cancel()) { + finished_.MarkFinished(); + } else if (out_counter_.Cancel()) { + finished_.MarkFinished(); + } + + for (auto&& input : inputs_) { + input->StopProducing(this); + } + } + + void StopProducing() override { outputs_[0]->StopProducing(); } + + Future<> finished() override { return finished_; } + private: + Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema auto build_schema = inputs_[0]->output_schema(); @@ -124,15 +228,12 @@ struct HashSemiJoinNode : ExecNode { } } ARROW_DCHECK(build_result_index > -1); - ARROW_LOG(DEBUG) << "build_result_index " << build_result_index; } // Performs the housekeeping work after the build-side is completed. // Note: this method is not thread safe, and hence should be guaranteed that it is // not accessed concurrently! Status BuildSideCompleted() { - ARROW_LOG(DEBUG) << "build side merge"; - // if the hash table has already been built, return if (hash_table_built_) return Status::OK(); @@ -169,9 +270,6 @@ struct HashSemiJoinNode : ExecNode { size_t thread_index = get_thread_index_(); ARROW_DCHECK(thread_index < local_states_.size()); - ARROW_LOG(DEBUG) << "ConsumeBuildBatch tid:" << thread_index - << " len:" << batch.length; - auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -197,9 +295,6 @@ struct HashSemiJoinNode : ExecNode { // consumes cached probe batches by invoking executor::Spawn. Status ConsumeCachedProbeBatches() { - ARROW_LOG(DEBUG) << "ConsumeCachedProbeBatches tid:" << get_thread_index_() - << " len:" << cached_probe_batches.size(); - // acquire the mutex to access cached_probe_batches, because while consuming, other // batches should not be cached! std::lock_guard lck(cached_probe_batches_mutex); @@ -234,8 +329,9 @@ struct HashSemiJoinNode : ExecNode { Status GenerateOutput(const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == batch.length) { // All NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:"; - outputs_[0]->InputReceived(this, batch.Slice(0, 0)); + ARROW_ASSIGN_OR_RAISE(auto empty_batch, + MakeEmptyExecBatch(output_schema_, ctx_->memory_pool())); + outputs_[0]->InputReceived(this, std::move(empty_batch)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered auto filter_arr = std::make_shared(group_ids_data.length, group_ids_data.buffers[0], @@ -248,12 +344,8 @@ struct HashSemiJoinNode : ExecNode { Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" - << " " << out_batch.length; outputs_[0]->InputReceived(this, std::move(out_batch)); } else { // all values are valid for output - ARROW_LOG(DEBUG) << "output seq:" - << " " << batch.length; outputs_[0]->InputReceived(this, std::move(batch)); } @@ -263,8 +355,6 @@ struct HashSemiJoinNode : ExecNode { // consumes a probe batch and increment probe batches count. Probing would query the // grouper[build_result_index] which have been merged with all others. Status ConsumeProbeBatch(ExecBatch batch) { - ARROW_LOG(DEBUG) << "ConsumeProbeBatch seq:"; - auto& final_grouper = *local_states_[build_result_index].grouper; // Create a batch with key columns @@ -292,7 +382,6 @@ struct HashSemiJoinNode : ExecNode { // cached_probe_batches_mutex, it should no longer be cached! instead, it can be // directly consumed! bool AttemptToCacheProbeBatch(ExecBatch* batch) { - ARROW_LOG(DEBUG) << "cache tid:" << get_thread_index_() << " len:" << batch->length; std::lock_guard lck(cached_probe_batches_mutex); if (cached_probe_batches_consumed) { return false; @@ -301,121 +390,8 @@ struct HashSemiJoinNode : ExecNode { return true; } - inline bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } - - // If all build side batches received? continue streaming using probing - // else cache the batches in thread-local state - void InputReceived(ExecNode* input, ExecBatch batch) override { - ARROW_LOG(DEBUG) << "input received input:" << (IsBuildInput(input) ? "b" : "p") - << " seq:" << 0 << " len:" << batch.length; - - ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - - if (finished_.is_finished()) { - return; - } - - if (IsBuildInput(input)) { // build input batch is received - // if a build input is received when build side is completed, something's wrong! - ARROW_DCHECK(!hash_table_built_); + bool IsBuildInput(ExecNode* input) { return input == inputs_[0]; } - ErrorIfNotOk(ConsumeBuildBatch(std::move(batch))); - } else { // probe input batch is received - if (hash_table_built_) { - // build side done, continue with probing. when hash_table_built_ is set, it is - // guaranteed that some thread has already called the ConsumeCachedProbeBatches - - // consume this probe batch - ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); - } else { // build side not completed. Cache this batch! - if (!AttemptToCacheProbeBatch(&batch)) { - // if the cache attempt fails, consume the batch - ErrorIfNotOk(ConsumeProbeBatch(std::move(batch))); - } - } - } - } - - void ErrorReceived(ExecNode* input, Status error) override { - ARROW_LOG(DEBUG) << "error received " << error.ToString(); - DCHECK_EQ(input, inputs_[0]); - - outputs_[0]->ErrorReceived(this, std::move(error)); - StopProducing(); - } - - void InputFinished(ExecNode* input, int num_total) override { - ARROW_LOG(DEBUG) << "input finished input:" << (IsBuildInput(input) ? "b" : "p") - << " tot:" << num_total; - - // bail if StopProducing was called - if (finished_.is_finished()) return; - - ARROW_DCHECK(input == inputs_[0] || input == inputs_[1]); - - // set total for build input - if (IsBuildInput(input) && build_counter_.SetTotal(num_total)) { - // only one thread would get inside this block! - // while incrementing, if the total is reached, call BuildSideCompleted. - ErrorIfNotOk(BuildSideCompleted()); - return; - } - - // set total for probe input. If it returns that probe side has completed, nothing to - // do, because probing inputs will be streamed to the output - // probe_counter_.SetTotal(num_total); - - // output will be streamed from the probe side. So, they will have the same total. - if (out_counter_.SetTotal(num_total)) { - // if out_counter has completed, the future is finished! - ErrorIfNotOk(ConsumeCachedProbeBatches()); - outputs_[0]->InputFinished(this, num_total); - finished_.MarkFinished(); - } else { - outputs_[0]->InputFinished(this, num_total); - } - } - - Status StartProducing() override { - ARROW_LOG(DEBUG) << "start prod"; - finished_ = Future<>::Make(); - - local_states_.resize(ThreadIndexer::Capacity()); - return Status::OK(); - } - - void PauseProducing(ExecNode* output) override {} - - void ResumeProducing(ExecNode* output) override {} - - void StopProducing(ExecNode* output) override { - ARROW_LOG(DEBUG) << "stop prod from node"; - - DCHECK_EQ(output, outputs_[0]); - - if (build_counter_.Cancel()) { - finished_.MarkFinished(); - } else if (out_counter_.Cancel()) { - finished_.MarkFinished(); - } - - for (auto&& input : inputs_) { - input->StopProducing(this); - } - } - - // TODO(niranda) couldn't there be multiple outputs for a Node? - void StopProducing() override { - ARROW_LOG(DEBUG) << "stop prod "; - outputs_[0]->StopProducing(); - } - - Future<> finished() override { - ARROW_LOG(DEBUG) << "finished? " << finished_.is_finished(); - return finished_; - } - - private: struct ThreadLocalState { std::unique_ptr grouper; }; @@ -456,7 +432,6 @@ Status HashSemiJoinNode::GenerateOutput(const ArrayData& group_ids_data, ExecBatch batch) { if (group_ids_data.GetNullCount() == group_ids_data.length) { // All NULLS! hence, all values are valid for output - ARROW_LOG(DEBUG) << "output seq: " << batch.length; outputs_[0]->InputReceived(this, std::move(batch)); } else if (group_ids_data.MayHaveNulls()) { // values need to be filtered // invert the validity buffer @@ -475,12 +450,12 @@ Status HashSemiJoinNode::GenerateOutput(const ArrayData& group_ids_data, Filter(rec_batch, filter_arr, /* null_selection = DROP*/ FilterOptions::Defaults(), ctx_)); auto out_batch = ExecBatch(*filtered.record_batch()); - ARROW_LOG(DEBUG) << "output seq:" << out_batch.length; outputs_[0]->InputReceived(this, std::move(out_batch)); } else { // No NULLS! hence, there are no valid outputs! - ARROW_LOG(DEBUG) << "output seq:"; - outputs_[0]->InputReceived(this, batch.Slice(0, 0)); + ARROW_ASSIGN_OR_RAISE(auto empty_batch, + MakeEmptyExecBatch(output_schema_, ctx_->memory_pool())); + outputs_[0]->InputReceived(this, std::move(empty_batch)); } return Status::OK(); } diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 274d55ebcc8..6f5ff86b899 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -30,26 +30,28 @@ using testing::UnorderedElementsAreArray; namespace arrow { namespace compute { -void GenerateBatchesFromString(const std::shared_ptr& schema, - const std::vector& json_strings, - BatchesWithSchema* out_batches, int multiplicity = 1) { +BatchesWithSchema GenerateBatchesFromString( + const std::shared_ptr& schema, + const std::vector& json_strings, int multiplicity = 1) { + BatchesWithSchema out_batches{{}, schema}; + std::vector descrs; for (auto&& field : schema->fields()) { descrs.emplace_back(field->type()); } for (auto&& s : json_strings) { - out_batches->batches.push_back(ExecBatchFromJSON(descrs, s)); + out_batches.batches.push_back(ExecBatchFromJSON(descrs, s)); } - size_t batch_count = out_batches->batches.size(); + size_t batch_count = out_batches.batches.size(); for (int repeat = 1; repeat < multiplicity; ++repeat) { for (size_t i = 0; i < batch_count; ++i) { - out_batches->batches.push_back(out_batches->batches[i]); + out_batches.batches.push_back(out_batches.batches[i]); } } - out_batches->schema = schema; + return out_batches; } void CheckRunOutput(JoinType type, const BatchesWithSchema& l_batches, @@ -89,35 +91,36 @@ void RunNonEmptyTest(JoinType type, bool parallel) { int multiplicity = parallel ? 100 : 1; - GenerateBatchesFromString(l_schema, - {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", - R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &l_batches, multiplicity); + l_batches = GenerateBatchesFromString( + l_schema, + {R"([[0,"d"], [1,"b"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", + R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, + multiplicity); - GenerateBatchesFromString( + r_batches = GenerateBatchesFromString( r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])", R"([["c", 3], ["g", 4]])", R"([["e", 5]])"}, - &r_batches, multiplicity); + multiplicity); switch (type) { case LEFT_SEMI: - GenerateBatchesFromString( + exp_batches = GenerateBatchesFromString( l_schema, {R"([[1,"b"]])", R"([])", R"([[5,"b"], [6,"c"], [7,"e"], [8,"e"]])"}, - &exp_batches, multiplicity); + multiplicity); break; case RIGHT_SEMI: - GenerateBatchesFromString( + exp_batches = GenerateBatchesFromString( r_schema, {R"([["b", 1], ["b", 2]])", R"([["c", 3]])", R"([["e", 5]])"}, - &exp_batches, multiplicity); + multiplicity); break; case LEFT_ANTI: - GenerateBatchesFromString( + exp_batches = GenerateBatchesFromString( l_schema, {R"([[0,"d"]])", R"([[2,"d"], [3,"a"], [4,"a"]])", R"([])"}, - &exp_batches, multiplicity); + multiplicity); break; case RIGHT_ANTI: - GenerateBatchesFromString(r_schema, {R"([["f", 0]])", R"([["g", 4]])", R"([])"}, - &exp_batches, multiplicity); + exp_batches = GenerateBatchesFromString( + r_schema, {R"([["f", 0]])", R"([["g", 4]])", R"([])"}, multiplicity); break; case INNER: case LEFT_OUTER: @@ -140,13 +143,13 @@ void RunEmptyTest(JoinType type, bool parallel) { BatchesWithSchema l_empty, r_empty, l_n_empty, r_n_empty; - GenerateBatchesFromString(l_schema, {R"([])"}, &l_empty, multiplicity); - GenerateBatchesFromString(r_schema, {R"([])"}, &r_empty, multiplicity); + l_empty = GenerateBatchesFromString(l_schema, {R"([])"}, multiplicity); + r_empty = GenerateBatchesFromString(r_schema, {R"([])"}, multiplicity); - GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])"}, &l_n_empty, - multiplicity); - GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, &r_n_empty, - multiplicity); + l_n_empty = + GenerateBatchesFromString(l_schema, {R"([[0,"d"], [1,"b"]])"}, multiplicity); + r_n_empty = GenerateBatchesFromString(r_schema, {R"([["f", 0], ["b", 1], ["b", 2]])"}, + multiplicity); std::vector l_keys{{"l_str"}}; std::vector r_keys{{"r_str"}}; From 8be570c94e41e8379b122498e5e81e2aa1272c7b Mon Sep 17 00:00:00 2001 From: niranda perera Date: Fri, 27 Aug 2021 15:22:38 -0400 Subject: [PATCH 29/37] incomplete --- .../arrow/compute/exec/hash_join_node_test.cc | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 6f5ff86b899..92710873a2b 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -17,11 +17,14 @@ #include +#include + #include "arrow/api.h" #include "arrow/compute/exec/options.h" #include "arrow/compute/exec/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" +#include "arrow/util/checked_cast.h" #include "arrow/util/make_unique.h" #include "arrow/util/thread_pool.h" @@ -212,6 +215,30 @@ TEST_P(HashJoinTest, TestSemiJoinstEmpty) { RunEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } +template +bool VerifySemiJoinOutput(int index_col, const std::vector& build_batches, + const std::vector& probe_batches, + const std::vector& output_batches, + bool anti_join = false) { + using T = typename arrow::TypeTraits::CType; + using ARRAY_T = typename arrow::TypeTraits::ArrayType; + + // populate hash set + std::unordered_set hash_set; + bool has_null = false; + for (auto&& b : build_batches) { + const std::shared_ptr& arr = b[index_col].array(); + VisitArrayDataInline( + arr, [&](T val) { hash_set.insert(val); }, [&]() { has_null = true; }); + } + + + + RecordBatchBuilder + + return true; +} + void TestJoinRandom(const std::shared_ptr& data_type, JoinType type, bool parallel, int num_batches, int batch_size) { auto l_schema = schema({field("l0", data_type), field("l1", data_type)}); From 49abd82738e7c5d8668a83a54f4ffa26a7d1cde8 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Sat, 28 Aug 2021 01:09:42 -0400 Subject: [PATCH 30/37] adding verification to joins --- cpp/src/arrow/compute/exec/exec_plan.cc | 2 + cpp/src/arrow/compute/exec/hash_join_node.cc | 92 +++++----- .../arrow/compute/exec/hash_join_node_test.cc | 168 +++++++++++++++--- 3 files changed, 189 insertions(+), 73 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 57aec22f87b..4f1e26a204e 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -314,6 +314,7 @@ void RegisterProjectNode(ExecFactoryRegistry*); void RegisterUnionNode(ExecFactoryRegistry*); void RegisterAggregateNode(ExecFactoryRegistry*); void RegisterSinkNode(ExecFactoryRegistry*); +void RegisterHashJoinNode(ExecFactoryRegistry*); } // namespace internal @@ -327,6 +328,7 @@ ExecFactoryRegistry* default_exec_factory_registry() { internal::RegisterUnionNode(this); internal::RegisterAggregateNode(this); internal::RegisterSinkNode(this); + internal::RegisterHashJoinNode(this); } Result GetFactory(const std::string& factory_name) override { diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 2f821599bba..213f8196b24 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -39,7 +39,7 @@ Status ValidateJoinInputs(const std::shared_ptr& left_schema, const std::shared_ptr& right_schema, const std::vector& left_keys, const std::vector& right_keys) { - if (left_keys.size() != right_keys.size()) { + if (left_keys.size() != right_keys.size() && left_keys.size() > 0) { return Status::Invalid("left and right key sizes do not match"); } @@ -481,50 +481,52 @@ Result MakeHashSemiJoinNode(ExecNode* build_input, ExecNode* probe_in build_input, probe_input, ctx, std::move(build_key_ids), std::move(probe_key_ids)); } -ExecFactoryRegistry::AddOnLoad kRegisterHashJoin( - "hash_join", - [](ExecPlan* plan, std::vector inputs, - const ExecNodeOptions& options) -> Result { - RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 2, "HashJoinNode")); - - const auto& join_options = checked_cast(options); - - static std::string join_type_string[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", - "RIGHT_ANTI", "INNER", "LEFT_OUTER", - "RIGHT_OUTER", "FULL_OUTER"}; - - auto join_type = join_options.join_type; - - ExecNode* left_input = inputs[0]; - ExecNode* right_input = inputs[1]; - const auto& left_keys = join_options.left_keys; - const auto& right_keys = join_options.right_keys; - - switch (join_type) { - case LEFT_SEMI: - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, right_keys, left_keys); - case RIGHT_SEMI: - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, left_keys, right_keys); - case LEFT_ANTI: - // left join--> build from right and probe from left - return MakeHashSemiJoinNode(right_input, left_input, right_keys, - left_keys); - case RIGHT_ANTI: - // right join--> build from left and probe from right - return MakeHashSemiJoinNode(left_input, right_input, left_keys, - right_keys); - case INNER: - case LEFT_OUTER: - case RIGHT_OUTER: - case FULL_OUTER: - return Status::NotImplemented(join_type_string[join_type] + - " joins not implemented!"); - default: - return Status::Invalid("invalid join type"); - } - }); +Result MakeHashJoinNode(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 2, "HashJoinNode")); + + const auto& join_options = checked_cast(options); + + static std::string join_type_string[] = {"LEFT_SEMI", "RIGHT_SEMI", "LEFT_ANTI", + "RIGHT_ANTI", "INNER", "LEFT_OUTER", + "RIGHT_OUTER", "FULL_OUTER"}; + + auto join_type = join_options.join_type; + + ExecNode* left_input = inputs[0]; + ExecNode* right_input = inputs[1]; + const auto& left_keys = join_options.left_keys; + const auto& right_keys = join_options.right_keys; + + switch (join_type) { + case LEFT_SEMI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, right_keys, left_keys); + case RIGHT_SEMI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, left_keys, right_keys); + case LEFT_ANTI: + // left join--> build from right and probe from left + return MakeHashSemiJoinNode(right_input, left_input, right_keys, left_keys); + case RIGHT_ANTI: + // right join--> build from left and probe from right + return MakeHashSemiJoinNode(left_input, right_input, left_keys, right_keys); + case INNER: + case LEFT_OUTER: + case RIGHT_OUTER: + case FULL_OUTER: + return Status::NotImplemented(join_type_string[join_type] + + " joins not implemented!"); + default: + return Status::Invalid("invalid join type"); + } +} + +namespace internal { +void RegisterHashJoinNode(ExecFactoryRegistry* registry) { + DCHECK_OK(registry->AddFactory("hash_join", MakeHashJoinNode)); +} +} // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index 92710873a2b..5efb14012d9 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -22,6 +22,7 @@ #include "arrow/api.h" #include "arrow/compute/exec/options.h" #include "arrow/compute/exec/test_util.h" +#include "arrow/compute/kernels/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" #include "arrow/util/checked_cast.h" @@ -215,32 +216,104 @@ TEST_P(HashJoinTest, TestSemiJoinstEmpty) { RunEmptyTest(std::get<0>(GetParam()), std::get<1>(GetParam())); } -template -bool VerifySemiJoinOutput(int index_col, const std::vector& build_batches, - const std::vector& probe_batches, - const std::vector& output_batches, - bool anti_join = false) { - using T = typename arrow::TypeTraits::CType; - using ARRAY_T = typename arrow::TypeTraits::ArrayType; - +template +static Status SimpleVerifySemiJoinOutputImpl(int index_col, + const std::shared_ptr& schema, + const std::vector& build_batches, + const std::vector& probe_batches, + const std::vector& output_batches, + bool anti_join = false) { // populate hash set - std::unordered_set hash_set; + std::unordered_set hash_set; bool has_null = false; for (auto&& b : build_batches) { const std::shared_ptr& arr = b[index_col].array(); VisitArrayDataInline( - arr, [&](T val) { hash_set.insert(val); }, [&]() { has_null = true; }); + *arr, [&](C_TYPE val) { hash_set.insert(val); }, [&]() { has_null = true; }); } + // probe hash set + RecordBatchVector exp_batches; + exp_batches.reserve(probe_batches.size()); + for (auto&& b : probe_batches) { + const std::shared_ptr& arr = b[index_col].array(); + + BooleanBuilder builder; + RETURN_NOT_OK(builder.Reserve(arr->length)); + VisitArrayDataInline( + *arr, + [&](C_TYPE val) { + auto res = hash_set.find(val); + // setting anti_join, would invert res != hash_set.end() + builder.UnsafeAppend(anti_join != (res != hash_set.end())); + }, + [&]() { builder.UnsafeAppend(anti_join != has_null); }); + + ARROW_ASSIGN_OR_RAISE(auto filter, builder.Finish()); + + ARROW_ASSIGN_OR_RAISE(auto rec_batch, b.ToRecordBatch(schema)); + ARROW_ASSIGN_OR_RAISE(auto filtered, Filter(rec_batch, filter)); + exp_batches.push_back(filtered.record_batch()); + } - RecordBatchBuilder + ARROW_ASSIGN_OR_RAISE(auto exp_table, Table::FromRecordBatches(exp_batches)); + std::vector sort_keys; + for (auto&& f : schema->fields()) { + sort_keys.emplace_back(f->name()); + } + ARROW_ASSIGN_OR_RAISE(auto exp_table_sort_ids, + SortIndices(exp_table, SortOptions(sort_keys))); + ARROW_ASSIGN_OR_RAISE(auto exp_table_sorted, Take(exp_table, exp_table_sort_ids)); + + // create a table from output batches + RecordBatchVector output_rbs; + for (auto&& b : output_batches) { + ARROW_ASSIGN_OR_RAISE(auto rb, b.ToRecordBatch(schema)); + output_rbs.push_back(std::move(rb)); + } - return true; + ARROW_ASSIGN_OR_RAISE(auto out_table, Table::FromRecordBatches(output_rbs)); + ARROW_ASSIGN_OR_RAISE(auto out_table_sort_ids, + SortIndices(exp_table, SortOptions(sort_keys))); + ARROW_ASSIGN_OR_RAISE(auto out_table_sorted, Take(exp_table, exp_table_sort_ids)); + + AssertTablesEqual(*exp_table_sorted.table(), *out_table_sorted.table(), + /*same_chunk_layout=*/false, /*flatten=*/true); + + return Status::OK(); } -void TestJoinRandom(const std::shared_ptr& data_type, JoinType type, - bool parallel, int num_batches, int batch_size) { +template +struct SimpleVerifySemiJoinOutput {}; + +template +struct SimpleVerifySemiJoinOutput> { + static Status Verify(int index_col, const std::shared_ptr& schema, + const std::vector& build_batches, + const std::vector& probe_batches, + const std::vector& output_batches, + bool anti_join = false) { + return SimpleVerifySemiJoinOutputImpl::CType>( + index_col, schema, build_batches, probe_batches, output_batches, anti_join); + } +}; + +template +struct SimpleVerifySemiJoinOutput> { + static Status Verify(int index_col, const std::shared_ptr& schema, + const std::vector& build_batches, + const std::vector& probe_batches, + const std::vector& output_batches, + bool anti_join = false) { + return SimpleVerifySemiJoinOutputImpl( + index_col, schema, build_batches, probe_batches, output_batches, anti_join); + } +}; + +template +void TestSemiJoinRandom(JoinType type, bool parallel, int num_batches, int batch_size) { + auto data_type = default_type_instance(); auto l_schema = schema({field("l0", data_type), field("l1", data_type)}); auto r_schema = schema({field("r0", data_type), field("r1", data_type)}); @@ -274,25 +347,64 @@ void TestJoinRandom(const std::shared_ptr& data_type, JoinType type, ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); - // TODO(niranda) add a verification step for res + // verification step for res + switch (type) { + case LEFT_SEMI: + ASSERT_OK(SimpleVerifySemiJoinOutput::Verify( + 0, l_schema, r_batches.batches, l_batches.batches, res)); + return; + case RIGHT_SEMI: + ASSERT_OK(SimpleVerifySemiJoinOutput::Verify( + 0, r_schema, l_batches.batches, r_batches.batches, res)); + return; + case LEFT_ANTI: + ASSERT_OK(SimpleVerifySemiJoinOutput::Verify( + 0, l_schema, r_batches.batches, l_batches.batches, res, true)); + return; + case RIGHT_ANTI: + ASSERT_OK(SimpleVerifySemiJoinOutput::Verify( + 0, l_schema, r_batches.batches, l_batches.batches, res, true)); + return; + default: + FAIL() << "Unsupported join type"; + } } -class HashJoinTestRand : public testing::TestWithParam< - std::tuple, JoinType, bool>> {}; +static constexpr int kNumBatches = 100; +static constexpr int kBatchSize = 10; -static constexpr int kNumBatches = 1000; -static constexpr int kBatchSize = 100; +using TestingTypes = ::testing::Types; -INSTANTIATE_TEST_SUITE_P( - HashJoinTestRand, HashJoinTestRand, - ::testing::Combine(::testing::Values(int8(), int32(), int64(), float32(), float64()), - ::testing::Values(JoinType::LEFT_SEMI, JoinType::RIGHT_SEMI, - JoinType::LEFT_ANTI, JoinType::RIGHT_ANTI), - ::testing::Values(false, true))); +template +class HashJoinTestRand : public testing::Test {}; + +TYPED_TEST_SUITE(HashJoinTestRand, TestingTypes); + +TYPED_TEST(HashJoinTestRand, LeftSemiJoin) { + for (bool parallel : {false, true}) { + TestSemiJoinRandom(JoinType::LEFT_SEMI, parallel, kNumBatches, kBatchSize); + } +} + +TYPED_TEST(HashJoinTestRand, RightSemiJoin) { + for (bool parallel : {false, true}) { + TestSemiJoinRandom(JoinType::RIGHT_SEMI, parallel, kNumBatches, + kBatchSize); + } +} -TEST_P(HashJoinTestRand, TestingTypes) { - TestJoinRandom(std::get<0>(GetParam()), std::get<1>(GetParam()), - std::get<2>(GetParam()), kNumBatches, kBatchSize); +TYPED_TEST(HashJoinTestRand, LeftAntiJoin) { + for (bool parallel : {false, true}) { + TestSemiJoinRandom(JoinType::LEFT_ANTI, parallel, kNumBatches, kBatchSize); + } +} + +TYPED_TEST(HashJoinTestRand, RightAntiJoin) { + for (bool parallel : {false, true}) { + TestSemiJoinRandom(JoinType::RIGHT_ANTI, parallel, kNumBatches, + kBatchSize); + } } } // namespace compute From 83e5105f7e176443ace376ce905d310382d34b3f Mon Sep 17 00:00:00 2001 From: niranda perera Date: Sat, 28 Aug 2021 10:37:25 -0400 Subject: [PATCH 31/37] minor change --- cpp/src/arrow/compute/exec/hash_join_node.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 213f8196b24..ebebd072496 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -227,7 +227,7 @@ class HashSemiJoinNode : public ExecNode { build_result_index = i; } } - ARROW_DCHECK(build_result_index > -1); + DCHECK_GT(build_result_index, -1); } // Performs the housekeeping work after the build-side is completed. From e652193b2d3ff5be8734f5e9c319efd7f858af47 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Mon, 30 Aug 2021 13:33:38 -0400 Subject: [PATCH 32/37] minor change --- cpp/src/arrow/compute/exec/hash_join_node.cc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index ebebd072496..2642e9ed617 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -39,7 +39,11 @@ Status ValidateJoinInputs(const std::shared_ptr& left_schema, const std::shared_ptr& right_schema, const std::vector& left_keys, const std::vector& right_keys) { - if (left_keys.size() != right_keys.size() && left_keys.size() > 0) { + if (left_keys.empty() || right_keys.empty()) { + return Status::Invalid("left and right key sizes can not be empty"); + } + + if (left_keys.size() != right_keys.size()) { return Status::Invalid("left and right key sizes do not match"); } From c74d1dfc2438c14e749a6907340d518084009b96 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 1 Sep 2021 16:37:51 -0400 Subject: [PATCH 33/37] changing threadIndexer to use GetThreadId --- cpp/src/arrow/compute/exec/CMakeLists.txt | 1 - .../compute/exec/hash_join_node_benchmark.cc | 18 ------------------ cpp/src/arrow/compute/exec/util.cc | 3 ++- cpp/src/arrow/compute/exec/util.h | 3 +-- .../arrow/compute/kernels/hash_aggregate.cc | 8 ++++---- 5 files changed, 7 insertions(+), 26 deletions(-) delete mode 100644 cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc diff --git a/cpp/src/arrow/compute/exec/CMakeLists.txt b/cpp/src/arrow/compute/exec/CMakeLists.txt index 718e32df500..0d1c0f6df29 100644 --- a/cpp/src/arrow/compute/exec/CMakeLists.txt +++ b/cpp/src/arrow/compute/exec/CMakeLists.txt @@ -29,4 +29,3 @@ add_arrow_compute_test(hash_join_node_test PREFIX "arrow-compute") add_arrow_compute_test(union_node_test PREFIX "arrow-compute") add_arrow_benchmark(expression_benchmark PREFIX "arrow-compute") -add_arrow_benchmark(hash_join_node_benchmark PREFIX "arrow-compute") diff --git a/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc b/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc deleted file mode 100644 index 91daa7d5295..00000000000 --- a/cpp/src/arrow/compute/exec/hash_join_node_benchmark.cc +++ /dev/null @@ -1,18 +0,0 @@ -// 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 "benchmark/benchmark.h" diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index 1d50c7a374d..3db241ad8e4 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -21,6 +21,7 @@ #include "arrow/table.h" #include "arrow/util/bit_util.h" #include "arrow/util/bitmap_ops.h" +#include "arrow/util/io_util.h" #include "arrow/util/thread_pool.h" #include "arrow/util/ubsan.h" @@ -299,7 +300,7 @@ Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inpu } size_t ThreadIndexer::operator()() { - auto id = std::this_thread::get_id(); + auto id = internal::GetThreadId(); auto guard = mutex_.Lock(); // acquire the lock const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 24306329bd6..5394a1249c0 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -19,7 +19,6 @@ #include #include -#include #include #include @@ -246,7 +245,7 @@ class ThreadIndexer { static size_t Check(size_t thread_index); util::Mutex mutex_; - std::unordered_map id_to_index_; + std::unordered_map id_to_index_; }; } // namespace compute diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index e833fc0b9da..3ca9b914f25 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -382,15 +382,15 @@ struct GrouperImpl : Grouper { int32_t total_length = 0; for (int64_t i = 0; i < batch.length; ++i) { auto total_length_before = total_length; - total_length += offsets_batch->at(i); - offsets_batch->at(i) = total_length_before; + total_length += (*offsets_batch)[i]; + (*offsets_batch)[i] = total_length_before; } - offsets_batch->at(batch.length) = total_length; + (*offsets_batch)[batch.length] = total_length; key_bytes_batch->resize(total_length); key_buf_ptrs->resize(batch.length); for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs->at(i) = key_bytes_batch->data() + offsets_batch->at(i); + key_buf_ptrs->at(i) = key_bytes_batch->data() + (*offsets_batch)[i]; } for (int i = 0; i < batch.num_values(); ++i) { From a2a74394a03337efc81bfa87bab42489dbcf6a42 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Wed, 1 Sep 2021 21:37:24 -0400 Subject: [PATCH 34/37] minor fix --- cpp/src/arrow/compute/exec/util.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index 3db241ad8e4..0d02b332aec 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -300,7 +300,7 @@ Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inpu } size_t ThreadIndexer::operator()() { - auto id = internal::GetThreadId(); + auto id = arrow::internal::GetThreadId(); auto guard = mutex_.Lock(); // acquire the lock const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; From 179f01e6e4a59f21be13499f2f165fb6f08cfc40 Mon Sep 17 00:00:00 2001 From: David Li Date: Wed, 29 Sep 2021 10:50:22 -0400 Subject: [PATCH 35/37] ARROW-13268: [C++] Fix build error --- cpp/src/arrow/compute/exec/hash_join_node.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index 2642e9ed617..9a867081f1b 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -105,7 +105,7 @@ class HashSemiJoinNode : public ExecNode { hash_table_built_(false), cached_probe_batches_consumed(false) {} - const char* kind_name() override { return "HashSemiJoinNode"; } + const char* kind_name() const override { return "HashSemiJoinNode"; } // If all build side batches received, continue streaming using probing // else cache the batches in thread-local state From 9160cc4b68df6f89ec5bfce4b6bd4d0b27cbf21f Mon Sep 17 00:00:00 2001 From: David Li Date: Wed, 29 Sep 2021 14:39:12 -0400 Subject: [PATCH 36/37] ARROW-13268: [C++] Fix build error --- cpp/src/arrow/compute/kernels/hash_aggregate.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index fb902d53a9f..63e3aa7aec7 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -450,7 +450,7 @@ struct GrouperImpl : Grouper { std::vector* key_buf_ptrs) const { offsets_batch->resize(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { - encoders_[i]->AddLength(batch[i], batch.length, offsets_batch.data()); + encoders_[i]->AddLength(batch[i], batch.length, offsets_batch->data()); } int32_t total_length = 0; @@ -468,7 +468,7 @@ struct GrouperImpl : Grouper { } for (int i = 0; i < batch.num_values(); ++i) { - RETURN_NOT_OK(encoders_[i]->Encode(batch[i], batch.length, key_buf_ptrs.data())); + RETURN_NOT_OK(encoders_[i]->Encode(batch[i], batch.length, key_buf_ptrs->data())); } return Status::OK(); From 81f822718bbc4521f7ae64c76f05337e1f556428 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 30 Sep 2021 01:35:51 -0400 Subject: [PATCH 37/37] adding grouper fast find --- cpp/src/arrow/compute/api_aggregate.h | 2 +- .../arrow/compute/kernels/hash_aggregate.cc | 68 ++++++++++++++----- 2 files changed, 51 insertions(+), 19 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index f1092a07a93..e4e8646de70 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -413,7 +413,7 @@ class ARROW_EXPORT Grouper { /// Finds/ queries the group IDs for the given ExecBatch for every index. Returns the /// group IDs as an integer array. If a group ID not found, a null will be added to that /// index. This is a thread-safe lookup. - virtual Result Find(const ExecBatch& batch) const = 0; + virtual Result Find(const ExecBatch& batch) = 0; /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 63e3aa7aec7..900cdd2e585 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -42,6 +42,7 @@ #include "arrow/compute/kernels/util_internal.h" #include "arrow/record_batch.h" #include "arrow/util/bit_run_reader.h" +#include "arrow/util/bitmap.h" #include "arrow/util/bitmap_ops.h" #include "arrow/util/bitmap_writer.h" #include "arrow/util/checked_cast.h" @@ -510,7 +511,7 @@ struct GrouperImpl : Grouper { return Datum(UInt32Array(batch.length, std::move(group_ids))); } - Result Find(const ExecBatch& batch) const override { + Result Find(const ExecBatch& batch) override { std::vector offsets_batch; std::vector key_bytes_batch; std::vector key_buf_ptrs; @@ -677,6 +678,7 @@ struct GrouperFastImpl : Grouper { return ConsumeImpl(batch); } + template Result ConsumeImpl(const ExecBatch& batch) { int64_t num_rows = batch.length; int num_columns = batch.num_values(); @@ -701,6 +703,15 @@ struct GrouperFastImpl : Grouper { ARROW_ASSIGN_OR_RAISE( group_ids, AllocateBuffer(sizeof(uint32_t) * num_rows, ctx_->memory_pool())); + std::shared_ptr group_ids_validity; + if (Find) { + ARROW_ASSIGN_OR_RAISE( + group_ids_validity, + AllocateBitmap(sizeof(uint32_t) * num_rows, ctx_->memory_pool())); + } else { + group_ids_validity = nullptr; + } + for (int icol = 0; icol < num_columns; ++icol) { const uint8_t* non_nulls = nullptr; if (batch[icol].array()->buffers[0] != NULLPTR) { @@ -746,16 +757,23 @@ struct GrouperFastImpl : Grouper { match_bitvector.mutable_data(), local_slots.mutable_data(), reinterpret_cast(group_ids->mutable_data()) + start_row); } - auto ids = util::TempVectorHolder(&temp_stack_, batch_size_next); - int num_ids; - util::BitUtil::bits_to_indexes(0, encode_ctx_.hardware_flags, batch_size_next, - match_bitvector.mutable_data(), &num_ids, - ids.mutable_data()); - - RETURN_NOT_OK(map_.map_new_keys( - num_ids, ids.mutable_data(), minibatch_hashes_.data(), - reinterpret_cast(group_ids->mutable_data()) + start_row)); + if (Find) { + // In find mode, don't insert the new keys. just copy the match bitvector to + // the group_ids_validity buffer. Valid group_ids are already populated. + arrow::internal::CopyBitmap(match_bitvector.mutable_data(), 0, batch_size_next, + group_ids_validity->mutable_data(), start_row); + } else { + auto ids = util::TempVectorHolder(&temp_stack_, batch_size_next); + int num_ids; + util::BitUtil::bits_to_indexes(0, encode_ctx_.hardware_flags, batch_size_next, + match_bitvector.mutable_data(), &num_ids, + ids.mutable_data()); + + RETURN_NOT_OK(map_.map_new_keys( + num_ids, ids.mutable_data(), minibatch_hashes_.data(), + reinterpret_cast(group_ids->mutable_data()) + start_row)); + } start_row += batch_size_next; if (minibatch_size_ * 2 <= minibatch_size_max_) { @@ -763,12 +781,27 @@ struct GrouperFastImpl : Grouper { } } - return Datum(UInt32Array(batch.length, std::move(group_ids))); + return Datum( + UInt32Array(batch.length, std::move(group_ids), std::move(group_ids_validity))); } - Result Find(const ExecBatch& batch) const override { - // todo impl this - return Result(); + Result Find(const ExecBatch& batch) override { + // ARROW-14027: broadcast scalar arguments for now + for (int i = 0; i < batch.num_values(); i++) { + if (batch.values[i].is_scalar()) { + ExecBatch expanded = batch; + for (int j = i; j < expanded.num_values(); j++) { + if (expanded.values[j].is_scalar()) { + ARROW_ASSIGN_OR_RAISE( + expanded.values[j], + MakeArrayFromScalar(*expanded.values[j].scalar(), expanded.length, + ctx_->memory_pool())); + } + } + return ConsumeImpl(expanded); + } + } + return ConsumeImpl(batch); } uint32_t num_groups() const override { return static_cast(rows_.length()); } @@ -2590,10 +2623,9 @@ Result ResolveKernels( Result> Grouper::Make(const std::vector& descrs, ExecContext* ctx) { - // TODO(niranda) re-enable this! - // if (GrouperFastImpl::CanUse(descrs)) { - // return GrouperFastImpl::Make(descrs, ctx); - // } + if (GrouperFastImpl::CanUse(descrs)) { + return GrouperFastImpl::Make(descrs, ctx); + } return GrouperImpl::Make(descrs, ctx); }