From 8fe9d7e1b5905e50134bc942f921b4e842533416 Mon Sep 17 00:00:00 2001 From: michalursa Date: Mon, 5 Jul 2021 23:14:36 -0700 Subject: [PATCH 01/41] ARROW-12759: [C++][Compute] Add ExecNode for group by --- cpp/src/arrow/compute/exec/exec_plan.cc | 248 ++++++++++++++++++ cpp/src/arrow/compute/exec/exec_plan.h | 9 + cpp/src/arrow/compute/exec/plan_test.cc | 12 +- cpp/src/arrow/compute/exec/test_util.cc | 8 +- .../arrow/compute/kernels/hash_aggregate.cc | 4 +- 5 files changed, 269 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 2dcbfb24724..472e2f47d3b 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -565,5 +565,253 @@ AsyncGenerator> MakeSinkNode(ExecNode* input, return out; } +struct GroupByNode : ExecNode { + GroupByNode(ExecNode* input, std::string label, std::shared_ptr output_schema, + ExecContext* ctx, const std::vector&& key_field_ids, + std::unique_ptr&& grouper, + const std::vector&& agg_src_field_ids, + const std::vector&& agg_kernels, + std::vector>&& agg_states) + : ExecNode(input->plan(), std::move(label), {input}, {"groupby"}, + std::move(output_schema), /*num_outputs=*/1), + ctx_(ctx), + key_field_ids_(std::move(key_field_ids)), + grouper_(std::move(grouper)), + agg_src_field_ids_(std::move(agg_src_field_ids)), + agg_kernels_(std::move(agg_kernels)), + agg_states_(std::move(agg_states)) {} + + const char* kind_name() override { return "GroupByNode"; } + + Status ProcessInputBatch(const ExecBatch& 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)); + + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper_->Consume(key_batch)); + + // Execute aggregate kernels + auto num_groups = grouper_->num_groups(); + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext kernel_ctx{ctx_}; + kernel_ctx.SetState(agg_states_[i].get()); + ARROW_ASSIGN_OR_RAISE( + auto agg_batch, ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch, + Datum(num_groups)})); + RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + } + + return Status::OK(); + } + + Status OutputResult() { + // Finalize output + ArrayDataVector out_data(agg_kernels_.size() + key_field_ids_.size()); + auto it = out_data.begin(); + + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + batch_ctx.SetState(agg_states_[i].get()); + Datum out; + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); + *it++ = out.array(); + } + + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, grouper_->GetUniques()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } + + uint32_t num_groups = grouper_->num_groups(); + int num_result_batches = (num_groups + output_batch_size_ - 1) / output_batch_size_; + outputs_[0]->InputFinished(this, num_result_batches); + + for (int i = 0; i < num_result_batches; ++i) { + // Check finished flag + if (finished_) { + break; + } + + // Slice arrays + int64_t batch_start = i * output_batch_size_; + int64_t batch_length = + std::min(output_batch_size_, static_cast(num_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, i, output_batch); + } + + return Status::OK(); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + std::unique_lock lock(mutex_); + + Status status = ProcessInputBatch(batch); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + + ++num_input_batches_processed_; + if (num_input_batches_processed_ == num_input_batches_total_) { + 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]); + + std::unique_lock lock(mutex_); + + num_input_batches_total_ = seq; + } + + Status StartProducing() override { 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); + + std::unique_lock lock(mutex_); + + finished_ = true; + } + + void StopProducing() override { StopProducing(outputs_[0]); } + + private: + ExecContext* ctx_; + std::mutex mutex_; + bool finished_{false}; + int num_input_batches_processed_{0}; + int num_input_batches_total_{-1}; + int output_batch_size_{32 * 1024}; + + const std::vector key_field_ids_; + std::unique_ptr grouper_; + const std::vector agg_src_field_ids_; + const std::vector agg_kernels_; + std::vector> agg_states_; +}; + +namespace internal { +Result> GetKernels( + ExecContext* ctx, const std::vector& aggregates, + const std::vector& in_descrs); +Result>> InitKernels( + const std::vector& kernels, ExecContext* ctx, + const std::vector& aggregates, + const std::vector& in_descrs); +Result ResolveKernels( + const std::vector& aggregates, + const std::vector& kernels, + const std::vector>& states, ExecContext* ctx, + const std::vector& descrs); +} // namespace internal + +Result MakeGroupByNode(ExecNode* input, std::string label, + std::vector keys, + std::vector agg_srcs, + std::vector aggs, + ExecContext* ctx) { + // Get input schema + auto input_schema = input->output_schema(); + + // Find input field indices for key fields + std::vector key_field_ids(keys.size()); + for (size_t i = 0; i < keys.size(); ++i) { + int key_field_id = input_schema->GetFieldIndex(keys[i]); + if (key_field_id < 0) { + return Status::Invalid("Key field named '", keys[i], + "' not found or not unique in the input schema."); + } + key_field_ids[i] = key_field_id; + } + + // Build vector of key field data types + std::vector key_descrs(keys.size()); + for (size_t i = 0; i < keys.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(auto grouper, internal::Grouper::Make(key_descrs, ctx)); + + // Find input field indices for aggregates + std::vector agg_src_field_ids; + for (size_t i = 0; i < aggs.size(); ++i) { + int agg_src_field_id = input_schema->GetFieldIndex(agg_srcs[i]); + if (agg_src_field_id < 0) { + return Status::Invalid("Aggregate source field named '", agg_srcs[i], + "' not found or not unique in the input schema."); + } + } + + // Build vector of aggregate source field data types + DCHECK_EQ(agg_srcs.size(), aggs.size()); + std::vector agg_src_descrs(aggs.size()); + for (size_t i = 0; i < aggs.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()); + } + + // Construct aggregates + ARROW_ASSIGN_OR_RAISE(auto agg_kernels, + internal::GetKernels(ctx, aggs, agg_src_descrs)); + + ARROW_ASSIGN_OR_RAISE(auto agg_states, + internal::InitKernels(agg_kernels, ctx, aggs, agg_src_descrs)); + + ARROW_ASSIGN_OR_RAISE( + FieldVector agg_result_fields, + internal::ResolveKernels(aggs, agg_kernels, agg_states, ctx, agg_src_descrs)); + + // Build field vector for output schema + FieldVector output_fields{keys.size() + aggs.size()}; + + // Aggregate fields come before key fields to match the behavior of GroupBy function + for (size_t i = 0; i < aggs.size(); ++i) { + output_fields[i] = agg_result_fields[i]; + } + size_t base = aggs.size(); + for (size_t i = 0; i < keys.size(); ++i) { + int key_field_id = key_field_ids[i]; + output_fields[base + i] = input_schema->field(key_field_id); + } + + return input->plan()->EmplaceNode( + input, std::move(label), schema(std::move(output_fields)), ctx, + std::move(key_field_ids), std::move(grouper), std::move(agg_src_field_ids), + std::move(agg_kernels), std::move(agg_states)); +} + } // 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 21a757af5a1..b487f3bda9b 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -22,6 +22,7 @@ #include #include +#include "arrow/compute/api_aggregate.h" #include "arrow/compute/type_fwd.h" #include "arrow/type_fwd.h" #include "arrow/util/macros.h" @@ -264,5 +265,13 @@ ARROW_EXPORT Result MakeProjectNode(ExecNode* input, std::string label, std::vector exprs); +/// \brief Make a node which groups input rows based on key fields and computes +/// aggregates for each group +Result MakeGroupByNode(ExecNode* input, std::string label, + std::vector keys, + std::vector agg_srcs, + std::vector aggs, + ExecContext* ctx); + } // 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 75b71f97535..22a4f2c853f 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include + #include #include -#include - #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/expression.h" @@ -197,7 +197,7 @@ Result MakeTestSourceNode(ExecPlan* plan, std::string label, bool slow) { DCHECK_GT(batches_with_schema.batches.size(), 0); - auto opt_batches = internal::MapVector( + auto opt_batches = arrow::internal::MapVector( [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, std::move(batches_with_schema.batches)); @@ -207,10 +207,10 @@ Result MakeTestSourceNode(ExecPlan* plan, std::string label, // emulate batches completing initial decode-after-scan on a cpu thread ARROW_ASSIGN_OR_RAISE( gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - internal::GetCpuThreadPool())); + arrow::internal::GetCpuThreadPool())); // ensure that callbacks are not executed immediately on a background thread - gen = MakeTransferredGenerator(std::move(gen), internal::GetCpuThreadPool()); + gen = MakeTransferredGenerator(std::move(gen), arrow::internal::GetCpuThreadPool()); } else { gen = MakeVectorGenerator(std::move(opt_batches)); } @@ -236,7 +236,7 @@ Result> StartAndCollect( plan->StopProducing(); - return internal::MapVector( + return arrow::internal::MapVector( [](util::optional batch) { return std::move(*batch); }, collected); } diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index 6fbfa2a430c..389f8fbd3c2 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -17,6 +17,9 @@ #include "arrow/compute/exec/test_util.h" +#include +#include + #include #include #include @@ -26,9 +29,6 @@ #include #include -#include -#include - #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/datum.h" @@ -131,7 +131,7 @@ ExecNode* MakeDummyNode(ExecPlan* plan, std::string label, std::vector& descrs, util::string_view json) { - auto fields = internal::MapVector( + auto fields = arrow::internal::MapVector( [](const ValueDescr& descr) { return field("", descr.type); }, descrs); ExecBatch batch{*RecordBatchFromJSON(schema(std::move(fields)), json)}; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index e282035d82a..b6ffe0094be 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -1109,6 +1109,8 @@ HashAggregateKernel MakeKernel(InputType argument_type) { return kernel; } +} // namespace + Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs) { @@ -1183,8 +1185,6 @@ Result ResolveKernels( return fields; } -} // namespace - Result> Grouper::Make(const std::vector& descrs, ExecContext* ctx) { if (GrouperFastImpl::CanUse(descrs)) { From 702019224b6cc6deb2e5e3b837c1c13d431f6ab6 Mon Sep 17 00:00:00 2001 From: michalursa Date: Tue, 6 Jul 2021 17:59:17 -0700 Subject: [PATCH 02/41] ARROW-12759: [C++][Compute] Add basic unit test for group by ExecNode --- cpp/src/arrow/compute/exec/exec_plan.cc | 24 +++++++++++------ cpp/src/arrow/compute/exec/exec_plan.h | 2 +- cpp/src/arrow/compute/exec/plan_test.cc | 26 +++++++++++++++++++ .../arrow/compute/kernels/hash_aggregate.cc | 4 +++ 4 files changed, 47 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 472e2f47d3b..77640141401 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -510,9 +510,12 @@ struct SinkNode : ExecNode { void InputReceived(ExecNode* input, int seq_num, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); if (stopped_) return; + producer_.Push(std::move(batch)); + + std::unique_lock lock(mutex_); + ++num_received_; if (num_received_ == emit_stop_) { InputFinishedUnlocked(); @@ -522,8 +525,6 @@ struct SinkNode : ExecNode { DCHECK_LE(seq_num, emit_stop_); } lock.unlock(); - - producer_.Push(std::move(batch)); } void ErrorReceived(ExecNode* input, Status error) override { @@ -685,8 +686,15 @@ struct GroupByNode : ExecNode { DCHECK_EQ(input, inputs_[0]); std::unique_lock lock(mutex_); - + num_input_batches_total_ = seq; + if (num_input_batches_processed_ == num_input_batches_total_) { + Status status = OutputResult(); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + } } Status StartProducing() override { return Status::OK(); } @@ -699,8 +707,6 @@ struct GroupByNode : ExecNode { DCHECK_EQ(output, outputs_[0]); inputs_[0]->StopProducing(this); - std::unique_lock lock(mutex_); - finished_ = true; } @@ -766,13 +772,14 @@ Result MakeGroupByNode(ExecNode* input, std::string label, ARROW_ASSIGN_OR_RAISE(auto grouper, internal::Grouper::Make(key_descrs, ctx)); // Find input field indices for aggregates - std::vector agg_src_field_ids; + std::vector agg_src_field_ids(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { int agg_src_field_id = input_schema->GetFieldIndex(agg_srcs[i]); if (agg_src_field_id < 0) { return Status::Invalid("Aggregate source field named '", agg_srcs[i], "' not found or not unique in the input schema."); } + agg_src_field_ids[i] = agg_src_field_id; } // Build vector of aggregate source field data types @@ -780,7 +787,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector agg_src_descrs(aggs.size()); for (size_t i = 0; i < aggs.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()); + agg_src_descrs[i] = ValueDescr(input_schema->field(agg_src_field_id)->type(), + ValueDescr::ARRAY); } // Construct aggregates diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index b487f3bda9b..fba42399b96 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -271,7 +271,7 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, std::vector agg_srcs, std::vector aggs, - ExecContext* ctx); + ExecContext* ctx = default_exec_context()); } // 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 22a4f2c853f..f699e3b7bd1 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -382,5 +382,31 @@ TEST(ExecPlanExecution, SourceProjectSink) { "[[null, 6], [true, 7], [true, 8]]")}))); } +TEST(ExecPlanExecution, GroupByOnly) { + BatchesWithSchema input; + input.batches = { + ExecBatchFromJSON({int32(), int32()}, "[[1, 12], [2, 7], [1, 3]]"), + ExecBatchFromJSON({int32(), int32()}, "[[5, 5], [2, 3], [1, 8]]")}; + input.schema = schema({field("a", int32()), field("b", int32())}); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + ASSERT_OK_AND_ASSIGN(auto scan, MakeTestSourceNode(plan.get(), "source", input, /*parallel=*/false, /*slow=*/false)); + ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(scan, + "gby", + {"a"}, + {"b"}, + {{"hash_sum", nullptr}})); + auto sink_gen = MakeSinkNode(gby, "sink"); + + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + ResultWith(UnorderedElementsAreArray( + {ExecBatchFromJSON({int64(), int32()}, "[[23, 1], [10, 2], [5, 5]]")} + ))); +} + +TEST(ExecPlanExecution, FilterProjectGroupByFilter) { + // TODO: implement +} + } // 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 b6ffe0094be..3c97b4b8f62 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -564,6 +564,10 @@ struct GrouperFastImpl : Grouper { varlen = batch[icol].array()->buffers[2]->data(); } + if (batch[icol].array()->offset > 0) { + return Status::NotImplemented("Non-zero offset in array is not supported"); + } + cols_[icol] = arrow::compute::KeyEncoder::KeyColumnArray( col_metadata_[icol], num_rows, non_nulls, fixedlen, varlen); } From eb79dd3e7798b988e9b2240f6d74cd5bc8c00e71 Mon Sep 17 00:00:00 2001 From: michalursa Date: Mon, 5 Jul 2021 23:14:36 -0700 Subject: [PATCH 03/41] ARROW-12759: [C++][Compute] Add ExecNode for group by --- cpp/src/arrow/compute/exec/exec_plan.cc | 248 ++++++++++++++++++ cpp/src/arrow/compute/exec/exec_plan.h | 9 + cpp/src/arrow/compute/exec/plan_test.cc | 10 +- cpp/src/arrow/compute/exec/test_util.cc | 8 +- .../arrow/compute/kernels/hash_aggregate.cc | 4 +- 5 files changed, 268 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index d0d50af1ac7..b2be080ef28 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -601,5 +601,253 @@ AsyncGenerator> MakeSinkNode(ExecNode* input, return out; } +struct GroupByNode : ExecNode { + GroupByNode(ExecNode* input, std::string label, std::shared_ptr output_schema, + ExecContext* ctx, const std::vector&& key_field_ids, + std::unique_ptr&& grouper, + const std::vector&& agg_src_field_ids, + const std::vector&& agg_kernels, + std::vector>&& agg_states) + : ExecNode(input->plan(), std::move(label), {input}, {"groupby"}, + std::move(output_schema), /*num_outputs=*/1), + ctx_(ctx), + key_field_ids_(std::move(key_field_ids)), + grouper_(std::move(grouper)), + agg_src_field_ids_(std::move(agg_src_field_ids)), + agg_kernels_(std::move(agg_kernels)), + agg_states_(std::move(agg_states)) {} + + const char* kind_name() override { return "GroupByNode"; } + + Status ProcessInputBatch(const ExecBatch& 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)); + + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper_->Consume(key_batch)); + + // Execute aggregate kernels + auto num_groups = grouper_->num_groups(); + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext kernel_ctx{ctx_}; + kernel_ctx.SetState(agg_states_[i].get()); + ARROW_ASSIGN_OR_RAISE( + auto agg_batch, ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch, + Datum(num_groups)})); + RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + } + + return Status::OK(); + } + + Status OutputResult() { + // Finalize output + ArrayDataVector out_data(agg_kernels_.size() + key_field_ids_.size()); + auto it = out_data.begin(); + + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + batch_ctx.SetState(agg_states_[i].get()); + Datum out; + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out)); + *it++ = out.array(); + } + + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, grouper_->GetUniques()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } + + uint32_t num_groups = grouper_->num_groups(); + int num_result_batches = (num_groups + output_batch_size_ - 1) / output_batch_size_; + outputs_[0]->InputFinished(this, num_result_batches); + + for (int i = 0; i < num_result_batches; ++i) { + // Check finished flag + if (finished_) { + break; + } + + // Slice arrays + int64_t batch_start = i * output_batch_size_; + int64_t batch_length = + std::min(output_batch_size_, static_cast(num_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, i, output_batch); + } + + return Status::OK(); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + std::unique_lock lock(mutex_); + + Status status = ProcessInputBatch(batch); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + + ++num_input_batches_processed_; + if (num_input_batches_processed_ == num_input_batches_total_) { + 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]); + + std::unique_lock lock(mutex_); + + num_input_batches_total_ = seq; + } + + Status StartProducing() override { 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); + + std::unique_lock lock(mutex_); + + finished_ = true; + } + + void StopProducing() override { StopProducing(outputs_[0]); } + + private: + ExecContext* ctx_; + std::mutex mutex_; + bool finished_{false}; + int num_input_batches_processed_{0}; + int num_input_batches_total_{-1}; + int output_batch_size_{32 * 1024}; + + const std::vector key_field_ids_; + std::unique_ptr grouper_; + const std::vector agg_src_field_ids_; + const std::vector agg_kernels_; + std::vector> agg_states_; +}; + +namespace internal { +Result> GetKernels( + ExecContext* ctx, const std::vector& aggregates, + const std::vector& in_descrs); +Result>> InitKernels( + const std::vector& kernels, ExecContext* ctx, + const std::vector& aggregates, + const std::vector& in_descrs); +Result ResolveKernels( + const std::vector& aggregates, + const std::vector& kernels, + const std::vector>& states, ExecContext* ctx, + const std::vector& descrs); +} // namespace internal + +Result MakeGroupByNode(ExecNode* input, std::string label, + std::vector keys, + std::vector agg_srcs, + std::vector aggs, + ExecContext* ctx) { + // Get input schema + auto input_schema = input->output_schema(); + + // Find input field indices for key fields + std::vector key_field_ids(keys.size()); + for (size_t i = 0; i < keys.size(); ++i) { + int key_field_id = input_schema->GetFieldIndex(keys[i]); + if (key_field_id < 0) { + return Status::Invalid("Key field named '", keys[i], + "' not found or not unique in the input schema."); + } + key_field_ids[i] = key_field_id; + } + + // Build vector of key field data types + std::vector key_descrs(keys.size()); + for (size_t i = 0; i < keys.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(auto grouper, internal::Grouper::Make(key_descrs, ctx)); + + // Find input field indices for aggregates + std::vector agg_src_field_ids; + for (size_t i = 0; i < aggs.size(); ++i) { + int agg_src_field_id = input_schema->GetFieldIndex(agg_srcs[i]); + if (agg_src_field_id < 0) { + return Status::Invalid("Aggregate source field named '", agg_srcs[i], + "' not found or not unique in the input schema."); + } + } + + // Build vector of aggregate source field data types + DCHECK_EQ(agg_srcs.size(), aggs.size()); + std::vector agg_src_descrs(aggs.size()); + for (size_t i = 0; i < aggs.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()); + } + + // Construct aggregates + ARROW_ASSIGN_OR_RAISE(auto agg_kernels, + internal::GetKernels(ctx, aggs, agg_src_descrs)); + + ARROW_ASSIGN_OR_RAISE(auto agg_states, + internal::InitKernels(agg_kernels, ctx, aggs, agg_src_descrs)); + + ARROW_ASSIGN_OR_RAISE( + FieldVector agg_result_fields, + internal::ResolveKernels(aggs, agg_kernels, agg_states, ctx, agg_src_descrs)); + + // Build field vector for output schema + FieldVector output_fields{keys.size() + aggs.size()}; + + // Aggregate fields come before key fields to match the behavior of GroupBy function + for (size_t i = 0; i < aggs.size(); ++i) { + output_fields[i] = agg_result_fields[i]; + } + size_t base = aggs.size(); + for (size_t i = 0; i < keys.size(); ++i) { + int key_field_id = key_field_ids[i]; + output_fields[base + i] = input_schema->field(key_field_id); + } + + return input->plan()->EmplaceNode( + input, std::move(label), schema(std::move(output_fields)), ctx, + std::move(key_field_ids), std::move(grouper), std::move(agg_src_field_ids), + std::move(agg_kernels), std::move(agg_states)); +} + } // 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 6c29ddfa7a6..f3edc44f473 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -22,6 +22,7 @@ #include #include +#include "arrow/compute/api_aggregate.h" #include "arrow/compute/exec.h" #include "arrow/compute/type_fwd.h" #include "arrow/type_fwd.h" @@ -269,5 +270,13 @@ ARROW_EXPORT Result MakeProjectNode(ExecNode* input, std::string label, std::vector exprs); +/// \brief Make a node which groups input rows based on key fields and computes +/// aggregates for each group +Result MakeGroupByNode(ExecNode* input, std::string label, + std::vector keys, + std::vector agg_srcs, + std::vector aggs, + ExecContext* ctx); + } // 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 9ebafc42668..b57d7351020 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include + #include #include -#include - #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/expression.h" @@ -206,7 +206,7 @@ Result MakeTestSourceNode(ExecPlan* plan, std::string label, bool slow) { DCHECK_GT(batches_with_schema.batches.size(), 0); - auto opt_batches = internal::MapVector( + auto opt_batches = arrow::internal::MapVector( [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, std::move(batches_with_schema.batches)); @@ -216,10 +216,10 @@ Result MakeTestSourceNode(ExecPlan* plan, std::string label, // emulate batches completing initial decode-after-scan on a cpu thread ARROW_ASSIGN_OR_RAISE( gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - internal::GetCpuThreadPool())); + arrow::internal::GetCpuThreadPool())); // ensure that callbacks are not executed immediately on a background thread - gen = MakeTransferredGenerator(std::move(gen), internal::GetCpuThreadPool()); + gen = MakeTransferredGenerator(std::move(gen), arrow::internal::GetCpuThreadPool()); } else { gen = MakeVectorGenerator(std::move(opt_batches)); } diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index bd203b354f0..64def3c7c7c 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -17,6 +17,9 @@ #include "arrow/compute/exec/test_util.h" +#include +#include + #include #include #include @@ -27,9 +30,6 @@ #include #include -#include -#include - #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/datum.h" @@ -134,7 +134,7 @@ ExecNode* MakeDummyNode(ExecPlan* plan, std::string label, std::vector& descrs, util::string_view json) { - auto fields = internal::MapVector( + auto fields = arrow::internal::MapVector( [](const ValueDescr& descr) { return field("", descr.type); }, descrs); ExecBatch batch{*RecordBatchFromJSON(schema(std::move(fields)), json)}; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index ed40a6b1b8c..246c456aaae 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -1114,6 +1114,8 @@ HashAggregateKernel MakeKernel(InputType argument_type) { return kernel; } +} // namespace + Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs) { @@ -1188,8 +1190,6 @@ Result ResolveKernels( return fields; } -} // namespace - Result> Grouper::Make(const std::vector& descrs, ExecContext* ctx) { if (GrouperFastImpl::CanUse(descrs)) { From 7cbe2d62920cd1643cef37f3c7156edce74e7868 Mon Sep 17 00:00:00 2001 From: michalursa Date: Tue, 6 Jul 2021 17:59:17 -0700 Subject: [PATCH 04/41] ARROW-12759: [C++][Compute] Add basic unit test for group by ExecNode --- cpp/src/arrow/compute/exec/exec_plan.cc | 15 ++++++++++---- cpp/src/arrow/compute/exec/exec_plan.h | 2 +- cpp/src/arrow/compute/exec/plan_test.cc | 26 +++++++++++++++++++++++++ 3 files changed, 38 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index b2be080ef28..19804309cf6 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -723,6 +723,13 @@ struct GroupByNode : ExecNode { std::unique_lock lock(mutex_); num_input_batches_total_ = seq; + if (num_input_batches_processed_ == num_input_batches_total_) { + Status status = OutputResult(); + if (!status.ok()) { + ErrorReceived(input, status); + return; + } + } } Status StartProducing() override { return Status::OK(); } @@ -735,8 +742,6 @@ struct GroupByNode : ExecNode { DCHECK_EQ(output, outputs_[0]); inputs_[0]->StopProducing(this); - std::unique_lock lock(mutex_); - finished_ = true; } @@ -802,13 +807,14 @@ Result MakeGroupByNode(ExecNode* input, std::string label, ARROW_ASSIGN_OR_RAISE(auto grouper, internal::Grouper::Make(key_descrs, ctx)); // Find input field indices for aggregates - std::vector agg_src_field_ids; + std::vector agg_src_field_ids(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { int agg_src_field_id = input_schema->GetFieldIndex(agg_srcs[i]); if (agg_src_field_id < 0) { return Status::Invalid("Aggregate source field named '", agg_srcs[i], "' not found or not unique in the input schema."); } + agg_src_field_ids[i] = agg_src_field_id; } // Build vector of aggregate source field data types @@ -816,7 +822,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector agg_src_descrs(aggs.size()); for (size_t i = 0; i < aggs.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()); + agg_src_descrs[i] = + ValueDescr(input_schema->field(agg_src_field_id)->type(), ValueDescr::ARRAY); } // Construct aggregates diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index f3edc44f473..1dc864cbd07 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -276,7 +276,7 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, std::vector agg_srcs, std::vector aggs, - ExecContext* ctx); + ExecContext* ctx = default_exec_context()); } // 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 b57d7351020..e06406bea2e 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -423,5 +423,31 @@ TEST(ExecPlanExecution, SourceProjectSink) { "[[null, 6], [true, 7], [true, 8]]")})))); } +TEST(ExecPlanExecution, GroupByOnly) { + BatchesWithSchema input; + input.batches = { + ExecBatchFromJSON({int32(), int32()}, "[[1, 12], [2, 7], [1, 3]]"), + ExecBatchFromJSON({int32(), int32()}, "[[5, 5], [2, 3], [1, 8]]")}; + input.schema = schema({field("a", int32()), field("b", int32())}); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + ASSERT_OK_AND_ASSIGN(auto scan, MakeTestSourceNode(plan.get(), "source", input, /*parallel=*/false, /*slow=*/false)); + ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(scan, + "gby", + {"a"}, + {"b"}, + {{"hash_sum", nullptr}})); + auto sink_gen = MakeSinkNode(gby, "sink"); + + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + ResultWith(UnorderedElementsAreArray( + {ExecBatchFromJSON({int64(), int32()}, "[[23, 1], [10, 2], [5, 5]]")} + ))); +} + +TEST(ExecPlanExecution, FilterProjectGroupByFilter) { + // TODO: implement +} + } // namespace compute } // namespace arrow From 9e0b35c46b9bdf6c81c07c20d3bb6e19775d9713 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 7 Jul 2021 17:06:07 -0400 Subject: [PATCH 05/41] add support for merging grouped aggregation state --- cpp/src/arrow/array/builder_base.h | 2 + cpp/src/arrow/compute/api_aggregate.h | 2 +- cpp/src/arrow/compute/exec.cc | 7 + cpp/src/arrow/compute/exec.h | 2 + cpp/src/arrow/compute/kernel.h | 8 +- .../arrow/compute/kernels/hash_aggregate.cc | 177 +++++++++++++----- .../compute/kernels/hash_aggregate_test.cc | 72 +++++-- 7 files changed, 209 insertions(+), 61 deletions(-) diff --git a/cpp/src/arrow/array/builder_base.h b/cpp/src/arrow/array/builder_base.h index 8e60c306796..c2aba4e959f 100644 --- a/cpp/src/arrow/array/builder_base.h +++ b/cpp/src/arrow/array/builder_base.h @@ -50,6 +50,8 @@ class ARROW_EXPORT ArrayBuilder { public: explicit ArrayBuilder(MemoryPool* pool) : pool_(pool), null_bitmap_builder_(pool) {} + ARROW_DEFAULT_MOVE_AND_ASSIGN(ArrayBuilder); + virtual ~ArrayBuilder() = default; /// For nested types. Since the objects are owned by this class instance, we diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 7b6e2ef96de..9a88ad224ce 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -411,7 +411,7 @@ struct ARROW_EXPORT Aggregate { /// This will be replaced by streaming execution operators. ARROW_EXPORT Result GroupBy(const std::vector& arguments, const std::vector& keys, - const std::vector& aggregates, + const std::vector& aggregates, bool use_threads = false, ExecContext* ctx = default_exec_context()); } // namespace internal diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index 8a469e3fe12..c63345d58ff 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -102,6 +103,12 @@ void PrintTo(const ExecBatch& batch, std::ostream* os) { } } +std::string ExecBatch::ToString() const { + std::stringstream ss; + PrintTo(*this, &ss); + return ss.str(); +} + ExecBatch ExecBatch::Slice(int64_t offset, int64_t length) const { ExecBatch out = *this; for (auto& value : out.values) { diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index 77d04b86ceb..148e5bbfade 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -230,6 +230,8 @@ struct ARROW_EXPORT ExecBatch { return result; } + std::string ToString() const; + ARROW_EXPORT friend void PrintTo(const ExecBatch&, std::ostream*); }; diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index c88c924817c..8d4c6eacf03 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -52,7 +52,7 @@ struct ARROW_EXPORT KernelState { /// \brief Context/state for the execution of a particular kernel. class ARROW_EXPORT KernelContext { public: - explicit KernelContext(ExecContext* exec_ctx) : exec_ctx_(exec_ctx), state_() {} + explicit KernelContext(ExecContext* exec_ctx) : exec_ctx_(exec_ctx) {} /// \brief Allocate buffer from the context's memory pool. The contents are /// not initialized. @@ -80,7 +80,7 @@ class ARROW_EXPORT KernelContext { private: ExecContext* exec_ctx_; - KernelState* state_; + KernelState* state_ = NULLPTR; }; /// \brief The standard kernel execution API that must be implemented for @@ -683,8 +683,10 @@ struct ScalarAggregateKernel : public Kernel { using HashAggregateConsume = std::function; +// XXX should resize be a method here? + using HashAggregateMerge = - std::function; + std::function; // Finalize returns Datum to permit multiple return values using HashAggregateFinalize = std::function; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 246c456aaae..0f2f34a80c7 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -17,7 +17,9 @@ #include #include +#include #include +#include #include #include @@ -39,6 +41,8 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" +#include "arrow/util/task_group.h" +#include "arrow/util/thread_pool.h" #include "arrow/visitor_inline.h" namespace arrow { @@ -751,18 +755,30 @@ struct GroupedAggregator : KernelState { virtual Status Init(ExecContext*, const FunctionOptions*, const std::shared_ptr&) = 0; + virtual Status Reserve(int64_t added_groups) = 0; + virtual Status Consume(const ExecBatch& batch) = 0; + virtual Status Merge(GroupedAggregator&& other, const ExecBatch& group_id_mapping) { + // TODO(ARROW-11840) merge two hash tables + return Status::NotImplemented("Merge hashed aggregations"); + } + virtual Result Finalize() = 0; - template - Status MaybeReserve(int64_t old_num_groups, const ExecBatch& batch, - const Reserve& reserve) { - int64_t new_num_groups = batch[2].scalar_as().value; + static const uint32_t* GroupIds(const ExecBatch& batch) { + return batch[1].array()->GetValues(1); + } + + static uint32_t NumGroups(const ExecBatch& batch) { + return batch[2].scalar_as().value; + } + + Status MaybeReserve(int64_t old_num_groups, int64_t new_num_groups) { if (new_num_groups <= old_num_groups) { return Status::OK(); } - return reserve(new_num_groups - old_num_groups); + return Reserve(new_num_groups - old_num_groups); } virtual std::shared_ptr out_type() const = 0; @@ -779,11 +795,29 @@ struct GroupedCountImpl : public GroupedAggregator { return Status::OK(); } + Status Reserve(int64_t added_groups) override { + num_groups_ += added_groups; + return counts_.Append(added_groups * sizeof(int64_t), 0); + } + + Status Merge(GroupedAggregator&& other, const ExecBatch& group_id_mapping) override { + RETURN_NOT_OK(MaybeReserve( + num_groups_, group_id_mapping.values[1].scalar_as().value)); + + auto raw_mapping = group_id_mapping.values[0].array()->GetValues(1); + + auto raw_counts = reinterpret_cast(counts_.mutable_data()); + auto other_raw_counts = reinterpret_cast( + checked_cast(other).counts_.mutable_data()); + + for (int64_t i = 0; i < group_id_mapping.length; ++i) { + raw_counts[raw_mapping[i]] += other_raw_counts[i]; + } + return Status::OK(); + } + Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, batch, [&](int64_t added_groups) { - num_groups_ += added_groups; - return counts_.Append(added_groups * sizeof(int64_t), 0); - })); + RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); auto group_ids = batch[1].array()->GetValues(1); auto raw_counts = reinterpret_cast(counts_.mutable_data()); @@ -834,6 +868,8 @@ struct GroupedSumImpl : public GroupedAggregator { using ConsumeImpl = std::function&, const uint32_t*, void*, int64_t*)>; + using MergeImpl = + std::function; struct GetConsumeImpl { template ::Type> @@ -864,6 +900,7 @@ struct GroupedSumImpl : public GroupedAggregator { } ConsumeImpl consume_impl; + MergeImpl merge_impl; std::shared_ptr out_type; }; @@ -882,13 +919,15 @@ struct GroupedSumImpl : public GroupedAggregator { return Status::OK(); } + Status Reserve(int64_t added_groups) override { + num_groups_ += added_groups; + RETURN_NOT_OK(sums_.Append(added_groups * kSumSize, 0)); + RETURN_NOT_OK(counts_.Append(added_groups * sizeof(int64_t), 0)); + return Status::OK(); + } + Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, batch, [&](int64_t added_groups) { - num_groups_ += added_groups; - RETURN_NOT_OK(sums_.Append(added_groups * kSumSize, 0)); - RETURN_NOT_OK(counts_.Append(added_groups * sizeof(int64_t), 0)); - return Status::OK(); - })); + RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); auto group_ids = batch[1].array()->GetValues(1); consume_impl_(batch[0].array(), group_ids, sums_.mutable_data(), @@ -926,6 +965,7 @@ struct GroupedSumImpl : public GroupedAggregator { BufferBuilder sums_, counts_; std::shared_ptr out_type_; ConsumeImpl consume_impl_; + MergeImpl merge_impl_; MemoryPool* pool_; }; @@ -1025,15 +1065,17 @@ struct GroupedMinMaxImpl : public GroupedAggregator { return Status::OK(); } + Status Reserve(int64_t added_groups) override { + num_groups_ += added_groups; + RETURN_NOT_OK(resize_min_impl_(&mins_, added_groups)); + RETURN_NOT_OK(resize_max_impl_(&maxes_, added_groups)); + RETURN_NOT_OK(has_values_.Append(added_groups, false)); + RETURN_NOT_OK(has_nulls_.Append(added_groups, false)); + return Status::OK(); + } + Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, batch, [&](int64_t added_groups) { - num_groups_ += added_groups; - RETURN_NOT_OK(resize_min_impl_(&mins_, added_groups)); - RETURN_NOT_OK(resize_max_impl_(&maxes_, added_groups)); - RETURN_NOT_OK(has_values_.Append(added_groups, false)); - RETURN_NOT_OK(has_nulls_.Append(added_groups, false)); - return Status::OK(); - })); + RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); auto group_ids = batch[1].array()->GetValues(1); consume_impl_(batch[0].array(), group_ids, mins_.mutable_data(), @@ -1100,9 +1142,10 @@ HashAggregateKernel MakeKernel(InputType argument_type) { return checked_cast(ctx->state())->Consume(batch); }; - kernel.merge = [](KernelContext* ctx, KernelState&&, KernelState*) { - // TODO(ARROW-11840) merge two hash tables - return Status::NotImplemented("Merge hashed aggregations"); + kernel.merge = [](KernelContext* ctx, KernelState&& other, + const ExecBatch& group_id_mapping) { + return checked_cast(ctx->state()) + ->Merge(checked_cast(other), group_id_mapping); }; kernel.finalize = [](KernelContext* ctx, Datum* out) { @@ -1199,7 +1242,13 @@ Result> Grouper::Make(const std::vector& de } Result GroupBy(const std::vector& arguments, const std::vector& keys, - const std::vector& aggregates, ExecContext* ctx) { + const std::vector& aggregates, bool use_threads, + ExecContext* ctx) { + auto task_group = + use_threads + ? arrow::internal::TaskGroup::MakeThreaded(arrow::internal::GetCpuThreadPool()) + : arrow::internal::TaskGroup::MakeSerial(); + // Construct and initialize HashAggregateKernels ARROW_ASSIGN_OR_RAISE(auto argument_descrs, ExecBatch::Make(arguments).Map( @@ -1207,24 +1256,33 @@ Result GroupBy(const std::vector& arguments, const std::vector>> states( + task_group->parallelism()); + for (auto& state : states) { + ARROW_ASSIGN_OR_RAISE(state, InitKernels(kernels, ctx, aggregates, argument_descrs)); + } ARROW_ASSIGN_OR_RAISE( FieldVector out_fields, - ResolveKernels(aggregates, kernels, states, ctx, argument_descrs)); + ResolveKernels(aggregates, kernels, states[0], ctx, argument_descrs)); using arrow::compute::detail::ExecBatchIterator; ARROW_ASSIGN_OR_RAISE(auto argument_batch_iterator, ExecBatchIterator::Make(arguments, ctx->exec_chunksize())); - // Construct Grouper + // Construct Groupers ARROW_ASSIGN_OR_RAISE(auto key_descrs, ExecBatch::Make(keys).Map([](ExecBatch batch) { return batch.GetDescriptors(); })); - ARROW_ASSIGN_OR_RAISE(auto grouper, Grouper::Make(key_descrs, ctx)); + std::vector> groupers(task_group->parallelism()); + for (auto& grouper : groupers) { + ARROW_ASSIGN_OR_RAISE(grouper, Grouper::Make(key_descrs, ctx)); + } + + std::mutex mutex; + std::unordered_map thread_ids; int i = 0; for (ValueDescr& key_descr : key_descrs) { @@ -1240,16 +1298,49 @@ Result GroupBy(const std::vector& arguments, const std::vectorNext(&key_batch)) { if (key_batch.length == 0) continue; - // compute a batch of group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); + task_group->Append([&, key_batch, argument_batch] { + size_t thread_index; + { + std::unique_lock lock(mutex); + auto it = thread_ids.emplace(std::this_thread::get_id(), thread_ids.size()).first; + thread_index = it->second; + DCHECK_LT(static_cast(thread_index), task_group->parallelism()); + } + + auto grouper = groupers[thread_index].get(); - // consume group ids with HashAggregateKernels - for (size_t i = 0; i < kernels.size(); ++i) { - KernelContext batch_ctx{ctx}; - batch_ctx.SetState(states[i].get()); - ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make({argument_batch[i], id_batch, - Datum(grouper->num_groups())})); - RETURN_NOT_OK(kernels[i]->consume(&batch_ctx, batch)); + // compute a batch of group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); + + // consume group ids with HashAggregateKernels + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[thread_index][i].get()); + ARROW_ASSIGN_OR_RAISE( + auto batch, + ExecBatch::Make({argument_batch[i], id_batch, Datum(grouper->num_groups())})); + RETURN_NOT_OK(kernels[i]->consume(&batch_ctx, batch)); + } + + return Status::OK(); + }); + } + + RETURN_NOT_OK(task_group->Finish()); + + // Merge if necessary + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[0][i].get()); + + for (size_t thread_index = 1; thread_index < thread_ids.size(); ++thread_index) { + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, groupers[thread_index]->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum transposition, groupers[0]->Consume(other_keys)); + ARROW_ASSIGN_OR_RAISE( + auto group_id_mapping, + ExecBatch::Make({std::move(transposition), Datum(groupers[0]->num_groups())})); + RETURN_NOT_OK(kernels[i]->merge(&batch_ctx, std::move(*states[thread_index][i]), + group_id_mapping)); } } @@ -1259,13 +1350,13 @@ Result GroupBy(const std::vector& arguments, const std::vectorfinalize(&batch_ctx, &out)); *it++ = out.array(); } - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, grouper->GetUniques()); + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, groupers[0]->GetUniques()); for (const auto& key : out_keys.values) { *it++ = key.array(); } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 8c8a4b23932..378f3348c87 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -30,6 +30,7 @@ #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" +#include "arrow/compute/exec/test_util.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/codegen_internal.h" #include "arrow/compute/kernels/test_util.h" @@ -175,12 +176,19 @@ struct TestGrouper { } void ExpectConsume(const std::string& key_json, const std::string& expected) { - ExpectConsume(ExecBatch(*RecordBatchFromJSON(key_schema_, key_json)), + ExpectConsume(ExecBatchFromJSON(descrs_, key_json), ArrayFromJSON(uint32(), expected)); } - void ExpectConsume(const std::vector& key_batch, Datum expected) { - ExpectConsume(*ExecBatch::Make(key_batch), expected); + void ExpectConsume(const std::vector& key_values, Datum expected) { + ASSERT_OK_AND_ASSIGN(auto key_batch, ExecBatch::Make(key_values)); + ExpectConsume(key_batch, expected); + } + + void ExpectConsume(const ExecBatch& key_batch, Datum expected) { + Datum ids; + ConsumeAndValidate(key_batch, &ids); + AssertEquivalentIds(expected, ids); } void AssertEquivalentIds(const Datum& expected, const Datum& actual) { @@ -190,10 +198,8 @@ struct TestGrouper { int64_t num_ids = left->length(); auto left_data = left->data(); auto right_data = right->data(); - const uint32_t* left_ids = - reinterpret_cast(left_data->buffers[1]->data()); - const uint32_t* right_ids = - reinterpret_cast(right_data->buffers[1]->data()); + auto left_ids = reinterpret_cast(left_data->buffers[1]->data()); + auto right_ids = reinterpret_cast(right_data->buffers[1]->data()); uint32_t max_left_id = 0; uint32_t max_right_id = 0; for (int64_t i = 0; i < num_ids; ++i) { @@ -224,13 +230,6 @@ struct TestGrouper { } } - void ExpectConsume(const ExecBatch& key_batch, Datum expected) { - Datum ids; - ConsumeAndValidate(key_batch, &ids); - AssertEquivalentIds(expected, ids); - // AssertDatumsEqual(expected, ids, /*verbose=*/true); - } - void ConsumeAndValidate(const ExecBatch& key_batch, Datum* ids = nullptr) { ASSERT_OK_AND_ASSIGN(Datum id_batch, grouper_->Consume(key_batch)); @@ -520,6 +519,51 @@ TEST(GroupBy, Errors) { batch->GetColumnByName("group_id"), Datum(uint32_t(4))})); } +TEST(GroupBy, CountOnly) { + for (bool use_threads : {true}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + + auto table = + TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ + [1.0, 1], + [null, 1] + ])", + R"([ + [0.0, 2], + [null, 3], + [4.0, null], + [3.25, 1], + [0.125, 2] + ])", + R"([ + [-0.25, 2], + [0.75, null], + [null, 3] + ])"}); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_count", nullptr}, + }, + use_threads)); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("hash_count", int64()), + field("key_0", int64()), + }), + R"([ + [2, 1], + [3, 2], + [0, 3], + [2, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); + } +} + TEST(GroupBy, SumOnly) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ From fa084cbd123670d0c6c0e403ea6b06e670f39288 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 8 Jul 2021 14:57:02 -0400 Subject: [PATCH 06/41] remove confusing Grouped*Impl::*Impl-based impls --- cpp/src/arrow/compute/api_vector.h | 3 +- cpp/src/arrow/compute/kernel.h | 21 +- .../arrow/compute/kernels/hash_aggregate.cc | 452 ++++++++---------- .../compute/kernels/hash_aggregate_test.cc | 72 ++- 4 files changed, 269 insertions(+), 279 deletions(-) diff --git a/cpp/src/arrow/compute/api_vector.h b/cpp/src/arrow/compute/api_vector.h index 6021492320e..65ba5df4a4b 100644 --- a/cpp/src/arrow/compute/api_vector.h +++ b/cpp/src/arrow/compute/api_vector.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include "arrow/compute/function.h" #include "arrow/datum.h" @@ -87,7 +88,7 @@ enum class SortOrder { class ARROW_EXPORT SortKey : public util::EqualityComparable { public: explicit SortKey(std::string name, SortOrder order = SortOrder::Ascending) - : name(name), order(order) {} + : name(std::move(name)), order(order) {} using util::EqualityComparable::Equals; using util::EqualityComparable::operator==; diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index 8d4c6eacf03..fdc9d07083e 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -681,12 +681,12 @@ struct ScalarAggregateKernel : public Kernel { // ---------------------------------------------------------------------- // HashAggregateKernel (for HashAggregateFunction) -using HashAggregateConsume = std::function; +using HashAggregateResize = std::function; -// XXX should resize be a method here? +using HashAggregateConsume = std::function; using HashAggregateMerge = - std::function; + std::function; // Finalize returns Datum to permit multiple return values using HashAggregateFinalize = std::function; @@ -696,6 +696,7 @@ using HashAggregateFinalize = std::function; /// kernel are the init, consume, merge, and finalize functions. /// /// * init: creates a new KernelState for a kernel. +/// * resize: ensure that the KernelState can accommodate the specified number of groups. /// * consume: processes an ExecBatch (which includes the argument as well /// as an array of group identifiers) and updates the KernelState found in the /// KernelContext. @@ -706,20 +707,24 @@ struct HashAggregateKernel : public Kernel { HashAggregateKernel() = default; HashAggregateKernel(std::shared_ptr sig, KernelInit init, - HashAggregateConsume consume, HashAggregateMerge merge, - HashAggregateFinalize finalize) + HashAggregateResize resize, HashAggregateConsume consume, + HashAggregateMerge merge, HashAggregateFinalize finalize) : Kernel(std::move(sig), std::move(init)), + resize(std::move(resize)), consume(std::move(consume)), merge(std::move(merge)), finalize(std::move(finalize)) {} HashAggregateKernel(std::vector in_types, OutputType out_type, - KernelInit init, HashAggregateMerge merge, - HashAggregateConsume consume, HashAggregateFinalize finalize) + KernelInit init, HashAggregateConsume consume, + HashAggregateResize resize, HashAggregateMerge merge, + HashAggregateFinalize finalize) : HashAggregateKernel( KernelSignature::Make(std::move(in_types), std::move(out_type)), - std::move(init), std::move(consume), std::move(merge), std::move(finalize)) {} + std::move(init), std::move(resize), std::move(consume), std::move(merge), + std::move(finalize)) {} + HashAggregateResize resize; HashAggregateConsume consume; HashAggregateMerge merge; HashAggregateFinalize finalize; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 0f2f34a80c7..462cd799b36 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -752,59 +752,94 @@ struct GrouperFastImpl : Grouper { /// Implementations should be default constructible and perform initialization in /// Init(). struct GroupedAggregator : KernelState { - virtual Status Init(ExecContext*, const FunctionOptions*, - const std::shared_ptr&) = 0; + virtual Status Init(ExecContext*, const FunctionOptions*) = 0; - virtual Status Reserve(int64_t added_groups) = 0; + virtual Status Resize(int64_t new_num_groups) = 0; virtual Status Consume(const ExecBatch& batch) = 0; - virtual Status Merge(GroupedAggregator&& other, const ExecBatch& group_id_mapping) { + virtual Status Merge(GroupedAggregator&& other, const ArrayData& group_id_mapping) { // TODO(ARROW-11840) merge two hash tables return Status::NotImplemented("Merge hashed aggregations"); } virtual Result Finalize() = 0; - static const uint32_t* GroupIds(const ExecBatch& batch) { - return batch[1].array()->GetValues(1); - } + virtual std::shared_ptr out_type() const = 0; +}; - static uint32_t NumGroups(const ExecBatch& batch) { - return batch[2].scalar_as().value; - } +template +Result> HashAggregateInit(KernelContext* ctx, + const KernelInitArgs& args) { + auto impl = ::arrow::internal::make_unique(); + RETURN_NOT_OK(impl->Init(ctx->exec_context(), args.options)); + return std::move(impl); +} - Status MaybeReserve(int64_t old_num_groups, int64_t new_num_groups) { - if (new_num_groups <= old_num_groups) { - return Status::OK(); - } - return Reserve(new_num_groups - old_num_groups); - } +HashAggregateKernel MakeKernel(InputType argument_type, KernelInit init) { + HashAggregateKernel kernel; - virtual std::shared_ptr out_type() const = 0; -}; + kernel.init = std::move(init); + + kernel.signature = KernelSignature::Make( + {std::move(argument_type), InputType::Array(Type::UINT32)}, + OutputType( + [](KernelContext* ctx, const std::vector&) -> Result { + return checked_cast(ctx->state())->out_type(); + })); + + kernel.resize = [](KernelContext* ctx, int64_t num_groups) { + return checked_cast(ctx->state())->Resize(num_groups); + }; + + kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { + return checked_cast(ctx->state())->Consume(batch); + }; + + kernel.merge = [](KernelContext* ctx, KernelState&& other, + const ArrayData& group_id_mapping) { + return checked_cast(ctx->state()) + ->Merge(checked_cast(other), group_id_mapping); + }; + + kernel.finalize = [](KernelContext* ctx, Datum* out) { + ARROW_ASSIGN_OR_RAISE(*out, + checked_cast(ctx->state())->Finalize()); + return Status::OK(); + }; + + return kernel; +} + +Status AddHashAggKernels( + const std::vector>& types, + Result make_kernel(const std::shared_ptr&), + HashAggregateFunction* function) { + for (const auto& ty : types) { + ARROW_ASSIGN_OR_RAISE(auto kernel, make_kernel(ty)); + RETURN_NOT_OK(function->AddKernel(std::move(kernel))); + } + return Status::OK(); +} // ---------------------------------------------------------------------- // Count implementation struct GroupedCountImpl : public GroupedAggregator { - Status Init(ExecContext* ctx, const FunctionOptions* options, - const std::shared_ptr&) override { + Status Init(ExecContext* ctx, const FunctionOptions* options) override { options_ = checked_cast(*options); counts_ = BufferBuilder(ctx->memory_pool()); return Status::OK(); } - Status Reserve(int64_t added_groups) override { - num_groups_ += added_groups; + Status Resize(int64_t new_num_groups) override { + auto added_groups = new_num_groups - num_groups_; + num_groups_ = new_num_groups; return counts_.Append(added_groups * sizeof(int64_t), 0); } - Status Merge(GroupedAggregator&& other, const ExecBatch& group_id_mapping) override { - RETURN_NOT_OK(MaybeReserve( - num_groups_, group_id_mapping.values[1].scalar_as().value)); - - auto raw_mapping = group_id_mapping.values[0].array()->GetValues(1); + Status Merge(GroupedAggregator&& other, const ArrayData& group_id_mapping) override { + auto raw_mapping = group_id_mapping.GetValues(1); auto raw_counts = reinterpret_cast(counts_.mutable_data()); auto other_raw_counts = reinterpret_cast( @@ -817,8 +852,6 @@ struct GroupedCountImpl : public GroupedAggregator { } Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); - auto group_ids = batch[1].array()->GetValues(1); auto raw_counts = reinterpret_cast(counts_.mutable_data()); @@ -861,77 +894,41 @@ struct GroupedCountImpl : public GroupedAggregator { // ---------------------------------------------------------------------- // Sum implementation +template struct GroupedSumImpl : public GroupedAggregator { - // NB: whether we are accumulating into double, int64_t, or uint64_t - // we always have 64 bits per group in the sums buffer. - static constexpr size_t kSumSize = sizeof(int64_t); - - using ConsumeImpl = std::function&, - const uint32_t*, void*, int64_t*)>; - using MergeImpl = - std::function; - - struct GetConsumeImpl { - template ::Type> - Status Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, const uint32_t* group, - void* boxed_sums, int64_t* counts) { - auto sums = reinterpret_cast::CType*>(boxed_sums); - - VisitArrayDataInline( - *input, - [&](typename TypeTraits::CType value) { - sums[*group] += value; - counts[*group] += 1; - ++group; - }, - [&] { ++group; }); - }; - out_type = TypeTraits::type_singleton(); - return Status::OK(); - } - - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Summing data of type ", type); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("Summing data of type ", type); - } + using AccType = typename FindAccumulatorType::Type; - ConsumeImpl consume_impl; - MergeImpl merge_impl; - std::shared_ptr out_type; - }; - - Status Init(ExecContext* ctx, const FunctionOptions*, - const std::shared_ptr& input_type) override { + Status Init(ExecContext* ctx, const FunctionOptions*) override { pool_ = ctx->memory_pool(); sums_ = BufferBuilder(pool_); counts_ = BufferBuilder(pool_); - - GetConsumeImpl get_consume_impl; - RETURN_NOT_OK(VisitTypeInline(*input_type, &get_consume_impl)); - - consume_impl_ = std::move(get_consume_impl.consume_impl); - out_type_ = std::move(get_consume_impl.out_type); - + out_type_ = TypeTraits::type_singleton(); return Status::OK(); } - Status Reserve(int64_t added_groups) override { - num_groups_ += added_groups; - RETURN_NOT_OK(sums_.Append(added_groups * kSumSize, 0)); + Status Resize(int64_t new_num_groups) override { + auto added_groups = new_num_groups - num_groups_; + num_groups_ = new_num_groups; + RETURN_NOT_OK(sums_.Append(added_groups * sizeof(AccType), 0)); RETURN_NOT_OK(counts_.Append(added_groups * sizeof(int64_t), 0)); return Status::OK(); } Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); - - auto group_ids = batch[1].array()->GetValues(1); - consume_impl_(batch[0].array(), group_ids, sums_.mutable_data(), - reinterpret_cast(counts_.mutable_data())); + auto group = batch[1].array()->GetValues(1); + + auto sums = + reinterpret_cast::CType*>(sums_.mutable_data()); + auto counts = reinterpret_cast(counts_.mutable_data()); + + VisitArrayDataInline( + *batch[0].array(), + [&](typename TypeTraits::CType value) { + sums[*group] += value; + counts[*group] += 1; + ++group; + }, + [&] { ++group; }); return Status::OK(); } @@ -964,123 +961,94 @@ struct GroupedSumImpl : public GroupedAggregator { int64_t num_groups_ = 0; BufferBuilder sums_, counts_; std::shared_ptr out_type_; - ConsumeImpl consume_impl_; - MergeImpl merge_impl_; MemoryPool* pool_; }; +struct GroupedSumFactory { + template ::Type> + Status Visit(const T&) { + kernel = MakeKernel(std::move(argument_type), HashAggregateInit>); + return Status::OK(); + } + + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Summing data of type ", type); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("Summing data of type ", type); + } + + static Result Make(const std::shared_ptr& type) { + GroupedSumFactory factory; + factory.argument_type = InputType::Array(type); + RETURN_NOT_OK(VisitTypeInline(*type, &factory)); + return std::move(factory.kernel); + } + + HashAggregateKernel kernel; + InputType argument_type; +}; + // ---------------------------------------------------------------------- // MinMax implementation template -struct Extrema : std::numeric_limits {}; +struct AntiExtrema { + static constexpr CType anti_min() { return std::numeric_limits::max(); } + static constexpr CType anti_max() { return std::numeric_limits::min(); } +}; template <> -struct Extrema { - static constexpr float min() { return -std::numeric_limits::infinity(); } - static constexpr float max() { return std::numeric_limits::infinity(); } +struct AntiExtrema { + static constexpr float anti_min() { return std::numeric_limits::infinity(); } + static constexpr float anti_max() { return -std::numeric_limits::infinity(); } }; template <> -struct Extrema { - static constexpr double min() { return -std::numeric_limits::infinity(); } - static constexpr double max() { return std::numeric_limits::infinity(); } +struct AntiExtrema { + static constexpr double anti_min() { return std::numeric_limits::infinity(); } + static constexpr double anti_max() { return -std::numeric_limits::infinity(); } }; +template struct GroupedMinMaxImpl : public GroupedAggregator { - using ConsumeImpl = - std::function&, const uint32_t*, void*, void*, - uint8_t*, uint8_t*)>; - - using ResizeImpl = std::function; - - template - static ResizeImpl MakeResizeImpl(CType anti_extreme) { - // resize a min or max buffer, storing the correct anti extreme - return [anti_extreme](BufferBuilder* builder, int64_t added_groups) { - TypedBufferBuilder typed_builder(std::move(*builder)); - RETURN_NOT_OK(typed_builder.Append(added_groups, anti_extreme)); - *builder = std::move(*typed_builder.bytes_builder()); - return Status::OK(); - }; - } - - struct GetImpl { - template ::CType> - enable_if_number Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, const uint32_t* group, - void* mins, void* maxes, uint8_t* has_values, - uint8_t* has_nulls) { - auto raw_mins = reinterpret_cast(mins); - auto raw_maxes = reinterpret_cast(maxes); - - VisitArrayDataInline( - *input, - [&](CType val) { - raw_maxes[*group] = std::max(raw_maxes[*group], val); - raw_mins[*group] = std::min(raw_mins[*group], val); - BitUtil::SetBit(has_values, *group++); - }, - [&] { BitUtil::SetBit(has_nulls, *group++); }); - }; - - resize_min_impl = MakeResizeImpl(Extrema::max()); - resize_max_impl = MakeResizeImpl(Extrema::min()); - return Status::OK(); - } - - Status Visit(const BooleanType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - ConsumeImpl consume_impl; - ResizeImpl resize_min_impl, resize_max_impl; - }; + using CType = typename TypeTraits::CType; - Status Init(ExecContext* ctx, const FunctionOptions* options, - const std::shared_ptr& input_type) override { + Status Init(ExecContext* ctx, const FunctionOptions* options) override { options_ = *checked_cast(options); - type_ = input_type; - - mins_ = BufferBuilder(ctx->memory_pool()); - maxes_ = BufferBuilder(ctx->memory_pool()); + type_ = TypeTraits::type_singleton(); + mins_ = TypedBufferBuilder(ctx->memory_pool()); + maxes_ = TypedBufferBuilder(ctx->memory_pool()); has_values_ = TypedBufferBuilder(ctx->memory_pool()); has_nulls_ = TypedBufferBuilder(ctx->memory_pool()); - - GetImpl get_impl; - RETURN_NOT_OK(VisitTypeInline(*input_type, &get_impl)); - - consume_impl_ = std::move(get_impl.consume_impl); - resize_min_impl_ = std::move(get_impl.resize_min_impl); - resize_max_impl_ = std::move(get_impl.resize_max_impl); - return Status::OK(); } - Status Reserve(int64_t added_groups) override { - num_groups_ += added_groups; - RETURN_NOT_OK(resize_min_impl_(&mins_, added_groups)); - RETURN_NOT_OK(resize_max_impl_(&maxes_, added_groups)); + Status Resize(int64_t new_num_groups) override { + auto added_groups = new_num_groups - num_groups_; + num_groups_ = new_num_groups; + RETURN_NOT_OK(mins_.Append(added_groups, AntiExtrema::anti_min())); + RETURN_NOT_OK(maxes_.Append(added_groups, AntiExtrema::anti_max())); RETURN_NOT_OK(has_values_.Append(added_groups, false)); RETURN_NOT_OK(has_nulls_.Append(added_groups, false)); return Status::OK(); } Status Consume(const ExecBatch& batch) override { - RETURN_NOT_OK(MaybeReserve(num_groups_, NumGroups(batch))); + auto group = batch[1].array()->GetValues(1); - auto group_ids = batch[1].array()->GetValues(1); - consume_impl_(batch[0].array(), group_ids, mins_.mutable_data(), - maxes_.mutable_data(), has_values_.mutable_data(), - has_nulls_.mutable_data()); + auto raw_mins = reinterpret_cast(mins_.mutable_data()); + auto raw_maxes = reinterpret_cast(maxes_.mutable_data()); + + VisitArrayDataInline( + *batch[0].array(), + [&](CType val) { + raw_maxes[*group] = std::max(raw_maxes[*group], val); + raw_mins[*group] = std::min(raw_mins[*group], val); + BitUtil::SetBit(has_values_.mutable_data(), *group++); + }, + [&] { BitUtil::SetBit(has_nulls_.mutable_data(), *group++); }); return Status::OK(); } @@ -1109,53 +1077,38 @@ struct GroupedMinMaxImpl : public GroupedAggregator { } int64_t num_groups_; - BufferBuilder mins_, maxes_; + TypedBufferBuilder mins_, maxes_; TypedBufferBuilder has_values_, has_nulls_; std::shared_ptr type_; - ConsumeImpl consume_impl_; - ResizeImpl resize_min_impl_, resize_max_impl_; ScalarAggregateOptions options_; }; -template -HashAggregateKernel MakeKernel(InputType argument_type) { - HashAggregateKernel kernel; - - kernel.init = [](KernelContext* ctx, - const KernelInitArgs& args) -> Result> { - auto impl = ::arrow::internal::make_unique(); - // FIXME(bkietz) Init should not take a type. That should be an unboxed template arg - // for the Impl. Otherwise we're not exposing dispatch as well as we should. - RETURN_NOT_OK(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); - return std::move(impl); - }; - - kernel.signature = KernelSignature::Make( - {std::move(argument_type), InputType::Array(Type::UINT32), - InputType::Scalar(Type::UINT32)}, - OutputType( - [](KernelContext* ctx, const std::vector&) -> Result { - return checked_cast(ctx->state())->out_type(); - })); +struct GroupedMinMaxFactory { + template ::Type> + Status Visit(const T&) { + kernel = + MakeKernel(std::move(argument_type), HashAggregateInit>); + return Status::OK(); + } - kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { - return checked_cast(ctx->state())->Consume(batch); - }; + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Summing data of type ", type); + } - kernel.merge = [](KernelContext* ctx, KernelState&& other, - const ExecBatch& group_id_mapping) { - return checked_cast(ctx->state()) - ->Merge(checked_cast(other), group_id_mapping); - }; + Status Visit(const DataType& type) { + return Status::NotImplemented("Summing data of type ", type); + } - kernel.finalize = [](KernelContext* ctx, Datum* out) { - ARROW_ASSIGN_OR_RAISE(*out, - checked_cast(ctx->state())->Finalize()); - return Status::OK(); - }; + static Result Make(const std::shared_ptr& type) { + GroupedMinMaxFactory factory; + factory.argument_type = InputType::Array(type); + RETURN_NOT_OK(VisitTypeInline(*type, &factory)); + return std::move(factory.kernel); + } - return kernel; -} + HashAggregateKernel kernel; + InputType argument_type; +}; } // namespace @@ -1174,8 +1127,7 @@ Result> GetKernels( ctx->func_registry()->GetFunction(aggregates[i].function)); ARROW_ASSIGN_OR_RAISE( const Kernel* kernel, - function->DispatchExact( - {in_descrs[i], ValueDescr::Array(uint32()), ValueDescr::Scalar(uint32())})); + function->DispatchExact({in_descrs[i], ValueDescr::Array(uint32())})); kernels[i] = static_cast(kernel); } return kernels; @@ -1199,13 +1151,13 @@ Result>> InitKernels( KernelContext kernel_ctx{ctx}; ARROW_ASSIGN_OR_RAISE( - states[i], kernels[i]->init(&kernel_ctx, KernelInitArgs{kernels[i], - { - in_descrs[i].type, - uint32(), - uint32(), - }, - options})); + states[i], + kernels[i]->init(&kernel_ctx, KernelInitArgs{kernels[i], + { + in_descrs[i], + ValueDescr::Array(uint32()), + }, + options})); } return std::move(states); @@ -1224,9 +1176,8 @@ Result ResolveKernels( ARROW_ASSIGN_OR_RAISE(auto descr, kernels[i]->signature->out_type().Resolve( &kernel_ctx, { - descrs[i].type, - uint32(), - uint32(), + descrs[i], + ValueDescr::Array(uint32()), })); fields[i] = field(aggregates[i].function, std::move(descr.type)); } @@ -1316,9 +1267,8 @@ Result GroupBy(const std::vector& arguments, const std::vectornum_groups())})); + ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make({argument_batch[i], id_batch})); + RETURN_NOT_OK(kernels[i]->resize(&batch_ctx, grouper->num_groups())); RETURN_NOT_OK(kernels[i]->consume(&batch_ctx, batch)); } @@ -1329,18 +1279,19 @@ Result GroupBy(const std::vector& arguments, const std::vectorFinish()); // Merge if necessary - for (size_t i = 0; i < kernels.size(); ++i) { - KernelContext batch_ctx{ctx}; - batch_ctx.SetState(states[0][i].get()); + for (size_t thread_index = 1; thread_index < thread_ids.size(); ++thread_index) { + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, groupers[thread_index]->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum transposition, groupers[0]->Consume(other_keys)); + groupers[thread_index].reset(); - for (size_t thread_index = 1; thread_index < thread_ids.size(); ++thread_index) { - ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, groupers[thread_index]->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum transposition, groupers[0]->Consume(other_keys)); - ARROW_ASSIGN_OR_RAISE( - auto group_id_mapping, - ExecBatch::Make({std::move(transposition), Datum(groupers[0]->num_groups())})); + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[0][i].get()); + + RETURN_NOT_OK(kernels[i]->resize(&batch_ctx, groupers[0]->num_groups())); RETURN_NOT_OK(kernels[i]->merge(&batch_ctx, std::move(*states[thread_index][i]), - group_id_mapping)); + *transposition.array())); + states[thread_index][i].reset(); } } @@ -1423,18 +1374,18 @@ namespace { const FunctionDoc hash_count_doc{"Count the number of null / non-null values", ("By default, non-null values are counted.\n" "This can be changed through ScalarAggregateOptions."), - {"array", "group_id_array", "group_count"}, + {"array", "group_id_array"}, "ScalarAggregateOptions"}; const FunctionDoc hash_sum_doc{"Sum values of a numeric array", ("Null values are ignored."), - {"array", "group_id_array", "group_count"}}; + {"array", "group_id_array"}}; const FunctionDoc hash_min_max_doc{ "Compute the minimum and maximum values of a numeric array", ("Null values are ignored by default.\n" "This can be changed through ScalarAggregateOptions."), - {"array", "group_id_array", "group_count"}, + {"array", "group_id_array"}, "ScalarAggregateOptions"}; } // namespace @@ -1442,25 +1393,32 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) { { static auto default_scalar_aggregate_options = ScalarAggregateOptions::Defaults(); auto func = std::make_shared( - "hash_count", Arity::Ternary(), &hash_count_doc, + "hash_count", Arity::Binary(), &hash_count_doc, &default_scalar_aggregate_options); - DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + + DCHECK_OK(func->AddKernel( + MakeKernel(ValueDescr::ARRAY, HashAggregateInit))); DCHECK_OK(registry->AddFunction(std::move(func))); } { - auto func = std::make_shared("hash_sum", Arity::Ternary(), + auto func = std::make_shared("hash_sum", Arity::Binary(), &hash_sum_doc); - DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + DCHECK_OK(AddHashAggKernels({boolean()}, GroupedSumFactory::Make, func.get())); + DCHECK_OK(AddHashAggKernels(SignedIntTypes(), GroupedSumFactory::Make, func.get())); + DCHECK_OK(AddHashAggKernels(UnsignedIntTypes(), GroupedSumFactory::Make, func.get())); + DCHECK_OK( + AddHashAggKernels(FloatingPointTypes(), GroupedSumFactory::Make, func.get())); DCHECK_OK(registry->AddFunction(std::move(func))); } { static auto default_scalar_aggregate_options = ScalarAggregateOptions::Defaults(); auto func = std::make_shared( - "hash_min_max", Arity::Ternary(), &hash_min_max_doc, + "hash_min_max", Arity::Binary(), &hash_min_max_doc, &default_scalar_aggregate_options); - DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + DCHECK_OK(AddHashAggKernels({boolean()}, GroupedSumFactory::Make, func.get())); + DCHECK_OK(AddHashAggKernels(NumericTypes(), GroupedMinMaxFactory::Make, func.get())); DCHECK_OK(registry->AddFunction(std::move(func))); } } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 378f3348c87..b46b7122f55 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -39,6 +39,7 @@ #include "arrow/testing/generator.h" #include "arrow/testing/gtest_common.h" #include "arrow/testing/gtest_util.h" +#include "arrow/testing/matchers.h" #include "arrow/testing/random.h" #include "arrow/type.h" #include "arrow/type_traits.h" @@ -513,14 +514,27 @@ TEST(GroupBy, Errors) { [null, 3] ])"); - EXPECT_RAISES_WITH_MESSAGE_THAT( - NotImplemented, HasSubstr("Direct execution of HASH_AGGREGATE functions"), - CallFunction("hash_sum", {batch->GetColumnByName("argument"), - batch->GetColumnByName("group_id"), Datum(uint32_t(4))})); + EXPECT_THAT(CallFunction("hash_sum", {batch->GetColumnByName("argument"), + batch->GetColumnByName("group_id")}), + Raises(StatusCode::NotImplemented, + HasSubstr("Direct execution of HASH_AGGREGATE functions"))); } +namespace { +void SortBy(std::vector names, Datum* aggregated_and_grouped) { + SortOptions options{{SortKey("key_0", SortOrder::Ascending)}}; + + ASSERT_OK_AND_ASSIGN( + auto batch, RecordBatch::FromStructArray(aggregated_and_grouped->make_array())); + ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(batch, options)); + + ASSERT_OK_AND_ASSIGN(*aggregated_and_grouped, + Take(*aggregated_and_grouped, sort_indices)); +} +} // namespace + TEST(GroupBy, CountOnly) { - for (bool use_threads : {true}) { + for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); auto table = @@ -548,6 +562,7 @@ TEST(GroupBy, CountOnly) { {"hash_count", nullptr}, }, use_threads)); + SortBy({"key_0"}, &aggregated_and_grouped); AssertDatumsEqual(ArrayFromJSON(struct_({ field("hash_count", int64()), @@ -565,39 +580,50 @@ TEST(GroupBy, CountOnly) { } TEST(GroupBy, SumOnly) { - auto batch = RecordBatchFromJSON( - schema({field("argument", float64()), field("key", int64())}), R"([ + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + if (use_threads) continue; + + auto table = + TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], - [null, 1], + [null, 1] + ])", + R"([ [0.0, 2], [null, 3], [4.0, null], [3.25, 1], - [0.125, 2], + [0.125, 2] + ])", + R"([ [-0.25, 2], [0.75, null], [null, 3] - ])"); + ])"}); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({batch->GetColumnByName("argument")}, - {batch->GetColumnByName("key")}, - { - {"hash_sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_sum", nullptr}, + }, + use_threads)); + SortBy({"key_0"}, &aggregated_and_grouped); - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("hash_sum", float64()), - field("key_0", int64()), - }), - R"([ + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("hash_sum", float64()), + field("key_0", int64()), + }), + R"([ [4.25, 1], [-0.125, 2], [null, 3], [4.75, null] ])"), - aggregated_and_grouped, - /*verbose=*/true); + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, MinMaxOnly) { From 2d0f1743bf9612cdb6d344b59e99ea830922b4e6 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 8 Jul 2021 16:41:25 -0400 Subject: [PATCH 07/41] Add merge for other hash agg kernels --- .../arrow/compute/kernels/hash_aggregate.cc | 131 ++++++++++++------ .../compute/kernels/hash_aggregate_test.cc | 52 ++++--- 2 files changed, 115 insertions(+), 68 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 462cd799b36..79213b93b37 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -758,10 +758,7 @@ struct GroupedAggregator : KernelState { virtual Status Consume(const ExecBatch& batch) = 0; - virtual Status Merge(GroupedAggregator&& other, const ArrayData& group_id_mapping) { - // TODO(ARROW-11840) merge two hash tables - return Status::NotImplemented("Merge hashed aggregations"); - } + virtual Status Merge(GroupedAggregator&& other, const ArrayData& group_id_mapping) = 0; virtual Result Finalize() = 0; @@ -838,44 +835,45 @@ struct GroupedCountImpl : public GroupedAggregator { return counts_.Append(added_groups * sizeof(int64_t), 0); } - Status Merge(GroupedAggregator&& other, const ArrayData& group_id_mapping) override { - auto raw_mapping = group_id_mapping.GetValues(1); + Status Merge(GroupedAggregator&& raw_other, + const ArrayData& group_id_mapping) override { + auto other = checked_cast(&raw_other); - auto raw_counts = reinterpret_cast(counts_.mutable_data()); - auto other_raw_counts = reinterpret_cast( - checked_cast(other).counts_.mutable_data()); + auto counts = reinterpret_cast(counts_.mutable_data()); + auto other_counts = reinterpret_cast(other->counts_.mutable_data()); - for (int64_t i = 0; i < group_id_mapping.length; ++i) { - raw_counts[raw_mapping[i]] += other_raw_counts[i]; + auto g = group_id_mapping.GetValues(1); + for (int64_t other_g = 0; other_g < group_id_mapping.length; ++other_g, ++g) { + counts[*g] += other_counts[other_g]; } return Status::OK(); } Status Consume(const ExecBatch& batch) override { - auto group_ids = batch[1].array()->GetValues(1); - auto raw_counts = reinterpret_cast(counts_.mutable_data()); + auto counts = reinterpret_cast(counts_.mutable_data()); const auto& input = batch[0].array(); - if (!options_.skip_nulls) { - if (input->GetNullCount() != 0) { - for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { - auto g = group_ids[i]; - raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); - } + if (options_.skip_nulls) { + auto g_begin = + reinterpret_cast(batch[1].array()->buffers[1]->data()); + + arrow::internal::VisitSetBitRunsVoid(input->buffers[0], input->offset, + input->length, + [&](int64_t offset, int64_t length) { + auto g = g_begin + offset; + for (int64_t i = 0; i < length; ++i, ++g) { + counts[*g] += 1; + } + }); + } else if (input->MayHaveNulls()) { + auto g = batch[1].array()->GetValues(1); + + auto end = input->offset + input->length; + for (int64_t i = input->offset; i < end; ++i, ++g) { + counts[*g] += !BitUtil::GetBit(input->buffers[0]->data(), i); } - return Status::OK(); } - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_counts[g] += 1; - } - }); return Status::OK(); } @@ -897,6 +895,7 @@ struct GroupedCountImpl : public GroupedAggregator { template struct GroupedSumImpl : public GroupedAggregator { using AccType = typename FindAccumulatorType::Type; + using SumType = typename TypeTraits::CType; Status Init(ExecContext* ctx, const FunctionOptions*) override { pool_ = ctx->memory_pool(); @@ -915,20 +914,36 @@ struct GroupedSumImpl : public GroupedAggregator { } Status Consume(const ExecBatch& batch) override { - auto group = batch[1].array()->GetValues(1); - - auto sums = - reinterpret_cast::CType*>(sums_.mutable_data()); + auto sums = reinterpret_cast(sums_.mutable_data()); auto counts = reinterpret_cast(counts_.mutable_data()); + auto g = batch[1].array()->GetValues(1); VisitArrayDataInline( *batch[0].array(), [&](typename TypeTraits::CType value) { - sums[*group] += value; - counts[*group] += 1; - ++group; + sums[*g] += value; + counts[*g] += 1; + ++g; }, - [&] { ++group; }); + [&] { ++g; }); + return Status::OK(); + } + + Status Merge(GroupedAggregator&& raw_other, + const ArrayData& group_id_mapping) override { + auto other = checked_cast(&raw_other); + + auto counts = reinterpret_cast(counts_.mutable_data()); + auto sums = reinterpret_cast(sums_.mutable_data()); + + auto other_counts = reinterpret_cast(other->counts_.mutable_data()); + auto other_sums = reinterpret_cast(other->sums_.mutable_data()); + + auto g = group_id_mapping.GetValues(1); + for (int64_t other_g = 0; other_g < group_id_mapping.length; ++other_g, ++g) { + counts[*g] += other_counts[other_g]; + sums[*g] += other_sums[other_g]; + } return Status::OK(); } @@ -1036,19 +1051,43 @@ struct GroupedMinMaxImpl : public GroupedAggregator { } Status Consume(const ExecBatch& batch) override { - auto group = batch[1].array()->GetValues(1); - + auto g = batch[1].array()->GetValues(1); auto raw_mins = reinterpret_cast(mins_.mutable_data()); auto raw_maxes = reinterpret_cast(maxes_.mutable_data()); VisitArrayDataInline( *batch[0].array(), [&](CType val) { - raw_maxes[*group] = std::max(raw_maxes[*group], val); - raw_mins[*group] = std::min(raw_mins[*group], val); - BitUtil::SetBit(has_values_.mutable_data(), *group++); + raw_maxes[*g] = std::max(raw_maxes[*g], val); + raw_mins[*g] = std::min(raw_mins[*g], val); + BitUtil::SetBit(has_values_.mutable_data(), *g++); }, - [&] { BitUtil::SetBit(has_nulls_.mutable_data(), *group++); }); + [&] { BitUtil::SetBit(has_nulls_.mutable_data(), *g++); }); + return Status::OK(); + } + + Status Merge(GroupedAggregator&& raw_other, + const ArrayData& group_id_mapping) override { + auto other = checked_cast(&raw_other); + + auto raw_mins = reinterpret_cast(mins_.mutable_data()); + auto raw_maxes = reinterpret_cast(maxes_.mutable_data()); + + auto other_raw_mins = reinterpret_cast(other->mins_.mutable_data()); + auto other_raw_maxes = reinterpret_cast(other->maxes_.mutable_data()); + + auto g = group_id_mapping.GetValues(1); + for (int64_t other_g = 0; other_g < group_id_mapping.length; ++other_g, ++g) { + raw_mins[*g] = std::min(raw_mins[*g], other_raw_mins[other_g]); + raw_maxes[*g] = std::max(raw_maxes[*g], other_raw_maxes[other_g]); + + if (BitUtil::GetBit(other->has_values_.data(), other_g)) { + BitUtil::SetBit(has_values_.mutable_data(), *g); + } + if (BitUtil::GetBit(other->has_nulls_.data(), other_g)) { + BitUtil::SetBit(has_nulls_.mutable_data(), *g); + } + } return Status::OK(); } @@ -1084,8 +1123,8 @@ struct GroupedMinMaxImpl : public GroupedAggregator { }; struct GroupedMinMaxFactory { - template ::Type> - Status Visit(const T&) { + template + enable_if_number Visit(const T&) { kernel = MakeKernel(std::move(argument_type), HashAggregateInit>); return Status::OK(); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index b46b7122f55..1d02b0c780a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -582,7 +582,6 @@ TEST(GroupBy, CountOnly) { TEST(GroupBy, SumOnly) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); - if (use_threads) continue; auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ @@ -627,42 +626,51 @@ TEST(GroupBy, SumOnly) { } TEST(GroupBy, MinMaxOnly) { - auto batch = RecordBatchFromJSON( - schema({field("argument", float64()), field("key", int64())}), R"([ + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + + auto table = + TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], - [null, 1], + [null, 1] + ])", + R"([ [0.0, 2], [null, 3], [4.0, null], [3.25, 1], - [0.125, 2], + [0.125, 2] + ])", + R"([ [-0.25, 2], [0.75, null], [null, 3] - ])"); + ])"}); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({batch->GetColumnByName("argument")}, - {batch->GetColumnByName("key")}, - { - {"hash_min_max", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_min_max", nullptr}, + })); + SortBy({"key_0"}, &aggregated_and_grouped); - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("hash_min_max", struct_({ - field("min", float64()), - field("max", float64()), - })), - field("key_0", int64()), - }), - R"([ + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("hash_min_max", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("key_0", int64()), + }), + R"([ [{"min": 1.0, "max": 3.25}, 1], [{"min": -0.25, "max": 0.125}, 2], [{"min": null, "max": null}, 3], [{"min": 0.75, "max": 4.0}, null] ])"), - aggregated_and_grouped, - /*verbose=*/true); + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, CountAndSum) { From e1efb7017cd64efc7ffe186844e1776f277fa388 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 8 Jul 2021 16:51:53 -0400 Subject: [PATCH 08/41] add overload of Resize with default shrink_to_fit --- cpp/src/arrow/buffer.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h index 6c47a464b1d..cfd525ab2d6 100644 --- a/cpp/src/arrow/buffer.h +++ b/cpp/src/arrow/buffer.h @@ -416,7 +416,10 @@ class ARROW_EXPORT ResizableBuffer : public MutableBuffer { /// /// @param new_size The new size for the buffer. /// @param shrink_to_fit Whether to shrink the capacity if new size < current size - virtual Status Resize(const int64_t new_size, bool shrink_to_fit = true) = 0; + virtual Status Resize(const int64_t new_size, bool shrink_to_fit) = 0; + Status Resize(const int64_t new_size) { + return Resize(new_size, /*shrink_to_fit=*/true); + } /// Ensure that buffer has enough memory allocated to fit the indicated /// capacity (and meets the 64 byte padding requirement in Layout.md). From f102acd8045c3d40c3dba61209d1fd4bbc10edd2 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 9 Jul 2021 09:45:33 -0400 Subject: [PATCH 09/41] repair GroupByNode tests --- cpp/src/arrow/compute/exec/exec_plan.cc | 24 ++--- cpp/src/arrow/compute/exec/plan_test.cc | 113 +++++++++++++++++------- 2 files changed, 92 insertions(+), 45 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 19804309cf6..759ee691577 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -631,13 +631,15 @@ struct GroupByNode : ExecNode { ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper_->Consume(key_batch)); // Execute aggregate kernels - auto num_groups = grouper_->num_groups(); for (size_t i = 0; i < agg_kernels_.size(); ++i) { KernelContext kernel_ctx{ctx_}; kernel_ctx.SetState(agg_states_[i].get()); + ARROW_ASSIGN_OR_RAISE( - auto agg_batch, ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch, - Datum(num_groups)})); + auto agg_batch, + ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); + + RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, grouper_->num_groups())); RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); } @@ -788,12 +790,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, // Find input field indices for key fields std::vector key_field_ids(keys.size()); for (size_t i = 0; i < keys.size(); ++i) { - int key_field_id = input_schema->GetFieldIndex(keys[i]); - if (key_field_id < 0) { - return Status::Invalid("Key field named '", keys[i], - "' not found or not unique in the input schema."); - } - key_field_ids[i] = key_field_id; + ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(*input_schema)); + key_field_ids[i] = match[0]; } // Build vector of key field data types @@ -809,12 +807,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, // Find input field indices for aggregates std::vector agg_src_field_ids(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { - int agg_src_field_id = input_schema->GetFieldIndex(agg_srcs[i]); - if (agg_src_field_id < 0) { - return Status::Invalid("Aggregate source field named '", agg_srcs[i], - "' not found or not unique in the input schema."); - } - agg_src_field_ids[i] = agg_src_field_id; + ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(agg_srcs[i]).FindOne(*input_schema)); + agg_src_field_ids[i] = match[0]; } // Build vector of aggregate source field data types diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index e06406bea2e..2bb212698b0 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -382,10 +382,8 @@ TEST(ExecPlanExecution, SourceFilterSink) { MakeTestSourceNode(plan.get(), "source", basic_data, /*parallel=*/false, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN(auto predicate, - equal(field_ref("i32"), literal(6)).Bind(*basic_data.schema)); - - ASSERT_OK_AND_ASSIGN(auto filter, MakeFilterNode(source, "filter", predicate)); + ASSERT_OK_AND_ASSIGN( + auto filter, MakeFilterNode(source, "filter", equal(field_ref("i32"), literal(6)))); auto sink_gen = MakeSinkNode(filter, "sink"); @@ -404,15 +402,12 @@ TEST(ExecPlanExecution, SourceProjectSink) { MakeTestSourceNode(plan.get(), "source", basic_data, /*parallel=*/false, /*slow=*/false)); - std::vector exprs{ - not_(field_ref("bool")), - call("add", {field_ref("i32"), literal(1)}), - }; - for (auto& expr : exprs) { - ASSERT_OK_AND_ASSIGN(expr, expr.Bind(*basic_data.schema)); - } - - ASSERT_OK_AND_ASSIGN(auto projection, MakeProjectNode(source, "project", exprs)); + ASSERT_OK_AND_ASSIGN(auto projection, + MakeProjectNode(source, "project", + { + not_(field_ref("bool")), + call("add", {field_ref("i32"), literal(1)}), + })); auto sink_gen = MakeSinkNode(projection, "sink"); @@ -423,30 +418,88 @@ TEST(ExecPlanExecution, SourceProjectSink) { "[[null, 6], [true, 7], [true, 8]]")})))); } -TEST(ExecPlanExecution, GroupByOnly) { - BatchesWithSchema input; - input.batches = { - ExecBatchFromJSON({int32(), int32()}, "[[1, 12], [2, 7], [1, 3]]"), - ExecBatchFromJSON({int32(), int32()}, "[[5, 5], [2, 3], [1, 8]]")}; - input.schema = schema({field("a", int32()), field("b", int32())}); +namespace { + +BatchesWithSchema MakeGroupableBatches() { + BatchesWithSchema out; + + out.batches = {ExecBatchFromJSON({int32(), utf8()}, R"([ + [12, "alfa"], + [7, "beta"], + [3, "alfa"] + ])"), + ExecBatchFromJSON({int32(), utf8()}, R"([ + [-2, "alfa"], + [-1, "gama"], + [3, "alfa"] + ])"), + ExecBatchFromJSON({int32(), utf8()}, R"([ + [5, "gama"], + [3, "beta"], + [-8, "alfa"] + ])")}; + + out.schema = schema({field("i32", int32()), field("str", utf8())}); + + return out; +} +} // namespace + +TEST(ExecPlanExecution, SourceGroupedSum) { + auto input = MakeGroupableBatches(); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ASSERT_OK_AND_ASSIGN(auto scan, MakeTestSourceNode(plan.get(), "source", input, /*parallel=*/false, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(scan, - "gby", - {"a"}, - {"b"}, - {{"hash_sum", nullptr}})); + ASSERT_OK_AND_ASSIGN(auto source, + MakeTestSourceNode(plan.get(), "source", input, + /*parallel=*/false, /*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), - ResultWith(UnorderedElementsAreArray( - {ExecBatchFromJSON({int64(), int32()}, "[[23, 1], [10, 2], [5, 5]]")} - ))); + ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( + {int64(), utf8()}, R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")}))); } -TEST(ExecPlanExecution, FilterProjectGroupByFilter) { - // TODO: implement +TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { + auto input = MakeGroupableBatches(); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + ASSERT_OK_AND_ASSIGN(auto source, + MakeTestSourceNode(plan.get(), "source", input, + + /*parallel=*/false, /*slow=*/false)); + + ASSERT_OK_AND_ASSIGN( + auto filter, + MakeFilterNode(source, "filter", greater_equal(field_ref("i32"), literal(0)))); + + ASSERT_OK_AND_ASSIGN( + auto projection, + MakeProjectNode(filter, "project", + { + field_ref("str"), + call("multiply", {field_ref("i32"), literal(2)}), + })); + + ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(projection, "gby", /*keys=*/{"str"}, + /*targets=*/{"multiply(i32, 2)"}, + {{"hash_sum", nullptr}})); + + ASSERT_OK_AND_ASSIGN( + auto having, + MakeFilterNode(gby, "having", greater(field_ref("hash_sum"), literal(10)))); + + auto sink_gen = MakeSinkNode(having, "sink"); + + ASSERT_THAT( + StartAndCollect(plan.get(), sink_gen), + ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON({int64(), utf8()}, R"([ + [36, "alfa"], + [20, "beta"] + ])")}))); } } // namespace compute From 4db55561f34b917025016b5a30cbfa740af500e1 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 9 Jul 2021 11:01:57 -0400 Subject: [PATCH 10/41] export MakeGroupByNode --- cpp/src/arrow/compute/exec/exec_plan.h | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 1dc864cbd07..db591cc3855 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -272,6 +272,7 @@ Result MakeProjectNode(ExecNode* input, std::string label, /// \brief Make a node which groups input rows based on key fields and computes /// aggregates for each group +ARROW_EXPORT Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, std::vector agg_srcs, From 8b31684dd13f5616d750dcf6972d5595b80b5ab2 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 9 Jul 2021 17:09:21 -0400 Subject: [PATCH 11/41] implement ExecNode::finished() --- cpp/src/arrow/compute/exec/exec_plan.cc | 18 +++++++++++++----- cpp/src/arrow/compute/exec/plan_test.cc | 14 +++++++------- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 759ee691577..b8e9673b8d1 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -670,7 +670,7 @@ struct GroupByNode : ExecNode { for (int i = 0; i < num_result_batches; ++i) { // Check finished flag - if (finished_) { + if (finished_.is_finished()) { break; } @@ -727,14 +727,20 @@ struct GroupByNode : ExecNode { num_input_batches_total_ = seq; if (num_input_batches_processed_ == num_input_batches_total_) { Status status = OutputResult(); + lock.unlock(); + if (!status.ok()) { ErrorReceived(input, status); - return; + } else { + finished_.MarkFinished(); } } } - Status StartProducing() override { return Status::OK(); } + Status StartProducing() override { + finished_ = Future<>::Make(); + return Status::OK(); + } void PauseProducing(ExecNode* output) override {} @@ -744,15 +750,17 @@ struct GroupByNode : ExecNode { DCHECK_EQ(output, outputs_[0]); inputs_[0]->StopProducing(this); - finished_ = true; + finished_.MarkFinished(); } void StopProducing() override { StopProducing(outputs_[0]); } + Future<> finished() override { return finished_; } + private: ExecContext* ctx_; std::mutex mutex_; - bool finished_{false}; + Future<> finished_ = Future<>::MakeFinished(); int num_input_batches_processed_{0}; int num_input_batches_total_{-1}; int output_batch_size_{32 * 1024}; diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 2bb212698b0..7dd45296afc 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -245,7 +245,7 @@ Future> StartAndCollect( return AllComplete({plan->finished(), Future<>(collected_fut)}) .Then([collected_fut]() -> Result> { ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); - return internal::MapVector( + return ::arrow::internal::MapVector( [](util::optional batch) { return std::move(*batch); }, std::move(collected)); }); @@ -458,8 +458,8 @@ TEST(ExecPlanExecution, SourceGroupedSum) { auto sink_gen = MakeSinkNode(gby, "sink"); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( - {int64(), utf8()}, R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")}))); + Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( + {int64(), utf8()}, R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); } TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { @@ -494,12 +494,12 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { auto sink_gen = MakeSinkNode(having, "sink"); - ASSERT_THAT( - StartAndCollect(plan.get(), sink_gen), - ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON({int64(), utf8()}, R"([ + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith( + UnorderedElementsAreArray({ExecBatchFromJSON({int64(), utf8()}, R"([ [36, "alfa"], [20, "beta"] - ])")}))); + ])")})))); } } // namespace compute From 6110546a2dcdbb11b97e097046596eff6c69b8f1 Mon Sep 17 00:00:00 2001 From: michalursa Date: Fri, 9 Jul 2021 14:47:38 -0700 Subject: [PATCH 12/41] GrouperFastImpl: fix wrong merge --- cpp/src/arrow/compute/exec/exec_plan.cc | 2 -- cpp/src/arrow/compute/exec/plan_test.cc | 26 ------------------- .../arrow/compute/kernels/hash_aggregate.cc | 2 -- 3 files changed, 30 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index de951ebc790..a8592ef8a10 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -546,7 +546,6 @@ struct SinkNode : ExecNode { void InputReceived(ExecNode* input, int seq_num, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); if (finished_.is_finished()) return; producer_.Push(std::move(batch)); @@ -563,7 +562,6 @@ struct SinkNode : ExecNode { if (emit_stop_ != -1) { DCHECK_LE(seq_num, emit_stop_); } - lock.unlock(); } void ErrorReceived(ExecNode* input, Status error) override { diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index da1f8e1aa92..7dd45296afc 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -502,31 +502,5 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { ])")})))); } -TEST(ExecPlanExecution, GroupByOnly) { - BatchesWithSchema input; - input.batches = { - ExecBatchFromJSON({int32(), int32()}, "[[1, 12], [2, 7], [1, 3]]"), - ExecBatchFromJSON({int32(), int32()}, "[[5, 5], [2, 3], [1, 8]]")}; - input.schema = schema({field("a", int32()), field("b", int32())}); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ASSERT_OK_AND_ASSIGN(auto scan, MakeTestSourceNode(plan.get(), "source", input, /*parallel=*/false, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(scan, - "gby", - {"a"}, - {"b"}, - {{"hash_sum", nullptr}})); - auto sink_gen = MakeSinkNode(gby, "sink"); - - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - ResultWith(UnorderedElementsAreArray( - {ExecBatchFromJSON({int64(), int32()}, "[[23, 1], [10, 2], [5, 5]]")} - ))); -} - -TEST(ExecPlanExecution, FilterProjectGroupByFilter) { - // TODO: implement -} - } // 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 39d0faa3e69..79213b93b37 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -1151,8 +1151,6 @@ struct GroupedMinMaxFactory { } // namespace -} // namespace - Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs) { From 831bfb30009cb0422f4cd0f4a3cfb681d454f256 Mon Sep 17 00:00:00 2001 From: michalursa Date: Mon, 12 Jul 2021 11:23:13 -0700 Subject: [PATCH 13/41] ExecNode: fixing SinkNode calling Finish --- cpp/src/arrow/compute/exec/exec_plan.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index a8592ef8a10..a859588802a 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -575,7 +575,9 @@ struct SinkNode : ExecNode { std::unique_lock lock(mutex_); emit_stop_ = seq_stop; lock.unlock(); - Finish(); + if (num_received_ == emit_stop_) { + Finish(); + } } private: From 3c4b9ba6869430950ef802e413f84af82bd6012e Mon Sep 17 00:00:00 2001 From: michalursa Date: Tue, 13 Jul 2021 01:35:06 -0700 Subject: [PATCH 14/41] GroupByNode: adding support for concurrent aggregation using per thread copy of data structures and merging them at the end --- cpp/src/arrow/compute/exec/exec_plan.cc | 331 +++++++++++++++++++----- 1 file changed, 260 insertions(+), 71 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index a859588802a..83300a3d528 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -29,10 +29,12 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" #include "arrow/util/optional.h" +#include "arrow/util/bit_util.h" namespace arrow { using internal::checked_cast; +using BitUtil::CountLeadingZeros; namespace compute { @@ -603,25 +605,160 @@ AsyncGenerator> MakeSinkNode(ExecNode* input, return out; } +template +class SharedSequenceOfObjects { +public: + SharedSequenceOfObjects(int log_max_objects = 16) { + objects_.resize(log_max_objects); + num_created_vectors_ = 0; + } + T* get(int id) { + int vid = vector_id(id); + ARROW_DCHECK(static_cast(vid) < objects_.size()); + if (vid >= num_created_vectors_) { + std::lock_guard lock(mutex_); + while (vid >= num_created_vectors_) { + objects_[num_created_vectors_].resize(1 << num_created_vectors_); + ++num_created_vectors_; + } + } + return &objects_[vid][id]; + } +private: + static int vector_id(int id) { + return 32 - CountLeadingZeros(static_cast(id)); + } + + int num_created_vectors_; + std::vector> objects_; + std::mutex mutex_; +}; + +class SmallUniqueIdAssignment { +public: + SmallUniqueIdAssignment() { + num_ids_ = 0; + } + int Get() { + std::lock_guard lock(mutex_); + if (stack_.empty()) { + return num_ids_++; + } else { + int last = stack_.back(); + stack_.pop_back(); + return last; + } + } + void Return(int id) { + std::lock_guard lock(mutex_); + stack_.push_back(id); + } + int num_ids() const { return num_ids_; } +private: + std::vector stack_; + int num_ids_; + std::mutex mutex_; +}; + +class SmallUniqueIdHolder { +public: + SmallUniqueIdHolder() = delete; + SmallUniqueIdHolder(const SmallUniqueIdHolder&) = delete; + SmallUniqueIdHolder(const SmallUniqueIdHolder&&) = delete; + SmallUniqueIdHolder(SmallUniqueIdAssignment* id_mgr) { + id_mgr_ = id_mgr; + id_ = id_mgr->Get(); + } + ~SmallUniqueIdHolder() { + id_mgr_->Return(id_); + } + int get() const { return id_; } +private: + SmallUniqueIdAssignment* id_mgr_; + int id_; +}; + +namespace internal { +Result> GetKernels( + ExecContext* ctx, const std::vector& aggregates, + const std::vector& in_descrs); +Result>> InitKernels( + const std::vector& kernels, ExecContext* ctx, + const std::vector& aggregates, + const std::vector& in_descrs); +Result ResolveKernels( + const std::vector& aggregates, + const std::vector& kernels, + const std::vector>& states, ExecContext* ctx, + const std::vector& descrs); +} // namespace internal + struct GroupByNode : ExecNode { GroupByNode(ExecNode* input, std::string label, std::shared_ptr output_schema, ExecContext* ctx, const std::vector&& key_field_ids, - std::unique_ptr&& grouper, const std::vector&& agg_src_field_ids, - const std::vector&& agg_kernels, - std::vector>&& agg_states) + const std::vector&& aggs, + const std::vector&& agg_kernels) : ExecNode(input->plan(), std::move(label), {input}, {"groupby"}, std::move(output_schema), /*num_outputs=*/1), ctx_(ctx), key_field_ids_(std::move(key_field_ids)), - grouper_(std::move(grouper)), agg_src_field_ids_(std::move(agg_src_field_ids)), - agg_kernels_(std::move(agg_kernels)), - agg_states_(std::move(agg_states)) {} + aggs_(std::move(aggs)), + agg_kernels_(std::move(agg_kernels)) { + 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 "GroupByNode"; } +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) { @@ -630,64 +767,130 @@ struct GroupByNode : ExecNode { ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); // Create a batch with group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper_->Consume(key_batch)); + 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(agg_states_[i].get()); + 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, grouper_->num_groups())); + 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 OutputResult() { - // Finalize output - ArrayDataVector out_data(agg_kernels_.size() + key_field_ids_.size()); - auto it = out_data.begin(); + 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(agg_states_[i].get()); + 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, grouper_->GetUniques()); + 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(); + } - uint32_t num_groups = grouper_->num_groups(); - int num_result_batches = (num_groups + output_batch_size_ - 1) / output_batch_size_; + // Slice arrays + int64_t batch_start = n * output_batch_size_; + int64_t batch_length = + std::min(output_batch_size_, static_cast(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 (static_cast(num_output_batches_processed) * output_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 num_result_batches = (num_out_groups_ + output_batch_size_ - 1) / output_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; } - // Slice arrays - int64_t batch_start = i * output_batch_size_; - int64_t batch_length = - std::min(output_batch_size_, static_cast(num_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, i, output_batch); + RETURN_NOT_OK(executor->Spawn([this, i]() { + Status status = OutputNthBatch(i); + if (!status.ok()) { + ErrorReceived(inputs_[0], status); + } + })); } return Status::OK(); @@ -696,7 +899,11 @@ struct GroupByNode : ExecNode { void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); + if (finished_.is_finished()) { + return; + } + + ARROW_DCHECK(num_input_batches_processed_.load() != num_input_batches_total_.load()); Status status = ProcessInputBatch(batch); if (!status.ok()) { @@ -704,8 +911,8 @@ struct GroupByNode : ExecNode { return; } - ++num_input_batches_processed_; - if (num_input_batches_processed_ == num_input_batches_total_) { + 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); @@ -724,17 +931,12 @@ struct GroupByNode : ExecNode { void InputFinished(ExecNode* input, int seq) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); - - num_input_batches_total_ = seq; - if (num_input_batches_processed_ == num_input_batches_total_) { + num_input_batches_total_.store(seq); + if (num_input_batches_processed_.load() == num_input_batches_total_.load()) { Status status = OutputResult(); - lock.unlock(); if (!status.ok()) { ErrorReceived(input, status); - } else { - finished_.MarkFinished(); } } } @@ -761,33 +963,28 @@ struct GroupByNode : ExecNode { private: ExecContext* ctx_; - std::mutex mutex_; Future<> finished_ = Future<>::MakeFinished(); - int num_input_batches_processed_{0}; - int num_input_batches_total_{-1}; int output_batch_size_{32 * 1024}; + std::atomic num_input_batches_processed_; + std::atomic num_input_batches_total_; + std::atomic num_output_batches_processed_; + const std::vector key_field_ids_; - std::unique_ptr grouper_; const std::vector agg_src_field_ids_; + const std::vector aggs_; const std::vector agg_kernels_; - std::vector> agg_states_; -}; -namespace internal { -Result> GetKernels( - ExecContext* ctx, const std::vector& aggregates, - const std::vector& in_descrs); -Result>> InitKernels( - const std::vector& kernels, ExecContext* ctx, - const std::vector& aggregates, - const std::vector& in_descrs); -Result ResolveKernels( - const std::vector& aggregates, - const std::vector& kernels, - const std::vector>& states, ExecContext* ctx, - const std::vector& descrs); -} // namespace internal + 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_; +}; Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, @@ -804,16 +1001,6 @@ Result MakeGroupByNode(ExecNode* input, std::string label, key_field_ids[i] = match[0]; } - // Build vector of key field data types - std::vector key_descrs(keys.size()); - for (size_t i = 0; i < keys.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(auto grouper, internal::Grouper::Make(key_descrs, ctx)); - // Find input field indices for aggregates std::vector agg_src_field_ids(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { @@ -854,10 +1041,12 @@ Result MakeGroupByNode(ExecNode* input, std::string label, output_fields[base + i] = input_schema->field(key_field_id); } + auto aggs_copy = aggs; + return input->plan()->EmplaceNode( input, std::move(label), schema(std::move(output_fields)), ctx, - std::move(key_field_ids), std::move(grouper), std::move(agg_src_field_ids), - std::move(agg_kernels), std::move(agg_states)); + std::move(key_field_ids), std::move(agg_src_field_ids), + std::move(aggs), std::move(agg_kernels)); } } // namespace compute From b36348e16f691da19b7e39ed520fd9d725f7643b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 09:56:27 -0400 Subject: [PATCH 15/41] lint fixes --- cpp/src/arrow/compute/exec/exec_plan.cc | 166 ++++++++++++------------ 1 file changed, 85 insertions(+), 81 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 04003171ad9..71f066bc919 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -31,16 +31,16 @@ #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/util/async_generator.h" +#include "arrow/util/bit_util.h" #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" #include "arrow/util/optional.h" -#include "arrow/util/bit_util.h" namespace arrow { +using BitUtil::CountLeadingZeros; using internal::checked_cast; using internal::checked_pointer_cast; -using BitUtil::CountLeadingZeros; namespace compute { @@ -567,7 +567,7 @@ struct SinkNode : ExecNode { producer_.Push(std::move(batch)); std::unique_lock lock(mutex_); - + ++num_received_; if (num_received_ == emit_stop_) { lock.unlock(); @@ -819,77 +819,76 @@ Result MakeScalarAggregateNode(ExecNode* input, std::string label, std::move(states)); } -template +template class SharedSequenceOfObjects { -public: - SharedSequenceOfObjects(int log_max_objects = 16) { - objects_.resize(log_max_objects); - num_created_vectors_ = 0; - } - T* get(int id) { - int vid = vector_id(id); - ARROW_DCHECK(static_cast(vid) < objects_.size()); - if (vid >= num_created_vectors_) { - std::lock_guard lock(mutex_); - while (vid >= num_created_vectors_) { - objects_[num_created_vectors_].resize(1 << num_created_vectors_); - ++num_created_vectors_; - } - } - return &objects_[vid][id]; - } -private: - static int vector_id(int id) { - return 32 - CountLeadingZeros(static_cast(id)); + public: + explicit SharedSequenceOfObjects(int log_max_objects = 16) { + objects_.resize(log_max_objects); + num_created_vectors_ = 0; + } + T* get(int id) { + int vid = vector_id(id); + ARROW_DCHECK(static_cast(vid) < objects_.size()); + if (vid >= num_created_vectors_) { + std::lock_guard lock(mutex_); + while (vid >= num_created_vectors_) { + objects_[num_created_vectors_].resize(1 << num_created_vectors_); + ++num_created_vectors_; + } } + return &objects_[vid][id]; + } + + private: + static int vector_id(int id) { + return 32 - CountLeadingZeros(static_cast(id)); + } - int num_created_vectors_; - std::vector> objects_; - std::mutex mutex_; + int num_created_vectors_; + std::vector> objects_; + std::mutex mutex_; }; class SmallUniqueIdAssignment { -public: - SmallUniqueIdAssignment() { - num_ids_ = 0; - } - int Get() { - std::lock_guard lock(mutex_); - if (stack_.empty()) { - return num_ids_++; - } else { - int last = stack_.back(); - stack_.pop_back(); - return last; - } + public: + SmallUniqueIdAssignment() { num_ids_ = 0; } + int Get() { + std::lock_guard lock(mutex_); + if (stack_.empty()) { + return num_ids_++; + } else { + int last = stack_.back(); + stack_.pop_back(); + return last; } - void Return(int id) { - std::lock_guard lock(mutex_); - stack_.push_back(id); - } - int num_ids() const { return num_ids_; } -private: - std::vector stack_; - int num_ids_; - std::mutex mutex_; + } + void Return(int id) { + std::lock_guard lock(mutex_); + stack_.push_back(id); + } + int num_ids() const { return num_ids_; } + + private: + std::vector stack_; + int num_ids_; + std::mutex mutex_; }; class SmallUniqueIdHolder { -public: - SmallUniqueIdHolder() = delete; - SmallUniqueIdHolder(const SmallUniqueIdHolder&) = delete; - SmallUniqueIdHolder(const SmallUniqueIdHolder&&) = delete; - SmallUniqueIdHolder(SmallUniqueIdAssignment* id_mgr) { - id_mgr_ = id_mgr; - id_ = id_mgr->Get(); - } - ~SmallUniqueIdHolder() { - id_mgr_->Return(id_); - } - int get() const { return id_; } -private: - SmallUniqueIdAssignment* id_mgr_; - int id_; + public: + SmallUniqueIdHolder() = delete; + SmallUniqueIdHolder(const SmallUniqueIdHolder&) = delete; + SmallUniqueIdHolder(const SmallUniqueIdHolder&&) = delete; + explicit SmallUniqueIdHolder(SmallUniqueIdAssignment* id_mgr) { + id_mgr_ = id_mgr; + id_ = id_mgr->Get(); + } + ~SmallUniqueIdHolder() { id_mgr_->Return(id_); } + int get() const { return id_; } + + private: + SmallUniqueIdAssignment* id_mgr_; + int id_; }; namespace internal { @@ -920,18 +919,18 @@ struct GroupByNode : ExecNode { agg_src_field_ids_(std::move(agg_src_field_ids)), aggs_(std::move(aggs)), agg_kernels_(std::move(agg_kernels)) { - num_input_batches_processed_.store(0); - num_input_batches_total_.store(-1); - num_output_batches_processed_.store(0); - output_started_.store(false); - } + 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 "GroupByNode"; } -private: + private: struct ThreadLocalState; -public: + public: Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema auto input_schema = inputs_[0]->output_schema(); @@ -956,12 +955,14 @@ struct GroupByNode : ExecNode { 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( + 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)); + internal::ResolveKernels(aggs_, agg_kernels_, state->agg_states, ctx_, + agg_src_descrs)); } } return Status::OK(); @@ -1017,7 +1018,7 @@ struct GroupByNode : ExecNode { 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())); + *transposition.array())); state->agg_states[i].reset(); } } @@ -1072,8 +1073,10 @@ struct GroupByNode : ExecNode { 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 (static_cast(num_output_batches_processed) * output_batch_size_ >= num_out_groups_) { + uint32_t num_output_batches_processed = + 1 + num_output_batches_processed_.fetch_add(1); + if (static_cast(num_output_batches_processed) * output_batch_size_ >= + num_out_groups_) { finished_.MarkFinished(); } @@ -1087,9 +1090,10 @@ struct GroupByNode : ExecNode { } RETURN_NOT_OK(Merge()); - RETURN_NOT_OK(Finalize()); + RETURN_NOT_OK(Finalize()); - int num_result_batches = (num_out_groups_ + output_batch_size_ - 1) / output_batch_size_; + int num_result_batches = + (num_out_groups_ + output_batch_size_ - 1) / output_batch_size_; outputs_[0]->InputFinished(this, num_result_batches); auto executor = arrow::internal::GetCpuThreadPool(); @@ -1259,8 +1263,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, return input->plan()->EmplaceNode( input, std::move(label), schema(std::move(output_fields)), ctx, - std::move(key_field_ids), std::move(agg_src_field_ids), - std::move(aggs), std::move(agg_kernels)); + std::move(key_field_ids), std::move(agg_src_field_ids), std::move(aggs), + std::move(agg_kernels)); } } // namespace compute From 87cb50efda25f007c6aeb51a027fe7101c4f9c15 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 10:03:01 -0400 Subject: [PATCH 16/41] msvc: add explicit cast to size_t --- cpp/src/arrow/compute/exec/exec_plan.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 71f066bc919..1905951c356 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -832,7 +832,8 @@ class SharedSequenceOfObjects { if (vid >= num_created_vectors_) { std::lock_guard lock(mutex_); while (vid >= num_created_vectors_) { - objects_[num_created_vectors_].resize(1 << num_created_vectors_); + objects_[num_created_vectors_].resize(static_cast(1) + << num_created_vectors_); ++num_created_vectors_; } } From d4916547537272e6cfde524d2e0933cb05df9169 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 11:38:26 -0400 Subject: [PATCH 17/41] relieve some constraints on MakeScanNode.options --- cpp/src/arrow/dataset/scanner.cc | 43 +++++++++++++++----- cpp/src/arrow/dataset/scanner_test.cc | 58 ++++++++++++++++----------- 2 files changed, 69 insertions(+), 32 deletions(-) diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index 2f7a115bb4b..b45eefba165 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -456,6 +456,12 @@ Result> FragmentsToBatches( }); } +const FieldVector kAugmentedFields{ + field("__fragment_index", int32()), + field("__batch_index", int32()), + field("__last_in_fragment", boolean()), +}; + Result MakeScanNode(compute::ExecPlan* plan, FragmentGenerator fragment_gen, std::shared_ptr options) { @@ -496,12 +502,12 @@ Result MakeScanNode(compute::ExecPlan* plan, return batch; }); - auto augmented_fields = options->dataset_schema->fields(); - augmented_fields.push_back(field("__fragment_index", int32())); - augmented_fields.push_back(field("__batch_index", int32())); - augmented_fields.push_back(field("__last_in_fragment", boolean())); - return compute::MakeSourceNode(plan, "dataset_scan", - schema(std::move(augmented_fields)), std::move(gen)); + auto fields = options->dataset_schema->fields(); + for (const auto& aug_field : kAugmentedFields) { + fields.push_back(aug_field); + } + return compute::MakeSourceNode(plan, "dataset_scan", schema(std::move(fields)), + std::move(gen)); } class OneShotScanTask : public ScanTask { @@ -1157,6 +1163,25 @@ Result SyncScanner::CountRows() { Result MakeScanNode(compute::ExecPlan* plan, std::shared_ptr dataset, std::shared_ptr scan_options) { + if (scan_options->dataset_schema == nullptr) { + scan_options->dataset_schema = dataset->schema(); + } + + if (!scan_options->filter.IsBound()) { + ARROW_ASSIGN_OR_RAISE(scan_options->filter, + scan_options->filter.Bind(*dataset->schema())); + } + + if (!scan_options->projection.IsBound()) { + auto fields = dataset->schema()->fields(); + for (const auto& aug_field : kAugmentedFields) { + fields.push_back(aug_field); + } + + ARROW_ASSIGN_OR_RAISE(scan_options->projection, + scan_options->projection.Bind(Schema(std::move(fields)))); + } + // using a generator for speculative forward compatibility with async fragment discovery ARROW_ASSIGN_OR_RAISE(auto fragments_it, dataset->GetFragments(scan_options->filter)); ARROW_ASSIGN_OR_RAISE(auto fragments_vec, fragments_it.ToVector()); @@ -1175,9 +1200,9 @@ Result MakeAugmentedProjectNode( } } - for (auto aug_name : {"__fragment_index", "__batch_index", "__last_in_fragment"}) { - exprs.push_back(compute::field_ref(aug_name)); - names.emplace_back(aug_name); + for (const auto& aug_field : kAugmentedFields) { + exprs.push_back(compute::field_ref(aug_field->name())); + names.push_back(aug_field->name()); } return compute::MakeProjectNode(input, std::move(label), std::move(exprs), std::move(names)); diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 74f558d1738..a3c252a8d65 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -1175,6 +1175,22 @@ DatasetAndBatches MakeBasicDataset() { return {dataset, batches}; } + +compute::Expression Materialize(std::vector names, + bool include_aug_fields = false) { + if (include_aug_fields) { + for (auto aug_name : {"__fragment_index", "__batch_index", "__last_in_fragment"}) { + names.emplace_back(aug_name); + } + } + + std::vector exprs; + for (const auto& name : names) { + exprs.push_back(field_ref(name)); + } + + return project(exprs, names); +} } // namespace TEST(ScanNode, Schema) { @@ -1184,7 +1200,7 @@ TEST(ScanNode, Schema) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); + options->projection = Materialize({}); // set an empty projection ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); @@ -1192,6 +1208,8 @@ TEST(ScanNode, Schema) { fields.push_back(field("__fragment_index", int32())); fields.push_back(field("__batch_index", int32())); fields.push_back(field("__last_in_fragment", boolean())); + // output_schema is *always* the full augmented dataset schema, regardless of projection + // (but some columns *may* be placeholder null Scalars if not projected) AssertSchemaEqual(Schema(fields), *scan->output_schema()); } @@ -1202,7 +1220,8 @@ TEST(ScanNode, Trivial) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); + // ensure all fields are materialized + options->projection = Materialize({"a", "b", "c"}, /*include_aug_fields=*/true); ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); auto sink_gen = MakeSinkNode(scan, "sink"); @@ -1220,9 +1239,9 @@ TEST(ScanNode, FilteredOnVirtualColumn) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); - ASSERT_OK_AND_ASSIGN(options->filter, - less(field_ref("c"), literal(30)).Bind(*basic.dataset->schema())); + options->filter = less(field_ref("c"), literal(30)); + // ensure all fields are materialized + options->projection = Materialize({"a", "b", "c"}, /*include_aug_fields=*/true); ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); @@ -1245,10 +1264,9 @@ TEST(ScanNode, DeferredFilterOnPhysicalColumn) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); - ASSERT_OK_AND_ASSIGN( - options->filter, - greater(field_ref("a"), literal(4)).Bind(*basic.dataset->schema())); + options->filter = greater(field_ref("a"), literal(4)); + // ensure all fields are materialized + options->projection = Materialize({"a", "b", "c"}, /*include_aug_fields=*/true); ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); @@ -1270,8 +1288,7 @@ TEST(ScanNode, DISABLED_ProjectionPushdown) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); - ASSERT_OK(SetProjection(options.get(), {field_ref("b")}, {"b"})); + options->projection = Materialize({"b"}, /*include_aug_fields=*/true); ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); @@ -1298,16 +1315,14 @@ TEST(ScanNode, MaterializationOfVirtualColumn) { auto options = std::make_shared(); options->use_async = true; - options->dataset_schema = basic.dataset->schema(); + options->projection = Materialize({"a", "b", "c"}, /*include_aug_fields=*/true); ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); ASSERT_OK_AND_ASSIGN( auto project, - compute::MakeProjectNode( - scan, "project", - {field_ref("a"), field_ref("b"), field_ref("c"), field_ref("__fragment_index"), - field_ref("__batch_index"), field_ref("__last_in_fragment")})); + dataset::MakeAugmentedProjectNode( + scan, "project", {field_ref("a"), field_ref("b"), field_ref("c")})); auto sink_gen = MakeSinkNode(project, "sink"); @@ -1352,16 +1367,12 @@ TEST(ScanNode, MinimalEndToEnd) { auto options = std::make_shared(); // sync scanning is not supported by ScanNode options->use_async = true; - // for now, we must replicate the dataset schema here - options->dataset_schema = dataset->schema(); // specify the filter compute::Expression b_is_true = field_ref("b"); - ASSERT_OK_AND_ASSIGN(b_is_true, b_is_true.Bind(*dataset->schema())); options->filter = b_is_true; // for now, specify the projection as the full project expression (eventually this can // just be a list of materialized field names) compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); - ASSERT_OK_AND_ASSIGN(a_times_2, a_times_2.Bind(*dataset->schema())); options->projection = call("project", {a_times_2}, compute::ProjectOptions{{"a * 2"}}); // construct the scan node @@ -1387,20 +1398,21 @@ TEST(ScanNode, MinimalEndToEnd) { std::shared_ptr sink_reader = compute::MakeGeneratorReader( schema({field("a * 2", int32())}), std::move(sink_gen), exec_context.memory_pool()); - // start the ExecPlan then wait 1s for completion + // start the ExecPlan ASSERT_OK(plan->StartProducing()); - ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; // collect sink_reader into a Table ASSERT_OK_AND_ASSIGN(auto collected, Table::FromRecordBatchReader(sink_reader.get())); + // wait 1s for completion + ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; + auto expected = TableFromJSON(schema({field("a * 2", int32())}), { R"([ {"a * 2": 4}, {"a * 2": null}, {"a * 2": null} ])"}); - AssertTablesEqual(*expected, *collected, /*same_chunk_layout=*/false); } From f2da288dc23167238da874d5b8e7305ceb9278d7 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 12:29:54 -0400 Subject: [PATCH 18/41] use ScalarAggregateNode in CountRows --- .../kernels/aggregate_basic_internal.h | 15 ++++++-- cpp/src/arrow/dataset/scanner.cc | 38 +++++-------------- 2 files changed, 21 insertions(+), 32 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h b/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h index e6755c05f5d..0b70f194d0e 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h +++ b/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include "arrow/compute/api_aggregate.h" #include "arrow/compute/kernels/aggregate_internal.h" @@ -59,8 +60,15 @@ struct SumImpl : public ScalarAggregator { using OutputType = typename TypeTraits::ScalarType; Status Consume(KernelContext*, const ExecBatch& batch) override { + if (batch[0].is_scalar() && batch[0].scalar()->is_valid) { + using ScalarType = typename TypeTraits::ScalarType; + this->count += batch.length; + this->sum += static_cast( + batch[0].scalar_as().value * batch.length); + return Status::OK(); + } const auto& data = batch[0].array(); - this->count = data->length - data->GetNullCount(); + this->count += data->length - data->GetNullCount(); if (is_boolean_type::value) { this->sum += static_cast(BooleanArray(data).true_count()); } else { @@ -223,9 +231,8 @@ struct MinMaxImpl : public ScalarAggregator { using ThisType = MinMaxImpl; using StateType = MinMaxState; - MinMaxImpl(const std::shared_ptr& out_type, - const ScalarAggregateOptions& options) - : out_type(out_type), options(options) {} + MinMaxImpl(std::shared_ptr out_type, ScalarAggregateOptions options) + : out_type(std::move(out_type)), options(std::move(options)) {} Status Consume(KernelContext*, const ExecBatch& batch) override { StateType local; diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index b45eefba165..db27cb03be4 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -782,21 +782,6 @@ Future> AsyncScanner::ToTableAsync( }); } -namespace { -Result GetSelectionSize(const Datum& selection, int64_t length) { - if (length == 0) return 0; - - if (selection.is_scalar()) { - if (!selection.scalar()->is_valid) return 0; - if (!selection.scalar_as().value) return 0; - return length; - } - - ARROW_ASSIGN_OR_RAISE(auto count, compute::Sum(selection)); - return static_cast(count.scalar_as().value); -} -} // namespace - Result AsyncScanner::CountRows() { ARROW_ASSIGN_OR_RAISE(auto fragment_gen, GetFragments()); ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make()); @@ -808,7 +793,7 @@ Result AsyncScanner::CountRows() { fragment_gen = MakeMappedGenerator( std::move(fragment_gen), [&](const std::shared_ptr& fragment) { - return fragment->CountRows(scan_options_->filter, scan_options_) + return fragment->CountRows(options->filter, options) .Then([&, fragment](util::optional fast_count) mutable -> std::shared_ptr { if (fast_count) { @@ -831,22 +816,19 @@ Result AsyncScanner::CountRows() { auto get_selection, compute::MakeProjectNode(scan, "get_selection", {options->filter})); + ARROW_ASSIGN_OR_RAISE( + auto sum_selection, + compute::MakeScalarAggregateNode(get_selection, "sum_selection", + {compute::internal::Aggregate{"sum", nullptr}})); + AsyncGenerator> sink_gen = - compute::MakeSinkNode(get_selection, "sink"); + compute::MakeSinkNode(sum_selection, "sink"); RETURN_NOT_OK(plan->StartProducing()); - RETURN_NOT_OK( - VisitAsyncGenerator(std::move(sink_gen), - [&](const util::optional& batch) { - // TODO replace with scalar aggregation node - ARROW_ASSIGN_OR_RAISE( - int64_t slow_count, - GetSelectionSize(batch->values[0], batch->length)); - total += slow_count; - return Status::OK(); - }) - .status()); + auto maybe_slow_count = sink_gen().result(); + ARROW_ASSIGN_OR_RAISE(auto slow_count, maybe_slow_count); + total += slow_count->values[0].scalar_as().value; plan->finished().Wait(); From 336cb6659601b6ec7bf248dd39c0b3e01e6ec93e Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 13:21:35 -0400 Subject: [PATCH 19/41] increment num_received_ inside the sync block with MaybeFinish() --- cpp/src/arrow/compute/exec/exec_plan.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 1905951c356..4bd9b08d934 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -676,7 +676,6 @@ struct ScalarAggregateNode : ExecNode { std::unique_lock lock(mutex_); auto it = thread_indices_.emplace(std::this_thread::get_id(), thread_indices_.size()).first; - ++num_received_; auto thread_index = it->second; lock.unlock(); @@ -688,6 +687,7 @@ struct ScalarAggregateNode : ExecNode { } lock.lock(); + ++num_received_; st = MaybeFinish(&lock); if (!st.ok()) { outputs_[0]->ErrorReceived(this, std::move(st)); From 9dd1751d675ae2088159ce9172b98802f3ae0781 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 14 Jul 2021 14:15:25 -0400 Subject: [PATCH 20/41] add more EndToEnd tests --- cpp/src/arrow/compute/exec/exec_plan.cc | 5 +- cpp/src/arrow/compute/exec/exec_plan.h | 3 +- cpp/src/arrow/dataset/scanner_test.cc | 159 ++++++++++++++++++++++++ 3 files changed, 163 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 4bd9b08d934..df71777c989 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1208,8 +1208,7 @@ struct GroupByNode : ExecNode { Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, std::vector agg_srcs, - std::vector aggs, - ExecContext* ctx) { + std::vector aggs) { // Get input schema auto input_schema = input->output_schema(); @@ -1236,6 +1235,8 @@ Result MakeGroupByNode(ExecNode* input, std::string label, ValueDescr(input_schema->field(agg_src_field_id)->type(), ValueDescr::ARRAY); } + auto ctx = input->plan()->exec_context(); + // Construct aggregates ARROW_ASSIGN_OR_RAISE(auto agg_kernels, internal::GetKernels(ctx, aggs, agg_src_descrs)); diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 934fa2f6f41..449555c474f 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -289,8 +289,7 @@ ARROW_EXPORT Result MakeGroupByNode(ExecNode* input, std::string label, std::vector keys, std::vector agg_srcs, - std::vector aggs, - ExecContext* ctx = default_exec_context()); + std::vector aggs); } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index a3c252a8d65..061994eb210 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -1416,5 +1416,164 @@ TEST(ScanNode, MinimalEndToEnd) { AssertTablesEqual(*expected, *collected, /*same_chunk_layout=*/false); } +TEST(ScanNode, MinimalScalarAggEndToEnd) { + // NB: This test is here for didactic purposes + + // Specify a MemoryPool and ThreadPool for the ExecPlan + compute::ExecContext exec_context(default_memory_pool(), internal::GetCpuThreadPool()); + + // A ScanNode is constructed from an ExecPlan (into which it is inserted), + // a Dataset (whose batches will be scanned), and ScanOptions (to specify a filter for + // predicate pushdown, a projection to skip materialization of unnecessary columns, ...) + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, + compute::ExecPlan::Make(&exec_context)); + + std::shared_ptr dataset = std::make_shared( + TableFromJSON(schema({field("a", int32()), field("b", boolean())}), + { + R"([{"a": 1, "b": null}, + {"a": 2, "b": true}])", + R"([{"a": null, "b": true}, + {"a": 3, "b": false}])", + R"([{"a": null, "b": true}, + {"a": 4, "b": false}])", + R"([{"a": 5, "b": null}, + {"a": 6, "b": false}, + {"a": 7, "b": false}])", + })); + + auto options = std::make_shared(); + // sync scanning is not supported by ScanNode + options->use_async = true; + // specify the filter + compute::Expression b_is_true = field_ref("b"); + options->filter = b_is_true; + // for now, specify the projection as the full project expression (eventually this can + // just be a list of materialized field names) + compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); + options->projection = call("project", {a_times_2}, compute::ProjectOptions{{"a * 2"}}); + + // construct the scan node + ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, + dataset::MakeScanNode(plan.get(), dataset, options)); + + // pipe the scan node into a filter node + ASSERT_OK_AND_ASSIGN(compute::ExecNode * filter, + compute::MakeFilterNode(scan, "filter", b_is_true)); + + // pipe the filter node into a project node + ASSERT_OK_AND_ASSIGN(compute::ExecNode * project, + compute::MakeProjectNode(filter, "project", {a_times_2})); + + // pipe the projection into a scalar aggregate node + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * sum, + compute::MakeScalarAggregateNode(project, "scalar_agg", + {compute::internal::Aggregate{"sum", nullptr}})); + + // finally, pipe the project node into a sink node + auto sink_gen = compute::MakeSinkNode(sum, "sink"); + + // translate sink_gen (async) to sink_reader (sync) + std::shared_ptr sink_reader = compute::MakeGeneratorReader( + schema({field("sum", int64())}), std::move(sink_gen), exec_context.memory_pool()); + + // start the ExecPlan + ASSERT_OK(plan->StartProducing()); + + // collect sink_reader into a Table + ASSERT_OK_AND_ASSIGN(auto collected, Table::FromRecordBatchReader(sink_reader.get())); + + // wait 1s for completion + ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; + + auto expected = TableFromJSON(schema({field("sum", int64())}), { + R"([ + {"sum": 4} + ])"}); + AssertTablesEqual(*expected, *collected, /*same_chunk_layout=*/false); +} + +TEST(ScanNode, MinimalGroupedAggEndToEnd) { + // NB: This test is here for didactic purposes + + // Specify a MemoryPool and ThreadPool for the ExecPlan + compute::ExecContext exec_context(default_memory_pool(), internal::GetCpuThreadPool()); + + // A ScanNode is constructed from an ExecPlan (into which it is inserted), + // a Dataset (whose batches will be scanned), and ScanOptions (to specify a filter for + // predicate pushdown, a projection to skip materialization of unnecessary columns, ...) + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, + compute::ExecPlan::Make(&exec_context)); + + std::shared_ptr dataset = std::make_shared( + TableFromJSON(schema({field("a", int32()), field("b", boolean())}), + { + R"([{"a": 1, "b": null}, + {"a": 2, "b": true}])", + R"([{"a": null, "b": true}, + {"a": 3, "b": false}])", + R"([{"a": null, "b": true}, + {"a": 4, "b": false}])", + R"([{"a": 5, "b": null}, + {"a": 6, "b": false}, + {"a": 7, "b": false}])", + })); + + auto options = std::make_shared(); + // sync scanning is not supported by ScanNode + options->use_async = true; + // specify the filter + compute::Expression b_is_true = field_ref("b"); + options->filter = b_is_true; + // for now, specify the projection as the full project expression (eventually this can + // just be a list of materialized field names) + compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); + compute::Expression b = field_ref("b"); + options->projection = + call("project", {a_times_2, b}, compute::ProjectOptions{{"a * 2", "b"}}); + + // construct the scan node + ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, + dataset::MakeScanNode(plan.get(), dataset, options)); + + // pipe the scan node into a project node + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * project, + compute::MakeProjectNode(scan, "project", {a_times_2, b}, {"a * 2", "b"})); + + // pipe the projection into a grouped aggregate node + ASSERT_OK_AND_ASSIGN(compute::ExecNode * sum, + compute::MakeGroupByNode( + project, "grouped_agg", /*keys=*/{"b"}, /*targets=*/{"a * 2"}, + {compute::internal::Aggregate{"hash_sum", nullptr}})); + + // finally, pipe the project node into a sink node + auto sink_gen = compute::MakeSinkNode(sum, "sink"); + + // translate sink_gen (async) to sink_reader (sync) + std::shared_ptr sink_reader = compute::MakeGeneratorReader( + schema({field("hash_sum", int64()), field("b", boolean())}), std::move(sink_gen), + exec_context.memory_pool()); + + // start the ExecPlan + ASSERT_OK(plan->StartProducing()); + + // collect sink_reader into a Table + ASSERT_OK_AND_ASSIGN(auto collected, Table::FromRecordBatchReader(sink_reader.get())); + + // wait 1s for completion + ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; + + auto expected = + TableFromJSON(schema({field("hash_sum", int64()), field("b", boolean())}), { + R"([ + {"hash_sum": 12, "b": null}, + {"hash_sum": 4, "b": true}, + {"hash_sum": 40, "b": false} + ])"}); + AssertTablesEqual(*expected, *collected, /*same_chunk_layout=*/false); +} + } // namespace dataset } // namespace arrow From 500ede5617831965d8dfae8e76a32405378a4f63 Mon Sep 17 00:00:00 2001 From: michalursa Date: Thu, 15 Jul 2021 17:48:54 -0700 Subject: [PATCH 21/41] GroupByNode: adding tests for parallel group by merge --- cpp/src/arrow/compute/exec/exec_plan.cc | 184 +++++++++++++++++- cpp/src/arrow/compute/exec/exec_plan.h | 9 + cpp/src/arrow/compute/exec/plan_test.cc | 128 +++++++----- .../compute/kernels/hash_aggregate_test.cc | 174 +++++++++-------- 4 files changed, 361 insertions(+), 134 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index df71777c989..c145296ec0d 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -22,10 +22,12 @@ #include #include +#include "arrow/array/concatenate.h" #include "arrow/array/util.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" #include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec_internal.h" #include "arrow/compute/registry.h" #include "arrow/datum.h" #include "arrow/record_batch.h" @@ -35,6 +37,9 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" #include "arrow/util/optional.h" +#include "arrow/util/task_group.h" +#include "arrow/util/thread_pool.h" +#include "arrow/util/vector.h" namespace arrow { @@ -338,6 +343,88 @@ ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, std::move(generator)); } +struct ParallelTestSourceNode : ExecNode { + ParallelTestSourceNode(ExecPlan* plan, std::string label, + std::shared_ptr schema, std::vector batches, + bool use_threads) + : ExecNode(plan, std::move(label), {}, {}, std::move(schema), + /*num_outputs=*/1), + batches_(std::move(batches)), + use_threads_(use_threads) {} + + const char* kind_name() override { return "ParallelTestSourceNode"; } + + [[noreturn]] static void NoInputs() { + DCHECK(false) << "no inputs; this should never be called"; + std::abort(); + } + [[noreturn]] void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); } + [[noreturn]] void ErrorReceived(ExecNode*, Status) override { NoInputs(); } + [[noreturn]] void InputFinished(ExecNode*, int) override { NoInputs(); } + + Status StartProducing() override { + DCHECK(!stop_requested_) << "Restarted ParllelTestSourceNode"; + + auto task_group = use_threads_ ? arrow::internal::TaskGroup::MakeThreaded( + arrow::internal::GetCpuThreadPool()) + : arrow::internal::TaskGroup::MakeSerial(); + + for (size_t i = 0; i < batches_.size(); ++i) { + task_group->Append([this, i] { + std::unique_lock lock(mutex_); + int seq = batch_count_++; + if (stop_requested_) { + return Status::OK(); + } + lock.unlock(); + + outputs_[0]->InputReceived(this, seq, batches_[seq]); + + if (seq + 1 == static_cast(batches_.size())) { + outputs_[0]->InputFinished(this, seq + 1); + } + + return Status::OK(); + }); + } + + RETURN_NOT_OK(task_group->Finish()); + + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + DCHECK_EQ(output, outputs_[0]); + StopProducing(); + } + + void StopProducing() override { + std::unique_lock lock(mutex_); + stop_requested_ = true; + } + + Future<> finished() override { return finished_; } + + private: + Future<> finished_ = Future<>::MakeFinished(); + std::mutex mutex_; + bool stop_requested_{false}; + int batch_count_{0}; + std::vector batches_; + bool use_threads_; +}; + +ExecNode* MakeParallelTestSourceNode(ExecPlan* plan, std::string label, + std::shared_ptr schema, + std::vector batches, bool use_threads) { + return plan->EmplaceNode( + plan, std::move(label), std::move(schema), std::move(batches), use_threads); +} + struct FilterNode : ExecNode { FilterNode(ExecNode* input, std::string label, Expression filter) : ExecNode(input->plan(), std::move(label), {input}, {"target"}, @@ -1062,9 +1149,9 @@ struct GroupByNode : ExecNode { } // Slice arrays - int64_t batch_start = n * output_batch_size_; - int64_t batch_length = - std::min(output_batch_size_, static_cast(num_out_groups_ - batch_start)); + 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] = @@ -1076,8 +1163,7 @@ struct GroupByNode : ExecNode { uint32_t num_output_batches_processed = 1 + num_output_batches_processed_.fetch_add(1); - if (static_cast(num_output_batches_processed) * output_batch_size_ >= - num_out_groups_) { + if (num_output_batches_processed * batch_size >= num_out_groups_) { finished_.MarkFinished(); } @@ -1093,8 +1179,8 @@ struct GroupByNode : ExecNode { RETURN_NOT_OK(Merge()); RETURN_NOT_OK(Finalize()); - int num_result_batches = - (num_out_groups_ + output_batch_size_ - 1) / output_batch_size_; + 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(); @@ -1181,9 +1267,16 @@ struct GroupByNode : 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; + } + ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); - int output_batch_size_{32 * 1024}; std::atomic num_input_batches_processed_; std::atomic num_input_batches_total_; @@ -1269,5 +1362,80 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::move(agg_kernels)); } +Result GroupByUsingExecPlan(const std::vector& arguments, + const std::vector& keys, + const std::vector& aggregates, + bool use_threads, ExecContext* ctx) { + using arrow::compute::detail::ExecBatchIterator; + + FieldVector scan_fields(arguments.size() + keys.size()); + std::vector keys_str(keys.size()); + std::vector arguments_str(arguments.size()); + for (size_t i = 0; i < arguments.size(); ++i) { + arguments_str[i] = std::string("agg_") + std::to_string(i); + scan_fields[i] = field(arguments_str[i], arguments[i].type()); + } + for (size_t i = 0; i < keys.size(); ++i) { + keys_str[i] = std::string("key_") + std::to_string(i); + scan_fields[arguments.size() + i] = field(keys_str[i], keys[i].type()); + } + + std::vector scan_batches; + std::vector inputs; + for (size_t i = 0; i < arguments.size(); ++i) { + inputs.push_back(arguments[i]); + } + for (size_t i = 0; i < keys.size(); ++i) { + inputs.push_back(keys[i]); + } + ARROW_ASSIGN_OR_RAISE(auto batch_iterator, + ExecBatchIterator::Make(inputs, ctx->exec_chunksize())); + ExecBatch batch; + while (batch_iterator->Next(&batch)) { + if (batch.length == 0) continue; + scan_batches.push_back(batch); + } + + ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(ctx)); + ExecNode* source; + source = MakeParallelTestSourceNode(plan.get(), "source", schema(scan_fields), + scan_batches, use_threads); + ARROW_ASSIGN_OR_RAISE( + auto gby, MakeGroupByNode(source, "gby", keys_str, arguments_str, aggregates)); + auto sink_gen = MakeSinkNode(gby, "sink"); + + RETURN_NOT_OK(plan->Validate()); + RETURN_NOT_OK(plan->StartProducing()); + + auto collected_fut = CollectAsyncGenerator(sink_gen); + + auto start_and_collect = + 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)); + }); + + std::vector output_batches = + std::move(start_and_collect.MoveResult().MoveValueUnsafe()); + + ArrayDataVector out_data(arguments.size() + keys.size()); + for (size_t i = 0; i < arguments.size() + keys.size(); ++i) { + std::vector> arrays(output_batches.size()); + for (size_t j = 0; j < output_batches.size(); ++j) { + arrays[j] = output_batches[j].values[i].make_array(); + } + ARROW_ASSIGN_OR_RAISE(auto concatenated_array, Concatenate(arrays)); + out_data[i] = concatenated_array->data(); + } + + int64_t length = out_data[0]->length; + return ArrayData::Make(struct_(gby->output_schema()->fields()), length, + {/*null_bitmap=*/nullptr}, std::move(out_data), + /*null_count=*/0); +} + } // 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 449555c474f..6bb48d8b71f 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -242,6 +242,10 @@ ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, std::shared_ptr output_schema, std::function>()>); +ExecNode* MakeParallelTestSourceNode(ExecPlan* plan, std::string label, + std::shared_ptr schema, + std::vector batches, bool use_threads); + /// \brief Add a sink node which forwards to an AsyncGenerator /// /// Emitted batches will not be ordered. @@ -291,5 +295,10 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector agg_srcs, std::vector aggs); +Result GroupByUsingExecPlan(const std::vector& arguments, + const std::vector& keys, + const std::vector& aggregates, + bool use_threads, ExecContext* ctx); + } // 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 8d84c1592a8..8223366c813 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -424,7 +424,7 @@ TEST(ExecPlanExecution, SourceProjectSink) { namespace { -BatchesWithSchema MakeGroupableBatches() { +BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { BatchesWithSchema out; out.batches = {ExecBatchFromJSON({int32(), utf8()}, R"([ @@ -443,6 +443,13 @@ BatchesWithSchema MakeGroupableBatches() { [-8, "alfa"] ])")}; + int num_batches = out.batches.size(); + for (int repeat = 1; repeat < multiplicity; ++repeat) { + for (int i = 0; i < num_batches; ++i) { + out.batches.push_back(out.batches[i]); + } + } + out.schema = schema({field("i32", int32()), field("str", utf8())}); return out; @@ -450,60 +457,83 @@ BatchesWithSchema MakeGroupableBatches() { } // namespace TEST(ExecPlanExecution, SourceGroupedSum) { - auto input = MakeGroupableBatches(); + for (bool use_threads : {false, true}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", input, - /*parallel=*/false, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN( - auto gby, MakeGroupByNode(source, "gby", /*keys=*/{"str"}, /*targets=*/{"i32"}, - {{"hash_sum", nullptr}})); - auto sink_gen = MakeSinkNode(gby, "sink"); + auto input = MakeGroupableBatches(/*multiplicity=*/use_threads ? 100 : 1); - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( - {int64(), utf8()}, R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + ExecNode* source; + if (use_threads) { + source = MakeParallelTestSourceNode(plan.get(), "source", input.schema, + input.batches, /*use_threads=*/true); + } else { + ASSERT_OK_AND_ASSIGN(source, + MakeTestSourceNode(plan.get(), "source", input, + /*parallel=*/false, /*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()}, + use_threads ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" + : R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); + } } TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { - auto input = MakeGroupableBatches(); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", input, - - /*parallel=*/false, /*slow=*/false)); - - ASSERT_OK_AND_ASSIGN( - auto filter, - MakeFilterNode(source, "filter", greater_equal(field_ref("i32"), literal(0)))); - - ASSERT_OK_AND_ASSIGN( - auto projection, - MakeProjectNode(filter, "project", - { - field_ref("str"), - call("multiply", {field_ref("i32"), literal(2)}), - })); - - ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(projection, "gby", /*keys=*/{"str"}, - /*targets=*/{"multiply(i32, 2)"}, - {{"hash_sum", nullptr}})); - - ASSERT_OK_AND_ASSIGN( - auto having, - MakeFilterNode(gby, "having", greater(field_ref("hash_sum"), literal(10)))); - - auto sink_gen = MakeSinkNode(having, "sink"); + for (bool use_threads : {false, true}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + + int batch_multiplicity = use_threads ? 100 : 1; + auto input = MakeGroupableBatches(/*multiplicity=*/batch_multiplicity); + + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + + ExecNode* source; + if (use_threads) { + source = MakeParallelTestSourceNode(plan.get(), "source", input.schema, + input.batches, /*use_threads=*/true); + } else { + ASSERT_OK_AND_ASSIGN(source, + MakeTestSourceNode(plan.get(), "source", input, + /*parallel=*/false, /*slow=*/false)); + } - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith( - UnorderedElementsAreArray({ExecBatchFromJSON({int64(), utf8()}, R"([ - [36, "alfa"], - [20, "beta"] - ])")})))); + ASSERT_OK_AND_ASSIGN( + auto filter, + MakeFilterNode(source, "filter", greater_equal(field_ref("i32"), literal(0)))); + + ASSERT_OK_AND_ASSIGN( + auto projection, + MakeProjectNode(filter, "project", + { + field_ref("str"), + call("multiply", {field_ref("i32"), literal(2)}), + })); + + ASSERT_OK_AND_ASSIGN(auto gby, MakeGroupByNode(projection, "gby", /*keys=*/{"str"}, + /*targets=*/{"multiply(i32, 2)"}, + {{"hash_sum", nullptr}})); + + ASSERT_OK_AND_ASSIGN( + auto having, + MakeFilterNode(gby, "having", + greater(field_ref("hash_sum"), literal(10 * batch_multiplicity)))); + + auto sink_gen = MakeSinkNode(having, "sink"); + + ASSERT_THAT( + StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( + {int64(), utf8()}, use_threads ? R"([[3600, "alfa"], [2000, "beta"]])" + : R"([[36, "alfa"], [20, "beta"]])")})))); + } } TEST(ExecPlanExecution, SourceScalarAggSink) { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 1d02b0c780a..276554c6fab 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -30,6 +30,8 @@ #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/test_util.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/codegen_internal.h" @@ -123,6 +125,25 @@ void ValidateGroupBy(const std::vector& aggregates, AssertDatumsEqual(expected, actual, /*verbose=*/true); } +ExecContext* small_chunksize_context() { + static ExecContext ctx; + ctx.set_exec_chunksize(2); + return &ctx; +} + +Result GroupByTest( + const std::vector& arguments, const std::vector& keys, + const std::vector<::arrow::compute::internal::Aggregate>& aggregates, + bool use_threads, bool use_exec_plan) { + if (use_exec_plan) { + return ::arrow::compute::GroupByUsingExecPlan(arguments, keys, aggregates, + use_threads, small_chunksize_context()); + } else { + return internal::GroupBy(arguments, keys, aggregates, use_threads, + default_exec_context()); + } +} + } // namespace TEST(Grouper, SupportedKeys) { @@ -534,143 +555,148 @@ void SortBy(std::vector names, Datum* aggregated_and_grouped) { } // namespace TEST(GroupBy, CountOnly) { - for (bool use_threads : {true, false}) { - SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + for (bool use_exec_plan : {false, true}) + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); - auto table = - TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ + auto table = TableFromJSON( + schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], [null, 1] ])", - R"([ + R"([ [0.0, 2], [null, 3], [4.0, null], [3.25, 1], [0.125, 2] ])", - R"([ + R"([ [-0.25, 2], [0.75, null], [null, 3] ])"}); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({table->GetColumnByName("argument")}, - {table->GetColumnByName("key")}, - { - {"hash_count", nullptr}, - }, - use_threads)); - SortBy({"key_0"}, &aggregated_and_grouped); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("hash_count", int64()), - field("key_0", int64()), - }), - R"([ + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupByTest({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_count", nullptr}, + }, + use_threads, use_exec_plan)); + SortBy({"key_0"}, &aggregated_and_grouped); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("hash_count", int64()), + field("key_0", int64()), + }), + R"([ [2, 1], [3, 2], [0, 3], [2, null] ])"), - aggregated_and_grouped, - /*verbose=*/true); - } + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, SumOnly) { - for (bool use_threads : {true, false}) { - SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + for (bool use_exec_plan : {false, true}) + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); - auto table = - TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ + auto table = TableFromJSON( + schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], [null, 1] ])", - R"([ + R"([ [0.0, 2], [null, 3], [4.0, null], [3.25, 1], [0.125, 2] ])", - R"([ + R"([ [-0.25, 2], [0.75, null], [null, 3] ])"}); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({table->GetColumnByName("argument")}, - {table->GetColumnByName("key")}, - { - {"hash_sum", nullptr}, - }, - use_threads)); - SortBy({"key_0"}, &aggregated_and_grouped); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("hash_sum", float64()), - field("key_0", int64()), - }), - R"([ + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupByTest({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_sum", nullptr}, + }, + use_threads, use_exec_plan)); + SortBy({"key_0"}, &aggregated_and_grouped); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("hash_sum", float64()), + field("key_0", int64()), + }), + R"([ [4.25, 1], [-0.125, 2], [null, 3], [4.75, null] ])"), - aggregated_and_grouped, - /*verbose=*/true); - } + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, MinMaxOnly) { - for (bool use_threads : {true, false}) { - SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + for (bool use_exec_plan : {false, true}) + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); - auto table = - TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ + auto table = TableFromJSON( + schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], [null, 1] ])", - R"([ + R"([ [0.0, 2], [null, 3], [4.0, null], [3.25, 1], [0.125, 2] ])", - R"([ + R"([ [-0.25, 2], [0.75, null], [null, 3] ])"}); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({table->GetColumnByName("argument")}, - {table->GetColumnByName("key")}, - { - {"hash_min_max", nullptr}, - })); - SortBy({"key_0"}, &aggregated_and_grouped); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("hash_min_max", struct_({ - field("min", float64()), - field("max", float64()), - })), - field("key_0", int64()), - }), - R"([ + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupByTest({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, + { + {"hash_min_max", nullptr}, + }, + use_threads, use_exec_plan)); + SortBy({"key_0"}, &aggregated_and_grouped); + + AssertDatumsEqual( + ArrayFromJSON(struct_({ + field("hash_min_max", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("key_0", int64()), + }), + R"([ [{"min": 1.0, "max": 3.25}, 1], [{"min": -0.25, "max": 0.125}, 2], [{"min": null, "max": null}, 3], [{"min": 0.75, "max": 4.0}, null] ])"), - aggregated_and_grouped, - /*verbose=*/true); - } + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, CountAndSum) { @@ -923,12 +949,6 @@ TEST(GroupBy, MinMaxWithNewGroupsInChunkedArray) { /*verbose=*/true); } -ExecContext* small_chunksize_context() { - static ExecContext ctx; - ctx.set_exec_chunksize(2); - return &ctx; -} - TEST(GroupBy, SmallChunkSizeSumOnly) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ From 1a496dce7eae7ed0a46529e860e5d9efaaf406c2 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 19 Jul 2021 16:43:53 -0400 Subject: [PATCH 22/41] merge failure: make_struct --- cpp/src/arrow/dataset/scanner_test.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index e8ce9072951..de7f780183a 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -1452,7 +1452,8 @@ TEST(ScanNode, MinimalScalarAggEndToEnd) { // for now, specify the projection as the full project expression (eventually this can // just be a list of materialized field names) compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); - options->projection = call("project", {a_times_2}, compute::ProjectOptions{{"a * 2"}}); + options->projection = + call("make_struct", {a_times_2}, compute::MakeStructOptions{{"a * 2"}}); // construct the scan node ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, @@ -1532,7 +1533,7 @@ TEST(ScanNode, MinimalGroupedAggEndToEnd) { compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); compute::Expression b = field_ref("b"); options->projection = - call("project", {a_times_2, b}, compute::ProjectOptions{{"a * 2", "b"}}); + call("make_struct", {a_times_2, b}, compute::MakeStructOptions{{"a * 2", "b"}}); // construct the scan node ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, From 810823597378a3b655a72c9c3610ae6c8a1a854f Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 12:18:37 -0400 Subject: [PATCH 23/41] remove TestParallelSourceNode --- cpp/src/arrow/compute/exec/exec_plan.cc | 101 ++---------------- cpp/src/arrow/compute/exec/exec_plan.h | 4 - cpp/src/arrow/compute/exec/plan_test.cc | 50 ++++----- .../compute/kernels/hash_aggregate_test.cc | 22 ++-- 4 files changed, 43 insertions(+), 134 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 0bad400ce6e..fa0e6504d76 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -343,88 +343,6 @@ ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, std::move(generator)); } -struct ParallelTestSourceNode : ExecNode { - ParallelTestSourceNode(ExecPlan* plan, std::string label, - std::shared_ptr schema, std::vector batches, - bool use_threads) - : ExecNode(plan, std::move(label), {}, {}, std::move(schema), - /*num_outputs=*/1), - batches_(std::move(batches)), - use_threads_(use_threads) {} - - const char* kind_name() override { return "ParallelTestSourceNode"; } - - [[noreturn]] static void NoInputs() { - DCHECK(false) << "no inputs; this should never be called"; - std::abort(); - } - [[noreturn]] void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); } - [[noreturn]] void ErrorReceived(ExecNode*, Status) override { NoInputs(); } - [[noreturn]] void InputFinished(ExecNode*, int) override { NoInputs(); } - - Status StartProducing() override { - DCHECK(!stop_requested_) << "Restarted ParllelTestSourceNode"; - - auto task_group = use_threads_ ? arrow::internal::TaskGroup::MakeThreaded( - arrow::internal::GetCpuThreadPool()) - : arrow::internal::TaskGroup::MakeSerial(); - - for (size_t i = 0; i < batches_.size(); ++i) { - task_group->Append([this, i] { - std::unique_lock lock(mutex_); - int seq = batch_count_++; - if (stop_requested_) { - return Status::OK(); - } - lock.unlock(); - - outputs_[0]->InputReceived(this, seq, batches_[seq]); - - if (seq + 1 == static_cast(batches_.size())) { - outputs_[0]->InputFinished(this, seq + 1); - } - - return Status::OK(); - }); - } - - RETURN_NOT_OK(task_group->Finish()); - - return Status::OK(); - } - - void PauseProducing(ExecNode* output) override {} - - void ResumeProducing(ExecNode* output) override {} - - void StopProducing(ExecNode* output) override { - DCHECK_EQ(output, outputs_[0]); - StopProducing(); - } - - void StopProducing() override { - std::unique_lock lock(mutex_); - stop_requested_ = true; - } - - Future<> finished() override { return finished_; } - - private: - Future<> finished_ = Future<>::MakeFinished(); - std::mutex mutex_; - bool stop_requested_{false}; - int batch_count_{0}; - std::vector batches_; - bool use_threads_; -}; - -ExecNode* MakeParallelTestSourceNode(ExecPlan* plan, std::string label, - std::shared_ptr schema, - std::vector batches, bool use_threads) { - return plan->EmplaceNode( - plan, std::move(label), std::move(schema), std::move(batches), use_threads); -} - struct FilterNode : ExecNode { FilterNode(ExecNode* input, std::string label, Expression filter) : ExecNode(input->plan(), std::move(label), {input}, {"target"}, @@ -1383,11 +1301,11 @@ Result GroupByUsingExecPlan(const std::vector& arguments, std::vector scan_batches; std::vector inputs; - for (size_t i = 0; i < arguments.size(); ++i) { - inputs.push_back(arguments[i]); + for (const auto& argument : arguments) { + inputs.push_back(argument); } - for (size_t i = 0; i < keys.size(); ++i) { - inputs.push_back(keys[i]); + for (const auto& key : keys) { + inputs.push_back(key); } ARROW_ASSIGN_OR_RAISE(auto batch_iterator, ExecBatchIterator::Make(inputs, ctx->exec_chunksize())); @@ -1398,9 +1316,12 @@ Result GroupByUsingExecPlan(const std::vector& arguments, } ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(ctx)); - ExecNode* source; - source = MakeParallelTestSourceNode(plan.get(), "source", schema(scan_fields), - scan_batches, use_threads); + auto source = MakeSourceNode( + plan.get(), "source", schema(std::move(scan_fields)), + MakeVectorGenerator(arrow::internal::MapVector( + [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, + std::move(scan_batches)))); + ARROW_ASSIGN_OR_RAISE( auto gby, MakeGroupByNode(source, "gby", keys_str, arguments_str, aggregates)); auto sink_gen = MakeSinkNode(gby, "sink"); @@ -1420,7 +1341,7 @@ Result GroupByUsingExecPlan(const std::vector& arguments, }); std::vector output_batches = - std::move(start_and_collect.MoveResult().MoveValueUnsafe()); + start_and_collect.MoveResult().MoveValueUnsafe(); ArrayDataVector out_data(arguments.size() + keys.size()); for (size_t i = 0; i < arguments.size() + keys.size(); ++i) { diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 6bb48d8b71f..32fcc132a76 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -242,10 +242,6 @@ ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, std::shared_ptr output_schema, std::function>()>); -ExecNode* MakeParallelTestSourceNode(ExecPlan* plan, std::string label, - std::shared_ptr schema, - std::vector batches, bool use_threads); - /// \brief Add a sink node which forwards to an AsyncGenerator /// /// Emitted batches will not be ordered. diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 2dc928782b9..eba3771e19f 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -457,22 +457,16 @@ BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { } // namespace TEST(ExecPlanExecution, SourceGroupedSum) { - for (bool use_threads : {false, true}) { - SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + for (bool parallel : {false, true}) { + SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); - auto input = MakeGroupableBatches(/*multiplicity=*/use_threads ? 100 : 1); + auto input = MakeGroupableBatches(/*multiplicity=*/parallel ? 100 : 1); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ExecNode* source; - if (use_threads) { - source = MakeParallelTestSourceNode(plan.get(), "source", input.schema, - input.batches, /*use_threads=*/true); - } else { - ASSERT_OK_AND_ASSIGN(source, - MakeTestSourceNode(plan.get(), "source", input, - /*parallel=*/false, /*slow=*/false)); - } + 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}})); @@ -481,30 +475,23 @@ TEST(ExecPlanExecution, SourceGroupedSum) { ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( {int64(), utf8()}, - use_threads ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" - : R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); + parallel ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" + : R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); } } TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { - for (bool use_threads : {false, true}) { - SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + for (bool parallel : {false, true}) { + SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); - int batch_multiplicity = use_threads ? 100 : 1; + int batch_multiplicity = parallel ? 100 : 1; auto input = MakeGroupableBatches(/*multiplicity=*/batch_multiplicity); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - ExecNode* source; - if (use_threads) { - source = MakeParallelTestSourceNode(plan.get(), "source", input.schema, - input.batches, /*use_threads=*/true); - } else { - ASSERT_OK_AND_ASSIGN(source, - MakeTestSourceNode(plan.get(), "source", input, - /*parallel=*/false, /*slow=*/false)); - } - + ASSERT_OK_AND_ASSIGN(auto source, + MakeTestSourceNode(plan.get(), "source", input, + /*parallel=*/parallel, /*slow=*/false)); ASSERT_OK_AND_ASSIGN( auto filter, MakeFilterNode(source, "filter", greater_equal(field_ref("i32"), literal(0)))); @@ -528,11 +515,10 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { auto sink_gen = MakeSinkNode(having, "sink"); - ASSERT_THAT( - StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( - {int64(), utf8()}, use_threads ? R"([[3600, "alfa"], [2000, "beta"]])" - : R"([[36, "alfa"], [20, "beta"]])")})))); + ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), + Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( + {int64(), utf8()}, parallel ? R"([[3600, "alfa"], [2000, "beta"]])" + : R"([[36, "alfa"], [20, "beta"]])")})))); } } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 276554c6fab..b0327c7aa81 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -50,6 +50,7 @@ #include "arrow/util/int_util_internal.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" +#include "arrow/util/thread_pool.h" using testing::HasSubstr; @@ -125,10 +126,12 @@ void ValidateGroupBy(const std::vector& aggregates, AssertDatumsEqual(expected, actual, /*verbose=*/true); } -ExecContext* small_chunksize_context() { - static ExecContext ctx; +ExecContext* small_chunksize_context(bool use_threads = false) { + static ExecContext ctx, + ctx_with_threads{default_memory_pool(), arrow::internal::GetCpuThreadPool()}; ctx.set_exec_chunksize(2); - return &ctx; + ctx_with_threads.set_exec_chunksize(2); + return use_threads ? &ctx_with_threads : &ctx; } Result GroupByTest( @@ -136,8 +139,8 @@ Result GroupByTest( const std::vector<::arrow::compute::internal::Aggregate>& aggregates, bool use_threads, bool use_exec_plan) { if (use_exec_plan) { - return ::arrow::compute::GroupByUsingExecPlan(arguments, keys, aggregates, - use_threads, small_chunksize_context()); + return GroupByUsingExecPlan(arguments, keys, aggregates, use_threads, + small_chunksize_context(use_threads)); } else { return internal::GroupBy(arguments, keys, aggregates, use_threads, default_exec_context()); @@ -555,7 +558,7 @@ void SortBy(std::vector names, Datum* aggregated_and_grouped) { } // namespace TEST(GroupBy, CountOnly) { - for (bool use_exec_plan : {false, true}) + for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -599,10 +602,11 @@ TEST(GroupBy, CountOnly) { aggregated_and_grouped, /*verbose=*/true); } + } } TEST(GroupBy, SumOnly) { - for (bool use_exec_plan : {false, true}) + for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -646,10 +650,11 @@ TEST(GroupBy, SumOnly) { aggregated_and_grouped, /*verbose=*/true); } + } } TEST(GroupBy, MinMaxOnly) { - for (bool use_exec_plan : {false, true}) + for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -697,6 +702,7 @@ TEST(GroupBy, MinMaxOnly) { aggregated_and_grouped, /*verbose=*/true); } + } } TEST(GroupBy, CountAndSum) { From e49c06030c1bbc1ff55ced6d2fa293b631264af4 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 12:22:07 -0400 Subject: [PATCH 24/41] msvc: explicit int cast --- cpp/src/arrow/compute/exec/plan_test.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index eba3771e19f..10348d6ffb6 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -443,9 +443,8 @@ BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { [-8, "alfa"] ])")}; - int num_batches = out.batches.size(); for (int repeat = 1; repeat < multiplicity; ++repeat) { - for (int i = 0; i < num_batches; ++i) { + for (size_t i = 0; i < out.batches.size(); ++i) { out.batches.push_back(out.batches[i]); } } From a0c5d094ea17624d2057bf6411ae6054ae751719 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 13:07:48 -0400 Subject: [PATCH 25/41] fix embarassing fork bomb --- cpp/src/arrow/compute/exec/plan_test.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 10348d6ffb6..aa807468bcb 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -443,8 +443,9 @@ BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { [-8, "alfa"] ])")}; + size_t batch_count = out.batches.size(); for (int repeat = 1; repeat < multiplicity; ++repeat) { - for (size_t i = 0; i < out.batches.size(); ++i) { + for (size_t i = 0; i < batch_count; ++i) { out.batches.push_back(out.batches[i]); } } From 9c5d9d3af7e96d146461fe627d945971628c717f Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 16:09:43 -0400 Subject: [PATCH 26/41] ensure ScalarAggregateNode::StopProducing prevents output from being emitted --- cpp/src/arrow/compute/exec/exec_plan.cc | 114 +++++++++++++++++------- cpp/src/arrow/compute/exec/exec_plan.h | 1 + 2 files changed, 85 insertions(+), 30 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index fa0e6504d76..ab5de78e4f8 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -733,6 +733,10 @@ struct ScalarAggregateNode : ExecNode { void StopProducing() override { inputs_[0]->StopProducing(this); + std::unique_lock lock(mutex_); + if (finished_.is_finished()) return; + states_.clear(); + lock.unlock(); finished_.MarkFinished(); } @@ -825,23 +829,65 @@ Result MakeScalarAggregateNode(ExecNode* input, std::string label, std::move(states)); } +template +constexpr int clz(T value) { + return value == 0 ? sizeof(T) * 8 : clz(value >> 1) - 1; +} + +static_assert(clz(0) == 32, ""); +static_assert(clz(1) == 31, ""); +static_assert(clz(2) == 30, ""); +static_assert(clz(3) == 30, ""); +static_assert(clz(4) == 29, ""); +static_assert(clz(7) == 29, ""); +static_assert(clz(8) == 28, ""); +static_assert(clz(unsigned(-1)) == 0, ""); +static_assert(clz(unsigned(-1) >> 1) == 1, ""); + +constexpr int vid(int id) { return 32 - clz(unsigned(id)); } + +constexpr int size(int vid) { return 1 << vid; } + +static_assert(vid(0) == 0, ""); +static_assert(size(vid(0)) == 1, ""); + +static_assert(vid(1) == 1, ""); +static_assert(size(vid(1)) == 2, ""); + +static_assert(vid(2) == 2, ""); +static_assert(vid(3) == 2, ""); +static_assert(size(vid(2)) == 4, ""); + +static_assert(vid(4) == 3, ""); +static_assert(vid(5) == 3, ""); +static_assert(vid(6) == 3, ""); +static_assert(vid(7) == 3, ""); +static_assert(size(vid(4)) == 8, ""); + +// ... + +static_assert(vid(1 << 14) == 15, ""); +// ... +static_assert(vid((1 << 15) - 1) == 15, ""); +static_assert(size(vid(1 << 14)) == 1 << 15, ""); + template class SharedSequenceOfObjects { public: explicit SharedSequenceOfObjects(int log_max_objects = 16) { objects_.resize(log_max_objects); - num_created_vectors_ = 0; } + T* get(int id) { int vid = vector_id(id); - ARROW_DCHECK(static_cast(vid) < objects_.size()); - if (vid >= num_created_vectors_) { - std::lock_guard lock(mutex_); - while (vid >= num_created_vectors_) { - objects_[num_created_vectors_].resize(static_cast(1) - << num_created_vectors_); - ++num_created_vectors_; - } + std::lock_guard lock(mutex_); + + DCHECK_LT(static_cast(vid), objects_.size()); + + while (vid >= num_created_vectors_) { + objects_[num_created_vectors_].resize(static_cast(1) + << num_created_vectors_); + ++num_created_vectors_; } return &objects_[vid][id]; } @@ -851,14 +897,15 @@ class SharedSequenceOfObjects { return 32 - CountLeadingZeros(static_cast(id)); } - int num_created_vectors_; + int num_created_vectors_ = 0; std::vector> objects_; std::mutex mutex_; }; class SmallUniqueIdAssignment { public: - SmallUniqueIdAssignment() { num_ids_ = 0; } + SmallUniqueIdAssignment() = default; + int Get() { std::lock_guard lock(mutex_); if (stack_.empty()) { @@ -869,28 +916,36 @@ class SmallUniqueIdAssignment { return last; } } + void Return(int id) { std::lock_guard lock(mutex_); stack_.push_back(id); } - int num_ids() const { return num_ids_; } + + int num_ids() const { + std::lock_guard lock(mutex_); + return num_ids_; + } private: std::vector stack_; - int num_ids_; - std::mutex mutex_; + int num_ids_ = 0; + mutable std::mutex mutex_; }; class SmallUniqueIdHolder { public: SmallUniqueIdHolder() = delete; SmallUniqueIdHolder(const SmallUniqueIdHolder&) = delete; - SmallUniqueIdHolder(const SmallUniqueIdHolder&&) = delete; + SmallUniqueIdHolder(SmallUniqueIdHolder&&) = delete; + explicit SmallUniqueIdHolder(SmallUniqueIdAssignment* id_mgr) { id_mgr_ = id_mgr; id_ = id_mgr->Get(); } + ~SmallUniqueIdHolder() { id_mgr_->Return(id_); } + int get() const { return id_; } private: @@ -961,16 +1016,15 @@ struct GroupByNode : ExecNode { 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)); - } + + 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(); } @@ -1012,15 +1066,15 @@ struct GroupByNode : ExecNode { 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); + DCHECK(state); + 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]); + 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())); @@ -1060,7 +1114,7 @@ struct GroupByNode : ExecNode { } Status OutputNthBatch(int n) { - ARROW_DCHECK(output_started_.load()); + DCHECK(output_started_.load()); // Check finished flag if (finished_.is_finished()) { @@ -1127,7 +1181,7 @@ struct GroupByNode : ExecNode { return; } - ARROW_DCHECK(num_input_batches_processed_.load() != num_input_batches_total_.load()); + DCHECK_NE(num_input_batches_processed_.load(), num_input_batches_total_.load()); Status status = ProcessInputBatch(batch); if (!status.ok()) { diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 32fcc132a76..84ebdac3941 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -291,6 +291,7 @@ Result MakeGroupByNode(ExecNode* input, std::string label, std::vector agg_srcs, std::vector aggs); +ARROW_EXPORT Result GroupByUsingExecPlan(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, From b35e1fe1c34f93f9ac694928d9e120c54ca2be0a Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 16:33:11 -0400 Subject: [PATCH 27/41] simplify GroupByNode state management --- cpp/src/arrow/compute/exec/exec_plan.cc | 152 +++--------------------- 1 file changed, 18 insertions(+), 134 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index ab5de78e4f8..3203bbdf112 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -829,130 +829,6 @@ Result MakeScalarAggregateNode(ExecNode* input, std::string label, std::move(states)); } -template -constexpr int clz(T value) { - return value == 0 ? sizeof(T) * 8 : clz(value >> 1) - 1; -} - -static_assert(clz(0) == 32, ""); -static_assert(clz(1) == 31, ""); -static_assert(clz(2) == 30, ""); -static_assert(clz(3) == 30, ""); -static_assert(clz(4) == 29, ""); -static_assert(clz(7) == 29, ""); -static_assert(clz(8) == 28, ""); -static_assert(clz(unsigned(-1)) == 0, ""); -static_assert(clz(unsigned(-1) >> 1) == 1, ""); - -constexpr int vid(int id) { return 32 - clz(unsigned(id)); } - -constexpr int size(int vid) { return 1 << vid; } - -static_assert(vid(0) == 0, ""); -static_assert(size(vid(0)) == 1, ""); - -static_assert(vid(1) == 1, ""); -static_assert(size(vid(1)) == 2, ""); - -static_assert(vid(2) == 2, ""); -static_assert(vid(3) == 2, ""); -static_assert(size(vid(2)) == 4, ""); - -static_assert(vid(4) == 3, ""); -static_assert(vid(5) == 3, ""); -static_assert(vid(6) == 3, ""); -static_assert(vid(7) == 3, ""); -static_assert(size(vid(4)) == 8, ""); - -// ... - -static_assert(vid(1 << 14) == 15, ""); -// ... -static_assert(vid((1 << 15) - 1) == 15, ""); -static_assert(size(vid(1 << 14)) == 1 << 15, ""); - -template -class SharedSequenceOfObjects { - public: - explicit SharedSequenceOfObjects(int log_max_objects = 16) { - objects_.resize(log_max_objects); - } - - T* get(int id) { - int vid = vector_id(id); - std::lock_guard lock(mutex_); - - DCHECK_LT(static_cast(vid), objects_.size()); - - while (vid >= num_created_vectors_) { - objects_[num_created_vectors_].resize(static_cast(1) - << num_created_vectors_); - ++num_created_vectors_; - } - return &objects_[vid][id]; - } - - private: - static int vector_id(int id) { - return 32 - CountLeadingZeros(static_cast(id)); - } - - int num_created_vectors_ = 0; - std::vector> objects_; - std::mutex mutex_; -}; - -class SmallUniqueIdAssignment { - public: - SmallUniqueIdAssignment() = default; - - int Get() { - std::lock_guard lock(mutex_); - if (stack_.empty()) { - return num_ids_++; - } else { - int last = stack_.back(); - stack_.pop_back(); - return last; - } - } - - void Return(int id) { - std::lock_guard lock(mutex_); - stack_.push_back(id); - } - - int num_ids() const { - std::lock_guard lock(mutex_); - return num_ids_; - } - - private: - std::vector stack_; - int num_ids_ = 0; - mutable std::mutex mutex_; -}; - -class SmallUniqueIdHolder { - public: - SmallUniqueIdHolder() = delete; - SmallUniqueIdHolder(const SmallUniqueIdHolder&) = delete; - SmallUniqueIdHolder(SmallUniqueIdHolder&&) = delete; - - explicit SmallUniqueIdHolder(SmallUniqueIdAssignment* id_mgr) { - id_mgr_ = id_mgr; - id_ = id_mgr->Get(); - } - - ~SmallUniqueIdHolder() { id_mgr_->Return(id_); } - - int get() const { return id_; } - - private: - SmallUniqueIdAssignment* id_mgr_; - int id_; -}; - namespace internal { Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, @@ -993,6 +869,14 @@ struct GroupByNode : ExecNode { struct ThreadLocalState; public: + ThreadLocalState* GetLocalState() { + std::unique_lock lock(mutex_); + auto it = + thread_indices_.emplace(std::this_thread::get_id(), thread_indices_.size()).first; + auto thread_index = it->second; + return &local_states_[thread_index]; + } + Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema auto input_schema = inputs_[0]->output_schema(); @@ -1030,9 +914,7 @@ struct GroupByNode : ExecNode { } Status ProcessInputBatch(const ExecBatch& batch) { - SmallUniqueIdHolder id_holder(&local_state_id_assignment_); - int id = id_holder.get(); - ThreadLocalState* state = local_states_.get(id); + ThreadLocalState* state = GetLocalState(); RETURN_NOT_OK(InitLocalStateIfNeeded(state)); // Create a batch with key columns @@ -1062,10 +944,9 @@ struct GroupByNode : ExecNode { } 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); + ThreadLocalState* state0 = &local_states_[0]; + for (size_t i = 1; i < local_states_.size(); ++i) { + ThreadLocalState* state = &local_states_[i]; DCHECK(state); DCHECK(state->grouper); ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); @@ -1090,7 +971,7 @@ struct GroupByNode : ExecNode { out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); auto it = out_data_.begin(); - ThreadLocalState* state = local_states_.get(0); + ThreadLocalState* state = &local_states_[0]; num_out_groups_ = state->grouper->num_groups(); // Aggregate fields come before key fields to match the behavior of GroupBy function @@ -1221,6 +1102,8 @@ struct GroupByNode : ExecNode { Status StartProducing() override { finished_ = Future<>::Make(); + auto executor = plan_->exec_context()->executor(); + local_states_.resize(executor ? executor->GetCapacity() : 1); return Status::OK(); } @@ -1251,6 +1134,7 @@ struct GroupByNode : ExecNode { ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); + std::mutex mutex_; std::atomic num_input_batches_processed_; std::atomic num_input_batches_total_; std::atomic num_output_batches_processed_; @@ -1264,8 +1148,8 @@ struct GroupByNode : ExecNode { std::unique_ptr grouper; std::vector> agg_states; }; - SharedSequenceOfObjects local_states_; - SmallUniqueIdAssignment local_state_id_assignment_; + std::vector local_states_; + std::unordered_map thread_indices_; uint32_t num_out_groups_{0}; ArrayDataVector out_data_; std::atomic output_started_; From 6f3b077852d915ae317d511d0e420804a39e9dc6 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 16:44:30 -0400 Subject: [PATCH 28/41] use a correct ExecContext in CountRows --- cpp/src/arrow/dataset/scanner.cc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index 57597002392..18542219a42 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -784,7 +784,11 @@ Future> AsyncScanner::ToTableAsync( Result AsyncScanner::CountRows() { ARROW_ASSIGN_OR_RAISE(auto fragment_gen, GetFragments()); - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make()); + + auto cpu_executor = scan_options_->use_threads ? internal::GetCpuThreadPool() : nullptr; + compute::ExecContext exec_context(scan_options_->pool, cpu_executor); + + ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(&exec_context)); // Drop projection since we only need to count rows auto options = std::make_shared(*scan_options_); RETURN_NOT_OK(SetProjection(options.get(), std::vector())); From a457e7aef1cc6128a667b325e3f2e224b79ce687 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 20 Jul 2021 17:40:04 -0400 Subject: [PATCH 29/41] wait for plan to finish before checking for errors --- cpp/src/arrow/compute/exec/exec_plan.cc | 6 ++++-- cpp/src/arrow/dataset/scanner.cc | 5 ++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 3203bbdf112..f119bc07fc3 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -947,8 +947,10 @@ struct GroupByNode : ExecNode { ThreadLocalState* state0 = &local_states_[0]; for (size_t i = 1; i < local_states_.size(); ++i) { ThreadLocalState* state = &local_states_[i]; - DCHECK(state); - DCHECK(state->grouper); + if (!state->grouper) { + continue; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, state->grouper->GetUniques()); ARROW_ASSIGN_OR_RAISE(Datum transposition, state0->grouper->Consume(other_keys)); state->grouper.reset(); diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index 18542219a42..192f84f46df 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -829,13 +829,12 @@ Result AsyncScanner::CountRows() { compute::MakeSinkNode(sum_selection, "sink"); RETURN_NOT_OK(plan->StartProducing()); - auto maybe_slow_count = sink_gen().result(); + plan->finished().Wait(); + ARROW_ASSIGN_OR_RAISE(auto slow_count, maybe_slow_count); total += slow_count->values[0].scalar_as().value; - plan->finished().Wait(); - return total.load(); } From d26564d56617d9c9fc11bc052eab557f59d046f1 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 21 Jul 2021 17:27:37 -0400 Subject: [PATCH 30/41] extract some helpers for tracking input count/completion --- cpp/src/arrow/compute/exec/exec_plan.cc | 346 ++++++++++++------------ cpp/src/arrow/compute/exec/exec_plan.h | 4 + 2 files changed, 177 insertions(+), 173 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index f119bc07fc3..599b5b9c5a8 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -244,6 +244,15 @@ Status ExecNode::Validate() const { return Status::OK(); } +bool ExecNode::ErrorIfNotOk(Status status) { + if (status.ok()) return false; + + for (auto out : outputs_) { + out->ErrorReceived(this, out == outputs_.back() ? std::move(status) : status); + } + return true; +} + struct SourceNode : ExecNode { SourceNode(ExecPlan* plan, std::string label, std::shared_ptr output_schema, AsyncGenerator> generator) @@ -384,10 +393,7 @@ struct FilterNode : ExecNode { DCHECK_EQ(input, inputs_[0]); auto maybe_filtered = DoFilter(std::move(batch)); - if (!maybe_filtered.ok()) { - outputs_[0]->ErrorReceived(this, maybe_filtered.status()); - return; - } + if (ErrorIfNotOk(maybe_filtered.status())) return; maybe_filtered->guarantee = batch.guarantee; outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe()); @@ -463,10 +469,7 @@ struct ProjectNode : ExecNode { DCHECK_EQ(input, inputs_[0]); auto maybe_projected = DoProject(std::move(batch)); - if (!maybe_projected.ok()) { - outputs_[0]->ErrorReceived(this, maybe_projected.status()); - return; - } + if (ErrorIfNotOk(maybe_projected.status())) return; maybe_projected->guarantee = batch.guarantee; outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe()); @@ -526,6 +529,44 @@ Result MakeProjectNode(ExecNode* input, std::string label, input, std::move(label), schema(std::move(fields)), std::move(exprs)); } +class InputCounter { + public: + InputCounter() = default; + + int count() const { return count_.load(); } + + util::optional total() const { + int total = total_.load(); + if (total == -1) return {}; + return total; + } + + // return true if the counter is complete + bool Increment() { + DCHECK_NE(count_.load(), total_.load()); + int count = count_.fetch_add(1) + 1; + return IsComplete(count, total_.load()); + } + + // return true if the counter is complete + bool SetTotal(int total) { + total_.store(total); + return IsComplete(count_.load(), total); + } + + private: + // ensure there is only one true return from Increment() or SetTotal() + bool IsComplete(int count, int total) { + if (count != total) return false; + + 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) @@ -567,21 +608,15 @@ struct SinkNode : ExecNode { void InputReceived(ExecNode* input, int seq_num, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - if (finished_.is_finished()) return; - - producer_.Push(std::move(batch)); - - std::unique_lock lock(mutex_); + bool did_push = producer_.Push(std::move(batch)); + if (!did_push) return; // producer_ was Closed already - ++num_received_; - if (num_received_ == emit_stop_) { - lock.unlock(); - Finish(); - return; + if (auto total = input_counter_.total()) { + DCHECK_LE(seq_num, *total); } - if (emit_stop_ != -1) { - DCHECK_LE(seq_num, emit_stop_); + if (input_counter_.Increment()) { + Finish(); } } @@ -593,10 +628,7 @@ struct SinkNode : ExecNode { } void InputFinished(ExecNode* input, int seq_stop) override { - std::unique_lock lock(mutex_); - emit_stop_ = seq_stop; - if (num_received_ == emit_stop_) { - lock.unlock(); + if (input_counter_.SetTotal(seq_stop)) { Finish(); } } @@ -608,10 +640,7 @@ struct SinkNode : ExecNode { } } - std::mutex mutex_; - - int num_received_ = 0; - int emit_stop_ = -1; + InputCounter input_counter_; Future<> finished_ = Future<>::MakeFinished(); PushGenerator>::Producer producer_; @@ -652,6 +681,23 @@ std::shared_ptr MakeGeneratorReader( return out; } +class ThreadIndexer { + public: + size_t operator()(std::mutex* mutex) { + std::unique_lock lock(*mutex); + return this->operator()(); + } + + size_t operator()() { + auto id = std::this_thread::get_id(); + const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; + return id_index.second; + } + + private: + std::unordered_map id_to_index_; +}; + struct ScalarAggregateNode : ExecNode { ScalarAggregateNode(ExecNode* input, std::string label, std::shared_ptr output_schema, @@ -667,8 +713,11 @@ struct ScalarAggregateNode : ExecNode { Status DoConsume(const ExecBatch& batch, size_t thread_index) { for (size_t i = 0; i < kernels_.size(); ++i) { + DCHECK_LT(thread_index, states_[i].size()); + KernelContext batch_ctx{plan()->exec_context()}; batch_ctx.SetState(states_[i][thread_index].get()); + ExecBatch single_column_batch{{batch.values[i]}, batch.length}; RETURN_NOT_OK(kernels_[i]->consume(&batch_ctx, single_column_batch)); } @@ -678,25 +727,12 @@ struct ScalarAggregateNode : ExecNode { void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); - auto it = - thread_indices_.emplace(std::this_thread::get_id(), thread_indices_.size()).first; - auto thread_index = it->second; - - lock.unlock(); + auto thread_index = get_thread_index_(&mutex_); - Status st = DoConsume(std::move(batch), thread_index); - if (!st.ok()) { - outputs_[0]->ErrorReceived(this, std::move(st)); - return; - } + if (ErrorIfNotOk(DoConsume(std::move(batch), thread_index))) return; - lock.lock(); - ++num_received_; - st = MaybeFinish(&lock); - if (!st.ok()) { - outputs_[0]->ErrorReceived(this, std::move(st)); - } + if (!input_counter_.Increment()) return; + if (ErrorIfNotOk(MaybeFinish())) return; } void ErrorReceived(ExecNode* input, Status error) override { @@ -704,15 +740,11 @@ struct ScalarAggregateNode : ExecNode { outputs_[0]->ErrorReceived(this, std::move(error)); } - void InputFinished(ExecNode* input, int seq) override { + void InputFinished(ExecNode* input, int num_total) override { DCHECK_EQ(input, inputs_[0]); - std::unique_lock lock(mutex_); - num_total_ = seq; - Status st = MaybeFinish(&lock); - if (!st.ok()) { - outputs_[0]->ErrorReceived(this, std::move(st)); - } + if (!input_counter_.SetTotal(num_total)) return; + if (ErrorIfNotOk(MaybeFinish())) return; } Status StartProducing() override { @@ -732,20 +764,20 @@ struct ScalarAggregateNode : ExecNode { } void StopProducing() override { - inputs_[0]->StopProducing(this); std::unique_lock lock(mutex_); - if (finished_.is_finished()) return; + if (states_.empty()) return; states_.clear(); lock.unlock(); + + inputs_[0]->StopProducing(this); finished_.MarkFinished(); } Future<> finished() override { return finished_; } private: - Status MaybeFinish(std::unique_lock* lock) { - if (num_received_ != num_total_) return Status::OK(); - + Status MaybeFinish() { + std::unique_lock lock(mutex_); if (states_.empty()) return Status::OK(); ExecBatch batch{{}, 1}; @@ -758,20 +790,22 @@ struct ScalarAggregateNode : ExecNode { RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); } states_.clear(); - lock->unlock(); - outputs_[0]->InputReceived(this, 0, batch); + lock.unlock(); + outputs_[0]->InputReceived(this, 0, batch); finished_.MarkFinished(); return Status::OK(); } Future<> finished_ = Future<>::MakeFinished(); std::vector kernels_; + std::vector>> states_; - std::unordered_map thread_indices_; std::mutex mutex_; - int num_received_ = 0, num_total_ = -1; + + ThreadIndexer get_thread_index_; + InputCounter input_counter_; }; Result MakeScalarAggregateNode(ExecNode* input, std::string label, @@ -830,18 +864,22 @@ Result MakeScalarAggregateNode(ExecNode* input, std::string label, } namespace internal { + Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs); + Result>> InitKernels( const std::vector& kernels, ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs); + Result ResolveKernels( const std::vector& aggregates, const std::vector& kernels, const std::vector>& states, ExecContext* ctx, const std::vector& descrs); + } // namespace internal struct GroupByNode : ExecNode { @@ -856,63 +894,10 @@ struct GroupByNode : ExecNode { key_field_ids_(std::move(key_field_ids)), agg_src_field_ids_(std::move(agg_src_field_ids)), aggs_(std::move(aggs)), - agg_kernels_(std::move(agg_kernels)) { - num_input_batches_processed_.store(0); - num_input_batches_total_.store(-1); - num_output_batches_processed_.store(0); - output_started_.store(false); - } + agg_kernels_(std::move(agg_kernels)) {} const char* kind_name() override { return "GroupByNode"; } - private: - struct ThreadLocalState; - - public: - ThreadLocalState* GetLocalState() { - std::unique_lock lock(mutex_); - auto it = - thread_indices_.emplace(std::this_thread::get_id(), thread_indices_.size()).first; - auto thread_index = it->second; - return &local_states_[thread_index]; - } - - 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); - } - - 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) { ThreadLocalState* state = GetLocalState(); RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -997,8 +982,6 @@ struct GroupByNode : ExecNode { } Status OutputNthBatch(int n) { - DCHECK(output_started_.load()); - // Check finished flag if (finished_.is_finished()) { return Status::OK(); @@ -1007,11 +990,9 @@ struct GroupByNode : ExecNode { // 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); + for (size_t i = 0; i < out_data_.size(); ++i) { + output_slices[i] = out_data_[i]->Slice(batch_start, batch_size); } ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); @@ -1027,11 +1008,6 @@ struct GroupByNode : ExecNode { } Status OutputResult() { - bool expected = false; - if (!output_started_.compare_exchange_strong(expected, true)) { - return Status::OK(); - } - RETURN_NOT_OK(Merge()); RETURN_NOT_OK(Finalize()); @@ -1039,73 +1015,59 @@ struct GroupByNode : ExecNode { int num_result_batches = (num_out_groups_ + batch_size - 1) / batch_size; outputs_[0]->InputFinished(this, num_result_batches); - auto executor = arrow::internal::GetCpuThreadPool(); + auto executor = ctx_->executor(); for (int i = 0; i < num_result_batches; ++i) { - // Check finished flag - if (finished_.is_finished()) { - break; + // bail if StopProducing was called + if (finished_.is_finished()) break; + + if (executor) { + RETURN_NOT_OK(executor->Spawn([this, i]() { + Status status = OutputNthBatch(i); + if (!status.ok()) { + ErrorReceived(inputs_[0], status); + } + })); + } else { + RETURN_NOT_OK(OutputNthBatch(i)); } - - 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]); + // bail if StopProducing was called + if (finished_.is_finished()) return; - if (finished_.is_finished()) { - return; - } + DCHECK_EQ(input, inputs_[0]); - DCHECK_NE(num_input_batches_processed_.load(), num_input_batches_total_.load()); + if (ErrorIfNotOk(ProcessInputBatch(batch))) return; - Status status = ProcessInputBatch(batch); - if (!status.ok()) { - ErrorReceived(input, status); - return; - } + if (!input_counter_.Increment()) 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; - } - } + if (ErrorIfNotOk(OutputResult())) 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 { + void InputFinished(ExecNode* input, int num_total) 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 (!input_counter_.SetTotal(num_total)) return; - if (!status.ok()) { - ErrorReceived(input, status); - } - } + if (ErrorIfNotOk(OutputResult())) return; } Status StartProducing() override { finished_ = Future<>::Make(); + auto executor = plan_->exec_context()->executor(); local_states_.resize(executor ? executor->GetCapacity() : 1); + return Status::OK(); } @@ -1117,6 +1079,7 @@ struct GroupByNode : ExecNode { DCHECK_EQ(output, outputs_[0]); inputs_[0]->StopProducing(this); + // FIXME StopProducing must be idempotent, but we may call MarkFinished twice finished_.MarkFinished(); } @@ -1125,6 +1088,48 @@ struct GroupByNode : ExecNode { Future<> finished() override { return finished_; } private: + struct ThreadLocalState { + std::unique_ptr grouper; + std::vector> agg_states; + }; + + ThreadLocalState* GetLocalState() { return &local_states_[get_thread_index_(&mutex_)]; } + + Status InitLocalStateIfNeeded(ThreadLocalState* state) { + // Get input schema + auto input_schema = inputs_[0]->output_schema(); + + if (state->grouper != nullptr) return Status::OK(); + + // 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_)); + + // 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); + } + + 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(); + } + int output_batch_size() const { int result = static_cast(ctx_->exec_chunksize()); if (result < 0) { @@ -1137,24 +1142,19 @@ struct GroupByNode : ExecNode { Future<> finished_ = Future<>::MakeFinished(); std::mutex mutex_; - std::atomic num_input_batches_processed_; - std::atomic num_input_batches_total_; - std::atomic num_output_batches_processed_; + std::atomic num_output_batches_processed_{0}; const std::vector key_field_ids_; const std::vector agg_src_field_ids_; const std::vector aggs_; const std::vector agg_kernels_; - struct ThreadLocalState { - std::unique_ptr grouper; - std::vector> agg_states; - }; + ThreadIndexer get_thread_index_; + InputCounter input_counter_; + std::vector local_states_; - std::unordered_map thread_indices_; uint32_t num_out_groups_{0}; ArrayDataVector out_data_; - std::atomic output_started_; }; Result MakeGroupByNode(ExecNode* input, std::string label, diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 84ebdac3941..07bb365bbc7 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -222,6 +222,10 @@ class ARROW_EXPORT ExecNode { std::vector input_labels, std::shared_ptr output_schema, int num_outputs); + // A helper method to send an error status to all outputs. + // Returns true if the status was an error. + bool ErrorIfNotOk(Status status); + ExecPlan* plan_; std::string label_; From 0e26d6c9cc8649cb31b1ac5515b055a438ce53dc Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 21 Jul 2021 20:36:45 -0400 Subject: [PATCH 31/41] ensure we call Future<>::MarkFinished only once --- cpp/src/arrow/compute/exec/exec_plan.cc | 67 +++++++++++++------------ 1 file changed, 36 insertions(+), 31 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 599b5b9c5a8..b952489e883 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -545,20 +545,23 @@ class InputCounter { bool Increment() { DCHECK_NE(count_.load(), total_.load()); int count = count_.fetch_add(1) + 1; - return IsComplete(count, total_.load()); + if (count != total_.load()) return false; + return DoneOnce(); } // return true if the counter is complete bool SetTotal(int total) { total_.store(total); - return IsComplete(count_.load(), total); + if (count_.load() != total) return false; + return DoneOnce(); } - private: - // ensure there is only one true return from Increment() or SetTotal() - bool IsComplete(int count, int total) { - if (count != total) return false; + // 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); } @@ -622,8 +625,12 @@ struct SinkNode : ExecNode { void ErrorReceived(ExecNode* input, Status error) override { DCHECK_EQ(input, inputs_[0]); + producer_.Push(std::move(error)); - Finish(); + + if (input_counter_.Cancel()) { + Finish(); + } inputs_[0]->StopProducing(this); } @@ -731,8 +738,9 @@ struct ScalarAggregateNode : ExecNode { if (ErrorIfNotOk(DoConsume(std::move(batch), thread_index))) return; - if (!input_counter_.Increment()) return; - if (ErrorIfNotOk(MaybeFinish())) return; + if (input_counter_.Increment()) { + ErrorIfNotOk(Finish()); + } } void ErrorReceived(ExecNode* input, Status error) override { @@ -743,8 +751,9 @@ struct ScalarAggregateNode : ExecNode { void InputFinished(ExecNode* input, int num_total) override { DCHECK_EQ(input, inputs_[0]); - if (!input_counter_.SetTotal(num_total)) return; - if (ErrorIfNotOk(MaybeFinish())) return; + if (input_counter_.SetTotal(num_total)) { + ErrorIfNotOk(Finish()); + } } Status StartProducing() override { @@ -764,21 +773,17 @@ struct ScalarAggregateNode : ExecNode { } void StopProducing() override { - std::unique_lock lock(mutex_); - if (states_.empty()) return; - states_.clear(); - lock.unlock(); - + if (input_counter_.Cancel()) { + finished_.MarkFinished(); + } inputs_[0]->StopProducing(this); - finished_.MarkFinished(); } Future<> finished() override { return finished_; } private: - Status MaybeFinish() { + Status Finish() { std::unique_lock lock(mutex_); - if (states_.empty()) return Status::OK(); ExecBatch batch{{}, 1}; batch.values.resize(kernels_.size()); @@ -789,7 +794,6 @@ struct ScalarAggregateNode : ExecNode { kernels_[i], &ctx, std::move(states_[i]))); RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); } - states_.clear(); lock.unlock(); @@ -898,7 +902,7 @@ struct GroupByNode : ExecNode { const char* kind_name() override { return "GroupByNode"; } - Status ProcessInputBatch(const ExecBatch& batch) { + Status Consume(ExecBatch batch) { ThreadLocalState* state = GetLocalState(); RETURN_NOT_OK(InitLocalStateIfNeeded(state)); @@ -1041,11 +1045,11 @@ struct GroupByNode : ExecNode { DCHECK_EQ(input, inputs_[0]); - if (ErrorIfNotOk(ProcessInputBatch(batch))) return; - - if (!input_counter_.Increment()) return; + if (ErrorIfNotOk(Consume(std::move(batch)))) return; - if (ErrorIfNotOk(OutputResult())) return; + if (input_counter_.Increment()) { + ErrorIfNotOk(OutputResult()); + } } void ErrorReceived(ExecNode* input, Status error) override { @@ -1057,9 +1061,9 @@ struct GroupByNode : ExecNode { void InputFinished(ExecNode* input, int num_total) override { DCHECK_EQ(input, inputs_[0]); - if (!input_counter_.SetTotal(num_total)) return; - - if (ErrorIfNotOk(OutputResult())) return; + if (input_counter_.SetTotal(num_total)) { + ErrorIfNotOk(OutputResult()); + } } Status StartProducing() override { @@ -1077,10 +1081,11 @@ struct GroupByNode : ExecNode { void StopProducing(ExecNode* output) override { DCHECK_EQ(output, outputs_[0]); - inputs_[0]->StopProducing(this); - // FIXME StopProducing must be idempotent, but we may call MarkFinished twice - finished_.MarkFinished(); + if (input_counter_.Cancel()) { + finished_.MarkFinished(); + } + inputs_[0]->StopProducing(this); } void StopProducing() override { StopProducing(outputs_[0]); } From 92319f7617e96bcdd128c085aced3e50b67afb73 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 09:08:48 -0400 Subject: [PATCH 32/41] remove unnecessary locking --- cpp/src/arrow/CMakeLists.txt | 1 + cpp/src/arrow/compute/exec/exec_plan.cc | 26 +++++++--------------- cpp/src/arrow/util/unreachable.cc | 29 +++++++++++++++++++++++++ cpp/src/arrow/util/unreachable.h | 22 +++++++++++++++++++ 4 files changed, 60 insertions(+), 18 deletions(-) create mode 100644 cpp/src/arrow/util/unreachable.cc create mode 100644 cpp/src/arrow/util/unreachable.h diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 88a92d8c2c9..bd2c2e1e3e9 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -218,6 +218,7 @@ set(ARROW_SRCS util/thread_pool.cc util/time.cc util/trie.cc + util/unreachable.cc util/uri.cc util/utf8.cc util/value_parsing.cc diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index b952489e883..7cfb69cafb9 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -39,6 +39,7 @@ #include "arrow/util/optional.h" #include "arrow/util/task_group.h" #include "arrow/util/thread_pool.h" +#include "arrow/util/unreachable.h" #include "arrow/util/vector.h" namespace arrow { @@ -263,8 +264,7 @@ struct SourceNode : ExecNode { const char* kind_name() override { return "SourceNode"; } [[noreturn]] static void NoInputs() { - DCHECK(false) << "no inputs; this should never be called"; - std::abort(); + Unreachable("no inputs; this should never be called"); } [[noreturn]] void InputReceived(ExecNode*, int, ExecBatch) override { NoInputs(); } [[noreturn]] void ErrorReceived(ExecNode*, Status) override { NoInputs(); } @@ -594,8 +594,7 @@ struct SinkNode : ExecNode { // sink nodes have no outputs from which to feel backpressure [[noreturn]] static void NoOutputs() { - DCHECK(false) << "no outputs; this should never be called"; - std::abort(); + Unreachable("no outputs; this should never be called"); } [[noreturn]] void ResumeProducing(ExecNode* output) override { NoOutputs(); } [[noreturn]] void PauseProducing(ExecNode* output) override { NoOutputs(); } @@ -690,18 +689,15 @@ std::shared_ptr MakeGeneratorReader( class ThreadIndexer { public: - size_t operator()(std::mutex* mutex) { - std::unique_lock lock(*mutex); - return this->operator()(); - } - size_t operator()() { + std::unique_lock lock(mutex_); auto id = std::this_thread::get_id(); const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; return id_index.second; } private: + std::mutex mutex_; std::unordered_map id_to_index_; }; @@ -734,7 +730,7 @@ struct ScalarAggregateNode : ExecNode { void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { DCHECK_EQ(input, inputs_[0]); - auto thread_index = get_thread_index_(&mutex_); + auto thread_index = get_thread_index_(); if (ErrorIfNotOk(DoConsume(std::move(batch), thread_index))) return; @@ -783,8 +779,6 @@ struct ScalarAggregateNode : ExecNode { private: Status Finish() { - std::unique_lock lock(mutex_); - ExecBatch batch{{}, 1}; batch.values.resize(kernels_.size()); @@ -795,9 +789,7 @@ struct ScalarAggregateNode : ExecNode { RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); } - lock.unlock(); - - outputs_[0]->InputReceived(this, 0, batch); + outputs_[0]->InputReceived(this, 0, std::move(batch)); finished_.MarkFinished(); return Status::OK(); } @@ -806,7 +798,6 @@ struct ScalarAggregateNode : ExecNode { std::vector kernels_; std::vector>> states_; - std::mutex mutex_; ThreadIndexer get_thread_index_; InputCounter input_counter_; @@ -1098,7 +1089,7 @@ struct GroupByNode : ExecNode { std::vector> agg_states; }; - ThreadLocalState* GetLocalState() { return &local_states_[get_thread_index_(&mutex_)]; } + ThreadLocalState* GetLocalState() { return &local_states_[get_thread_index_()]; } Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema @@ -1146,7 +1137,6 @@ struct GroupByNode : ExecNode { ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); - std::mutex mutex_; std::atomic num_output_batches_processed_{0}; const std::vector key_field_ids_; diff --git a/cpp/src/arrow/util/unreachable.cc b/cpp/src/arrow/util/unreachable.cc new file mode 100644 index 00000000000..4ffe3a8f787 --- /dev/null +++ b/cpp/src/arrow/util/unreachable.cc @@ -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. + +#include "arrow/util/unreachable.h" + +#include "arrow/util/logging.h" + +namespace arrow { + +[[noreturn]] void Unreachable(const char* message) { + DCHECK(false) << message; + std::abort(); +} + +} // namespace arrow diff --git a/cpp/src/arrow/util/unreachable.h b/cpp/src/arrow/util/unreachable.h new file mode 100644 index 00000000000..027f76e84d2 --- /dev/null +++ b/cpp/src/arrow/util/unreachable.h @@ -0,0 +1,22 @@ +// 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 { + +[[noreturn]] void Unreachable(const char* message = "Unreachable"); + +} // namespace arrow From 40223d8f4779de662b7ad27c224ff0a919f64c83 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 11:29:16 -0400 Subject: [PATCH 33/41] guard against hanging 0-groups case --- cpp/src/arrow/compute/exec/exec_plan.cc | 94 ++++++++++--------------- 1 file changed, 38 insertions(+), 56 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 7cfb69cafb9..c1059e6a1ca 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -529,9 +529,9 @@ Result MakeProjectNode(ExecNode* input, std::string label, input, std::move(label), schema(std::move(fields)), std::move(exprs)); } -class InputCounter { +class AtomicCounter { public: - InputCounter() = default; + AtomicCounter() = default; int count() const { return count_.load(); } @@ -646,7 +646,7 @@ struct SinkNode : ExecNode { } } - InputCounter input_counter_; + AtomicCounter input_counter_; Future<> finished_ = Future<>::MakeFinished(); PushGenerator>::Producer producer_; @@ -800,7 +800,7 @@ struct ScalarAggregateNode : ExecNode { std::vector>> states_; ThreadIndexer get_thread_index_; - InputCounter input_counter_; + AtomicCounter input_counter_; }; Result MakeScalarAggregateNode(ExecNode* input, std::string label, @@ -949,81 +949,61 @@ struct GroupByNode : ExecNode { return Status::OK(); } - Status Finalize() { - out_data_.resize(agg_kernels_.size() + key_field_ids_.size()); - auto it = out_data_.begin(); - + Result Finalize() { ThreadLocalState* state = &local_states_[0]; - num_out_groups_ = state->grouper->num_groups(); - // Aggregate fields come before key fields to match the behavior of GroupBy function + 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( + 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) { - // Check finished flag - if (finished_.is_finished()) { - return Status::OK(); - } + void OutputNthBatch(int n) { + // bail if StopProducing was called + if (finished_.is_finished()) return; - // Slice arrays int64_t batch_size = output_batch_size(); - int64_t batch_start = n * batch_size; - std::vector output_slices(out_data_.size()); - for (size_t i = 0; i < out_data_.size(); ++i) { - output_slices[i] = out_data_[i]->Slice(batch_start, batch_size); - } - - ARROW_ASSIGN_OR_RAISE(ExecBatch output_batch, ExecBatch::Make(output_slices)); - outputs_[0]->InputReceived(this, n, output_batch); + outputs_[0]->InputReceived(this, n, out_data_.Slice(batch_size * n, batch_size)); - uint32_t num_output_batches_processed = - 1 + num_output_batches_processed_.fetch_add(1); - if (num_output_batches_processed * batch_size >= num_out_groups_) { + if (output_counter_.Increment()) { finished_.MarkFinished(); } - - return Status::OK(); } Status OutputResult() { RETURN_NOT_OK(Merge()); - RETURN_NOT_OK(Finalize()); + ARROW_ASSIGN_OR_RAISE(out_data_, 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 num_output_batches = *output_counter_.total(); + outputs_[0]->InputFinished(this, num_output_batches); auto executor = ctx_->executor(); - for (int i = 0; i < num_result_batches; ++i) { - // bail if StopProducing was called - if (finished_.is_finished()) break; - + for (int i = 0; i < num_output_batches; ++i) { if (executor) { - RETURN_NOT_OK(executor->Spawn([this, i]() { - Status status = OutputNthBatch(i); - if (!status.ok()) { - ErrorReceived(inputs_[0], status); - } - })); + // bail if StopProducing was called + if (finished_.is_finished()) break; + + RETURN_NOT_OK(executor->Spawn([this, i] { OutputNthBatch(i); })); } else { - RETURN_NOT_OK(OutputNthBatch(i)); + OutputNthBatch(i); } } @@ -1050,6 +1030,9 @@ struct GroupByNode : ExecNode { } void InputFinished(ExecNode* input, int num_total) override { + // bail if StopProducing was called + if (finished_.is_finished()) return; + DCHECK_EQ(input, inputs_[0]); if (input_counter_.SetTotal(num_total)) { @@ -1075,6 +1058,8 @@ struct GroupByNode : ExecNode { if (input_counter_.Cancel()) { finished_.MarkFinished(); + } else if (output_counter_.Cancel()) { + finished_.MarkFinished(); } inputs_[0]->StopProducing(this); } @@ -1137,19 +1122,16 @@ struct GroupByNode : ExecNode { ExecContext* ctx_; Future<> finished_ = Future<>::MakeFinished(); - std::atomic num_output_batches_processed_{0}; - const std::vector key_field_ids_; const std::vector agg_src_field_ids_; const std::vector aggs_; const std::vector agg_kernels_; ThreadIndexer get_thread_index_; - InputCounter input_counter_; + AtomicCounter input_counter_, output_counter_; std::vector local_states_; - uint32_t num_out_groups_{0}; - ArrayDataVector out_data_; + ExecBatch out_data_; }; Result MakeGroupByNode(ExecNode* input, std::string label, From 2b21880b604ded0e12b1f34726dc783bcb274f77 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 11:55:42 -0400 Subject: [PATCH 34/41] paranoid assertions --- cpp/src/arrow/compute/exec/exec_plan.cc | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index c1059e6a1ca..1b526117619 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -716,7 +716,9 @@ struct ScalarAggregateNode : ExecNode { Status DoConsume(const ExecBatch& batch, size_t thread_index) { for (size_t i = 0; i < kernels_.size(); ++i) { - DCHECK_LT(thread_index, states_[i].size()); + DCHECK_LT(thread_index, states_[i].size()) + << "thread index " << thread_index << " is out of range [0, " + << states_[i].size() << ")"; KernelContext batch_ctx{plan()->exec_context()}; batch_ctx.SetState(states_[i][thread_index].get()); @@ -969,7 +971,7 @@ struct GroupByNode : ExecNode { state->grouper.reset(); if (output_counter_.SetTotal( - BitUtil::CeilDiv(out_data.length, output_batch_size()))) { + BitUtil::CeilDiv(static_cast(out_data.length), output_batch_size()))) { // this will be hit if out_data.length == 0 finished_.MarkFinished(); } @@ -1074,7 +1076,14 @@ struct GroupByNode : ExecNode { std::vector> agg_states; }; - ThreadLocalState* GetLocalState() { return &local_states_[get_thread_index_()]; } + ThreadLocalState* GetLocalState() { + size_t thread_index = get_thread_index_(); + DCHECK_LT(thread_index, local_states_.size()) + << "thread index " << thread_index << " is out of range [0, " + << local_states_.size() << ")"; + + return &local_states_[thread_index]; + } Status InitLocalStateIfNeeded(ThreadLocalState* state) { // Get input schema From 6cc8ad6652640444fa2ae4629d160a8f4500463f Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 12:16:45 -0400 Subject: [PATCH 35/41] msvc: explicit cast to int --- cpp/src/arrow/compute/exec/exec_plan.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 1b526117619..690feaf1c05 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -971,7 +971,7 @@ struct GroupByNode : ExecNode { state->grouper.reset(); if (output_counter_.SetTotal( - BitUtil::CeilDiv(static_cast(out_data.length), output_batch_size()))) { + static_cast(BitUtil::CeilDiv(out_data.length, output_batch_size())))) { // this will be hit if out_data.length == 0 finished_.MarkFinished(); } From 24ba7c004891d0894896ebc052d59a04fae67c5e Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 12:26:36 -0400 Subject: [PATCH 36/41] paranoid ~~assertions~~ IndexErrors --- ci/scripts/python_test.sh | 34 +++++++++++++++++++++++++ cpp/src/arrow/compute/exec/exec_plan.cc | 15 ++++++++--- 2 files changed, 45 insertions(+), 4 deletions(-) diff --git a/ci/scripts/python_test.sh b/ci/scripts/python_test.sh index 6e05af89a19..5dacca3a079 100755 --- a/ci/scripts/python_test.sh +++ b/ci/scripts/python_test.sh @@ -30,3 +30,37 @@ export LD_LIBRARY_PATH=${ARROW_HOME}/lib:${LD_LIBRARY_PATH} export PYTHONDEVMODE=1 pytest -r s -v ${PYTEST_ARGS} --pyargs pyarrow + +function print_coredumps() { + # The script expects core files relative to the current working directory, + # and the actual executable is a Python interpreter. So the corefile + # patterns must be set with prefix `core.python*`: + # + # In case of macOS: + # sudo sysctl -w kern.corefile=core.%N.%P + # On Linux: + # sudo sysctl -w kernel.core_pattern=core.%e.%p + # + # and the ulimit must be increased: + # ulimit -c unlimited + + PATTERN="^core\.python" + + COREFILES=$(ls | grep $PATTERN) + if [ -n "$COREFILES" ]; then + echo "Found core dump, printing backtrace:" + + for COREFILE in $COREFILES; do + # Print backtrace + if [ "$(uname)" == "Darwin" ]; then + lldb -c "${COREFILE}" --batch --one-line "thread backtrace all -e true" + else + gdb -c "${COREFILE}" $TEST_EXECUTABLE -ex "thread apply all bt" -ex "set pagination 0" -batch + fi + # Remove the coredump, regenerate it via running the test case directly + rm "${COREFILE}" + done + fi +} + +print_coredumps diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 690feaf1c05..a9f63bec598 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -716,9 +716,10 @@ struct ScalarAggregateNode : ExecNode { Status DoConsume(const ExecBatch& batch, size_t thread_index) { for (size_t i = 0; i < kernels_.size(); ++i) { - DCHECK_LT(thread_index, states_[i].size()) - << "thread index " << thread_index << " is out of range [0, " - << states_[i].size() << ")"; + if (thread_index >= states_[i].size()) { + return Status::IndexError("thread index ", thread_index, " is out of range [0, ", + states_[i].size(), ")"); + } KernelContext batch_ctx{plan()->exec_context()}; batch_ctx.SetState(states_[i][thread_index].get()); @@ -896,7 +897,13 @@ struct GroupByNode : ExecNode { const char* kind_name() override { return "GroupByNode"; } Status Consume(ExecBatch batch) { - ThreadLocalState* state = GetLocalState(); + size_t thread_index = get_thread_index_(); + if (thread_index >= local_states_.size()) { + return Status::IndexError("thread index ", thread_index, " is out of range [0, ", + local_states_.size(), ")"); + } + + auto state = &local_states_[thread_index]; RETURN_NOT_OK(InitLocalStateIfNeeded(state)); // Create a batch with key columns From 3c17e2afb3d299e918c1dd5302c59d546d75827e Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 12:55:10 -0400 Subject: [PATCH 37/41] Revert "try printing coredumps in python_test.sh" This reverts commit 92fb25e28e04efabe804dc5a0987aea87edb5a6f. --- ci/scripts/python_test.sh | 34 ---------------------------------- 1 file changed, 34 deletions(-) diff --git a/ci/scripts/python_test.sh b/ci/scripts/python_test.sh index 5dacca3a079..6e05af89a19 100755 --- a/ci/scripts/python_test.sh +++ b/ci/scripts/python_test.sh @@ -30,37 +30,3 @@ export LD_LIBRARY_PATH=${ARROW_HOME}/lib:${LD_LIBRARY_PATH} export PYTHONDEVMODE=1 pytest -r s -v ${PYTEST_ARGS} --pyargs pyarrow - -function print_coredumps() { - # The script expects core files relative to the current working directory, - # and the actual executable is a Python interpreter. So the corefile - # patterns must be set with prefix `core.python*`: - # - # In case of macOS: - # sudo sysctl -w kern.corefile=core.%N.%P - # On Linux: - # sudo sysctl -w kernel.core_pattern=core.%e.%p - # - # and the ulimit must be increased: - # ulimit -c unlimited - - PATTERN="^core\.python" - - COREFILES=$(ls | grep $PATTERN) - if [ -n "$COREFILES" ]; then - echo "Found core dump, printing backtrace:" - - for COREFILE in $COREFILES; do - # Print backtrace - if [ "$(uname)" == "Darwin" ]; then - lldb -c "${COREFILE}" --batch --one-line "thread backtrace all -e true" - else - gdb -c "${COREFILE}" $TEST_EXECUTABLE -ex "thread apply all bt" -ex "set pagination 0" -batch - fi - # Remove the coredump, regenerate it via running the test case directly - rm "${COREFILE}" - done - fi -} - -print_coredumps From 64ca1f13fb0eab6d036051752a83db60669679fc Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 13:20:06 -0400 Subject: [PATCH 38/41] overallocate state to always accommodate multiple threads --- cpp/src/arrow/compute/exec/exec_plan.cc | 33 ++++++++++++++----------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index a9f63bec598..6470e1be7e4 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -690,13 +690,27 @@ std::shared_ptr MakeGeneratorReader( class ThreadIndexer { public: size_t operator()() { - std::unique_lock lock(mutex_); 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 id_index.second; + + 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_; }; @@ -716,11 +730,6 @@ struct ScalarAggregateNode : ExecNode { Status DoConsume(const ExecBatch& batch, size_t thread_index) { for (size_t i = 0; i < kernels_.size(); ++i) { - if (thread_index >= states_[i].size()) { - return Status::IndexError("thread index ", thread_index, " is out of range [0, ", - states_[i].size(), ")"); - } - KernelContext batch_ctx{plan()->exec_context()}; batch_ctx.SetState(states_[i][thread_index].get()); @@ -839,7 +848,7 @@ Result MakeScalarAggregateNode(ExecNode* input, std::string label, } KernelContext kernel_ctx{exec_ctx}; - states[i].resize(exec_ctx->executor() ? exec_ctx->executor()->GetCapacity() : 1); + states[i].resize(ThreadIndexer::Capacity()); RETURN_NOT_OK(Kernel::InitAll(&kernel_ctx, KernelInitArgs{kernels[i], { @@ -1052,9 +1061,7 @@ struct GroupByNode : ExecNode { Status StartProducing() override { finished_ = Future<>::Make(); - auto executor = plan_->exec_context()->executor(); - local_states_.resize(executor ? executor->GetCapacity() : 1); - + local_states_.resize(ThreadIndexer::Capacity()); return Status::OK(); } @@ -1085,10 +1092,6 @@ struct GroupByNode : ExecNode { ThreadLocalState* GetLocalState() { size_t thread_index = get_thread_index_(); - DCHECK_LT(thread_index, local_states_.size()) - << "thread index " << thread_index << " is out of range [0, " - << local_states_.size() << ")"; - return &local_states_[thread_index]; } From 8b22540c5934d44668d00375d6de805628fcd712 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 13:50:41 -0400 Subject: [PATCH 39/41] don't redundantly resolve the output type --- cpp/src/arrow/compute/exec/exec_plan.cc | 4 ---- 1 file changed, 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 6470e1be7e4..20c8c347cc1 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -1123,10 +1123,6 @@ struct GroupByNode : ExecNode { 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(); } From a0e73643e24296fc3563af668a1045d5e4e4e2e2 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 15:55:49 -0400 Subject: [PATCH 40/41] Rely on FnOnce's releasing of resources in DoMarkFinishedOrFailed --- cpp/examples/arrow/compute_register_example.cc | 10 +++++----- cpp/src/arrow/util/future.cc | 1 - 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cpp/examples/arrow/compute_register_example.cc b/cpp/examples/arrow/compute_register_example.cc index d1a1372b82a..3c20a3d2a87 100644 --- a/cpp/examples/arrow/compute_register_example.cc +++ b/cpp/examples/arrow/compute_register_example.cc @@ -46,7 +46,7 @@ class ExampleFunctionOptionsType : public cp::FunctionOptionsType { } // optional: support for serialization // Result> Serialize(const FunctionOptions&) const override; - // Result> Deserialize(const Buffer& buffer) const override; + // Result> Deserialize(const Buffer&) const override; }; cp::FunctionOptionsType* GetExampleFunctionOptionsType() { @@ -74,8 +74,8 @@ const cp::FunctionDoc func_doc{ int main(int argc, char** argv) { const std::string name = "compute_register_example"; auto func = std::make_shared(name, cp::Arity::Unary(), &func_doc); - func->AddKernel({cp::InputType::Array(arrow::int64())}, arrow::int64(), - ExampleFunctionImpl); + ABORT_ON_FAILURE(func->AddKernel({cp::InputType::Array(arrow::int64())}, arrow::int64(), + ExampleFunctionImpl)); auto registry = cp::GetFunctionRegistry(); ABORT_ON_FAILURE(registry->AddFunction(std::move(func))); @@ -90,8 +90,8 @@ int main(int argc, char** argv) { std::cout << maybe_result->make_array()->ToString() << std::endl; - // Expression serialization will raise NotImplemented if an expression includes FunctionOptions - // for which serialization is not supported. + // Expression serialization will raise NotImplemented if an expression includes + // FunctionOptions for which serialization is not supported. auto expr = cp::call(name, {}, options); auto maybe_serialized = cp::Serialize(expr); std::cerr << maybe_serialized.status().ToString() << std::endl; diff --git a/cpp/src/arrow/util/future.cc b/cpp/src/arrow/util/future.cc index f288a15be3f..3f5a3bbf82c 100644 --- a/cpp/src/arrow/util/future.cc +++ b/cpp/src/arrow/util/future.cc @@ -320,7 +320,6 @@ class ConcreteFutureImpl : public FutureImpl { for (auto& callback_record : callbacks_) { RunOrScheduleCallback(std::move(callback_record), /*in_add_callback=*/false); } - callbacks_.clear(); } void DoWait() { From 5eaf8e9647ecf34cef80db94b6688db9b95b8b04 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 22 Jul 2021 16:28:41 -0400 Subject: [PATCH 41/41] ensure DoMarkFinishedOrFailed tolerates deletion of the called future --- cpp/src/arrow/util/future.cc | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/util/future.cc b/cpp/src/arrow/util/future.cc index 3f5a3bbf82c..fc8022a95e4 100644 --- a/cpp/src/arrow/util/future.cc +++ b/cpp/src/arrow/util/future.cc @@ -245,7 +245,8 @@ class ConcreteFutureImpl : public FutureImpl { CallbackRecord callback_record{std::move(callback), opts}; if (IsFutureFinished(state_)) { lock.unlock(); - RunOrScheduleCallback(std::move(callback_record), /*in_add_callback=*/true); + RunOrScheduleCallback(shared_from_this(), std::move(callback_record), + /*in_add_callback=*/true); } else { callbacks_.push_back(std::move(callback_record)); } @@ -263,8 +264,8 @@ class ConcreteFutureImpl : public FutureImpl { } } - bool ShouldScheduleCallback(const CallbackRecord& callback_record, - bool in_add_callback) { + static bool ShouldScheduleCallback(const CallbackRecord& callback_record, + bool in_add_callback) { switch (callback_record.options.should_schedule) { case ShouldSchedule::Never: return false; @@ -280,7 +281,9 @@ class ConcreteFutureImpl : public FutureImpl { } } - void RunOrScheduleCallback(CallbackRecord&& callback_record, bool in_add_callback) { + static void RunOrScheduleCallback(const std::shared_ptr& self, + CallbackRecord&& callback_record, + bool in_add_callback) { if (ShouldScheduleCallback(callback_record, in_add_callback)) { struct CallbackTask { void operator()() { std::move(callback)(*self); } @@ -289,10 +292,10 @@ class ConcreteFutureImpl : public FutureImpl { std::shared_ptr self; }; // Need to keep `this` alive until the callback has a chance to be scheduled. - CallbackTask task{std::move(callback_record.callback), shared_from_this()}; + CallbackTask task{std::move(callback_record.callback), self}; DCHECK_OK(callback_record.options.executor->Spawn(std::move(task))); } else { - std::move(callback_record.callback)(*this); + std::move(callback_record.callback)(*self); } } @@ -311,14 +314,17 @@ class ConcreteFutureImpl : public FutureImpl { } cv_.notify_all(); + auto callbacks = std::move(callbacks_); + auto self = shared_from_this(); + // run callbacks, lock not needed since the future is finished by this // point so nothing else can modify the callbacks list and it is safe // to iterate. // // In fact, it is important not to hold the locks because the callback // may be slow or do its own locking on other resources - for (auto& callback_record : callbacks_) { - RunOrScheduleCallback(std::move(callback_record), /*in_add_callback=*/false); + for (auto& callback_record : callbacks) { + RunOrScheduleCallback(self, std::move(callback_record), /*in_add_callback=*/false); } }