From 193f29affdaaa47fdf210387e06a1fb7d1da6040 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 23 Jul 2021 11:28:28 -0400 Subject: [PATCH 1/8] Refactor away from hard coded ExecNode factories to a registry Added factory for SourceExecNode and replacing old MakeProjectNode by the new registry use in plan_test Formatting.. add Declaration helper class to allow declarative construction of ExecPlans msvc: export classes and structs add operator| for dplyr-like pipelines make labels optional for ExecNodes remove operator| add example of registering a custom ExecNode msvc: try explicitly constructing std::function instead of relying on operator= remove usage of hard coded factories from plan_test.cc remove all usages of hard coded factory functions nit: simplify Declaration::Sequence slightly msvc: it's callable, really msvc: export nested structs too msvc: try the copy constructor, why not add comments for Declaration add support for AggregateNodeOptions::targets to ScalarAggregateNode revert msvc: try the copy constructor, why not extract exec node implementations to dedicated source files add output field names to aggregate options --- .../arrow/compute_register_example.cc | 62 + cpp/src/arrow/CMakeLists.txt | 5 + cpp/src/arrow/compute/exec/aggregate_node.cc | 626 ++++++++++ cpp/src/arrow/compute/exec/exec_plan.cc | 1070 +---------------- cpp/src/arrow/compute/exec/exec_plan.h | 168 ++- cpp/src/arrow/compute/exec/filter_node.cc | 142 +++ cpp/src/arrow/compute/exec/options.h | 115 ++ cpp/src/arrow/compute/exec/plan_test.cc | 283 +++-- cpp/src/arrow/compute/exec/project_node.cc | 133 ++ cpp/src/arrow/compute/exec/sink_node.cc | 140 +++ cpp/src/arrow/compute/exec/source_node.cc | 147 +++ cpp/src/arrow/compute/exec/test_util.cc | 13 +- cpp/src/arrow/compute/exec/util.cc | 22 + cpp/src/arrow/compute/exec/util.h | 51 + .../compute/kernels/hash_aggregate_test.cc | 92 ++ cpp/src/arrow/compute/type_fwd.h | 2 + cpp/src/arrow/dataset/dataset_internal.h | 14 + cpp/src/arrow/dataset/scanner.cc | 203 ++-- cpp/src/arrow/dataset/scanner.h | 23 +- cpp/src/arrow/dataset/scanner_test.cc | 162 ++- cpp/src/arrow/util/unreachable.h | 2 + 21 files changed, 2108 insertions(+), 1367 deletions(-) create mode 100644 cpp/src/arrow/compute/exec/aggregate_node.cc create mode 100644 cpp/src/arrow/compute/exec/filter_node.cc create mode 100644 cpp/src/arrow/compute/exec/options.h create mode 100644 cpp/src/arrow/compute/exec/project_node.cc create mode 100644 cpp/src/arrow/compute/exec/sink_node.cc create mode 100644 cpp/src/arrow/compute/exec/source_node.cc diff --git a/cpp/examples/arrow/compute_register_example.cc b/cpp/examples/arrow/compute_register_example.cc index 3c20a3d2a87..a4a05f81bc5 100644 --- a/cpp/examples/arrow/compute_register_example.cc +++ b/cpp/examples/arrow/compute_register_example.cc @@ -17,7 +17,11 @@ #include #include +#include #include +#include +#include +#include #include #include @@ -65,6 +69,45 @@ arrow::Status ExampleFunctionImpl(cp::KernelContext* ctx, const cp::ExecBatch& b return arrow::Status::OK(); } +class ExampleNodeOptions : public cp::ExecNodeOptions {}; + +// a basic ExecNode which ignores all input batches +class ExampleNode : public cp::ExecNode { + public: + ExampleNode(ExecNode* input, const ExampleNodeOptions&) + : ExecNode(/*plan=*/input->plan(), /*inputs=*/{input}, + /*input_labels=*/{"ignored"}, + /*output_schema=*/input->output_schema(), /*num_outputs=*/1) {} + + const char* kind_name() override { return "ExampleNode"; } + + arrow::Status StartProducing() override { + outputs_[0]->InputFinished(this, 0); + return arrow::Status::OK(); + } + + void ResumeProducing(ExecNode* output) override {} + void PauseProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { inputs_[0]->StopProducing(this); } + void StopProducing() override { inputs_[0]->StopProducing(); } + + void InputReceived(ExecNode* input, int seq_num, cp::ExecBatch batch) override {} + void ErrorReceived(ExecNode* input, arrow::Status error) override {} + void InputFinished(ExecNode* input, int seq_stop) override {} + + arrow::Future<> finished() override { return inputs_[0]->finished(); } +}; + +arrow::Result ExampleExecNodeFactory(cp::ExecPlan* plan, + std::vector inputs, + const cp::ExecNodeOptions& options) { + const auto& example_options = + arrow::internal::checked_cast(options); + + return plan->EmplaceNode(inputs[0], example_options); +} + const cp::FunctionDoc func_doc{ "Example function to demonstrate registering an out-of-tree function", "", @@ -96,5 +139,24 @@ int main(int argc, char** argv) { auto maybe_serialized = cp::Serialize(expr); std::cerr << maybe_serialized.status().ToString() << std::endl; + auto exec_registry = cp::default_exec_factory_registry(); + ABORT_ON_FAILURE( + exec_registry->AddFactory("compute_register_example", ExampleExecNodeFactory)); + + auto maybe_plan = cp::ExecPlan::Make(); + ABORT_ON_FAILURE(maybe_plan.status()); + auto plan = maybe_plan.ValueOrDie(); + + arrow::AsyncGenerator> source_gen, sink_gen; + ABORT_ON_FAILURE( + cp::Declaration::Sequence( + { + {"source", cp::SourceNodeOptions{arrow::schema({}), source_gen}}, + {"compute_register_example", ExampleNodeOptions{}}, + {"sink", cp::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get()) + .status()); + return EXIT_SUCCESS; } diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index f13e5b1ef75..308ee49972c 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -370,8 +370,13 @@ if(ARROW_COMPUTE) compute/api_vector.cc compute/cast.cc compute/exec.cc + compute/exec/aggregate_node.cc compute/exec/exec_plan.cc compute/exec/expression.cc + compute/exec/filter_node.cc + compute/exec/project_node.cc + compute/exec/source_node.cc + compute/exec/sink_node.cc compute/function.cc compute/function_internal.cc compute/kernel.cc diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc new file mode 100644 index 00000000000..79444dc575c --- /dev/null +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -0,0 +1,626 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/exec_plan.h" + +#include +#include +#include + +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/exec_internal.h" +#include "arrow/compute/registry.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/logging.h" +#include "arrow/util/thread_pool.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { + +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 + +namespace { + +class ThreadIndexer { + public: + size_t operator()() { + auto id = std::this_thread::get_id(); + + std::unique_lock lock(mutex_); + const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; + + return Check(id_index.second); + } + + static size_t Capacity() { + static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); + return max_size; + } + + private: + size_t Check(size_t thread_index) { + DCHECK_LT(thread_index, Capacity()) << "thread index " << thread_index + << " is out of range [0, " << Capacity() << ")"; + + return thread_index; + } + + std::mutex mutex_; + std::unordered_map id_to_index_; +}; + +struct ScalarAggregateNode : ExecNode { + ScalarAggregateNode(ExecPlan* plan, std::vector inputs, + std::shared_ptr output_schema, + std::vector target_field_ids, + std::vector kernels, + std::vector>> states) + : ExecNode(plan, std::move(inputs), {"target"}, + /*output_schema=*/std::move(output_schema), + /*num_outputs=*/1), + target_field_ids_(std::move(target_field_ids)), + kernels_(std::move(kernels)), + states_(std::move(states)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode")); + + const auto& aggregate_options = checked_cast(options); + auto aggregates = aggregate_options.aggregates; + + const auto& input_schema = *inputs[0]->output_schema(); + auto exec_ctx = plan->exec_context(); + + std::vector kernels(aggregates.size()); + std::vector>> states(kernels.size()); + FieldVector fields(kernels.size()); + const auto& field_names = aggregate_options.names; + std::vector target_field_ids(kernels.size()); + + for (size_t i = 0; i < kernels.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, + FieldRef(aggregate_options.targets[i]).FindOne(input_schema)); + target_field_ids[i] = match[0]; + + ARROW_ASSIGN_OR_RAISE( + auto function, exec_ctx->func_registry()->GetFunction(aggregates[i].function)); + + if (function->kind() != Function::SCALAR_AGGREGATE) { + return Status::Invalid("Provided non ScalarAggregateFunction ", + aggregates[i].function); + } + + auto in_type = ValueDescr::Array(input_schema.field(target_field_ids[i])->type()); + + ARROW_ASSIGN_OR_RAISE(const Kernel* kernel, function->DispatchExact({in_type})); + kernels[i] = static_cast(kernel); + + if (aggregates[i].options == nullptr) { + aggregates[i].options = function->default_options(); + } + + KernelContext kernel_ctx{exec_ctx}; + states[i].resize(ThreadIndexer::Capacity()); + RETURN_NOT_OK(Kernel::InitAll(&kernel_ctx, + KernelInitArgs{kernels[i], + { + in_type, + }, + aggregates[i].options}, + &states[i])); + + // pick one to resolve the kernel signature + kernel_ctx.SetState(states[i][0].get()); + ARROW_ASSIGN_OR_RAISE( + auto descr, kernels[i]->signature->out_type().Resolve(&kernel_ctx, {in_type})); + + fields[i] = field(field_names[i], std::move(descr.type)); + } + + return plan->EmplaceNode( + plan, std::move(inputs), schema(std::move(fields)), std::move(target_field_ids), + std::move(kernels), std::move(states)); + } + + const char* kind_name() override { return "ScalarAggregateNode"; } + + Status DoConsume(const ExecBatch& batch, size_t thread_index) { + for (size_t i = 0; i < kernels_.size(); ++i) { + KernelContext batch_ctx{plan()->exec_context()}; + batch_ctx.SetState(states_[i][thread_index].get()); + + ExecBatch single_column_batch{{batch.values[target_field_ids_[i]]}, batch.length}; + RETURN_NOT_OK(kernels_[i]->consume(&batch_ctx, single_column_batch)); + } + return Status::OK(); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + auto thread_index = get_thread_index_(); + + if (ErrorIfNotOk(DoConsume(std::move(batch), thread_index))) return; + + if (input_counter_.Increment()) { + ErrorIfNotOk(Finish()); + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + outputs_[0]->ErrorReceived(this, std::move(error)); + } + + void InputFinished(ExecNode* input, int num_total) override { + DCHECK_EQ(input, inputs_[0]); + + if (input_counter_.SetTotal(num_total)) { + ErrorIfNotOk(Finish()); + } + } + + Status StartProducing() override { + finished_ = Future<>::Make(); + // Scalar aggregates will only output a single batch + outputs_[0]->InputFinished(this, 1); + 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 { + if (input_counter_.Cancel()) { + finished_.MarkFinished(); + } + inputs_[0]->StopProducing(this); + } + + Future<> finished() override { return finished_; } + + private: + Status Finish() { + ExecBatch batch{{}, 1}; + batch.values.resize(kernels_.size()); + + for (size_t i = 0; i < kernels_.size(); ++i) { + KernelContext ctx{plan()->exec_context()}; + ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll( + kernels_[i], &ctx, std::move(states_[i]))); + RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); + } + + outputs_[0]->InputReceived(this, 0, std::move(batch)); + finished_.MarkFinished(); + return Status::OK(); + } + + Future<> finished_ = Future<>::MakeFinished(); + const std::vector target_field_ids_; + const std::vector kernels_; + + std::vector>> states_; + + ThreadIndexer get_thread_index_; + AtomicCounter input_counter_; +}; + +struct GroupByNode : ExecNode { + GroupByNode(ExecNode* input, std::shared_ptr output_schema, ExecContext* ctx, + const std::vector&& key_field_ids, + const std::vector&& agg_src_field_ids, + const std::vector&& aggs, + const std::vector&& agg_kernels) + : ExecNode(input->plan(), {input}, {"groupby"}, std::move(output_schema), + /*num_outputs=*/1), + ctx_(ctx), + key_field_ids_(std::move(key_field_ids)), + agg_src_field_ids_(std::move(agg_src_field_ids)), + aggs_(std::move(aggs)), + agg_kernels_(std::move(agg_kernels)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "GroupByNode")); + + auto input = inputs[0]; + const auto& aggregate_options = checked_cast(options); + const auto& keys = aggregate_options.keys; + const auto& aggs = aggregate_options.aggregates; + const auto& field_names = aggregate_options.names; + + // 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) { + ARROW_ASSIGN_OR_RAISE(auto match, keys[i].FindOne(*input_schema)); + key_field_ids[i] = match[0]; + } + + // Find input field indices for aggregates + std::vector agg_src_field_ids(aggs.size()); + for (size_t i = 0; i < aggs.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, + aggregate_options.targets[i].FindOne(*input_schema)); + agg_src_field_ids[i] = match[0]; + } + + // Build vector of aggregate source field data types + DCHECK_EQ(aggregate_options.targets.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(), ValueDescr::ARRAY); + } + + auto ctx = input->plan()->exec_context(); + + // 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]->WithName(field_names[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); + } + + auto aggs_copy = aggs; + + return input->plan()->EmplaceNode( + input, schema(std::move(output_fields)), ctx, std::move(key_field_ids), + std::move(agg_src_field_ids), std::move(aggs), std::move(agg_kernels)); + } + + const char* kind_name() override { return "GroupByNode"; } + + Status Consume(ExecBatch batch) { + 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 + std::vector keys(key_field_ids_.size()); + for (size_t i = 0; i < key_field_ids_.size(); ++i) { + keys[i] = batch.values[key_field_ids_[i]]; + } + ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); + + // Create a batch with group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); + + // Execute aggregate kernels + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext kernel_ctx{ctx_}; + kernel_ctx.SetState(state->agg_states[i].get()); + + ARROW_ASSIGN_OR_RAISE( + auto agg_batch, + ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); + + RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); + RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); + } + + return Status::OK(); + } + + Status Merge() { + ThreadLocalState* state0 = &local_states_[0]; + for (size_t i = 1; i < local_states_.size(); ++i) { + ThreadLocalState* state = &local_states_[i]; + 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(); + + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + 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(); + } + + Result Finalize() { + ThreadLocalState* state = &local_states_[0]; + + ExecBatch out_data{{}, state->grouper->num_groups()}; + out_data.values.resize(agg_kernels_.size() + key_field_ids_.size()); + + // Aggregate fields come before key fields to match the behavior of GroupBy function + for (size_t i = 0; i < agg_kernels_.size(); ++i) { + KernelContext batch_ctx{ctx_}; + batch_ctx.SetState(state->agg_states[i].get()); + RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out_data.values[i])); + state->agg_states[i].reset(); + } + + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); + std::move(out_keys.values.begin(), out_keys.values.end(), + out_data.values.begin() + agg_kernels_.size()); + state->grouper.reset(); + + if (output_counter_.SetTotal( + static_cast(BitUtil::CeilDiv(out_data.length, output_batch_size())))) { + // this will be hit if out_data.length == 0 + finished_.MarkFinished(); + } + return out_data; + } + + void OutputNthBatch(int n) { + // bail if StopProducing was called + if (finished_.is_finished()) return; + + int64_t batch_size = output_batch_size(); + outputs_[0]->InputReceived(this, n, out_data_.Slice(batch_size * n, batch_size)); + + if (output_counter_.Increment()) { + finished_.MarkFinished(); + } + } + + Status OutputResult() { + RETURN_NOT_OK(Merge()); + ARROW_ASSIGN_OR_RAISE(out_data_, Finalize()); + + int num_output_batches = *output_counter_.total(); + outputs_[0]->InputFinished(this, num_output_batches); + + auto executor = ctx_->executor(); + for (int i = 0; i < num_output_batches; ++i) { + if (executor) { + // bail if StopProducing was called + if (finished_.is_finished()) break; + + RETURN_NOT_OK(executor->Spawn([this, i] { OutputNthBatch(i); })); + } else { + OutputNthBatch(i); + } + } + + return Status::OK(); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + // bail if StopProducing was called + if (finished_.is_finished()) return; + + DCHECK_EQ(input, inputs_[0]); + + if (ErrorIfNotOk(Consume(std::move(batch)))) return; + + if (input_counter_.Increment()) { + ErrorIfNotOk(OutputResult()); + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + + outputs_[0]->ErrorReceived(this, std::move(error)); + } + + 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)) { + ErrorIfNotOk(OutputResult()); + } + } + + Status StartProducing() override { + finished_ = Future<>::Make(); + + local_states_.resize(ThreadIndexer::Capacity()); + return Status::OK(); + } + + void PauseProducing(ExecNode* output) override {} + + void ResumeProducing(ExecNode* output) override {} + + void StopProducing(ExecNode* output) override { + DCHECK_EQ(output, outputs_[0]); + + if (input_counter_.Cancel()) { + finished_.MarkFinished(); + } else if (output_counter_.Cancel()) { + finished_.MarkFinished(); + } + inputs_[0]->StopProducing(this); + } + + void StopProducing() override { StopProducing(outputs_[0]); } + + Future<> finished() override { return finished_; } + + private: + struct ThreadLocalState { + std::unique_ptr grouper; + std::vector> agg_states; + }; + + ThreadLocalState* GetLocalState() { + size_t thread_index = get_thread_index_(); + return &local_states_[thread_index]; + } + + 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)); + + return Status::OK(); + } + + 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(); + + 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_; + AtomicCounter input_counter_, output_counter_; + + std::vector local_states_; + ExecBatch out_data_; +}; + +class DefaultExecFactoryRegistry : public ExecFactoryRegistry { + public: + Result GetFactory(const std::string& factory_name) override { + auto it = factories_.find(factory_name); + if (it == factories_.end()) { + return Status::KeyError("ExecNode factory named ", factory_name, + " not present in registry."); + } + return it->second; + } + + Status AddFactory(std::string factory_name, Factory factory) override { + auto it_success = factories_.emplace(std::move(factory_name), std::move(factory)); + + if (!it_success.second) { + const auto& factory_name = it_success.first->first; + return Status::KeyError("ExecNode factory named ", factory_name, + " already registered."); + } + + return Status::OK(); + } + + private: + std::unordered_map factories_; +}; + +ExecFactoryRegistry::AddOnLoad kRegisterAggregate( + "aggregate", + [](ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) -> Result { + const auto& aggregate_options = checked_cast(options); + + if (aggregate_options.keys.empty()) { + // construct scalar agg node + return ScalarAggregateNode::Make(plan, std::move(inputs), options); + } + return GroupByNode::Make(plan, std::move(inputs), options); + }); + +} // namespace +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 4a4758c8471..5047c7a58d6 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -17,14 +17,9 @@ #include "arrow/compute/exec/exec_plan.h" -#include -#include #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" @@ -33,20 +28,13 @@ #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/task_group.h" -#include "arrow/util/thread_pool.h" -#include "arrow/util/unreachable.h" -#include "arrow/util/vector.h" namespace arrow { -using BitUtil::CountLeadingZeros; using internal::checked_cast; -using internal::checked_pointer_cast; namespace compute { @@ -209,11 +197,10 @@ void ExecPlan::StopProducing() { ToDerived(this)->StopProducing(); } Future<> ExecPlan::finished() { return ToDerived(this)->finished_; } -ExecNode::ExecNode(ExecPlan* plan, std::string label, NodeVector inputs, +ExecNode::ExecNode(ExecPlan* plan, NodeVector inputs, std::vector input_labels, std::shared_ptr output_schema, int num_outputs) : plan_(plan), - label_(std::move(label)), inputs_(std::move(inputs)), input_labels_(std::move(input_labels)), output_schema_(std::move(output_schema)), @@ -254,409 +241,13 @@ bool ExecNode::ErrorIfNotOk(Status status) { return true; } -struct SourceNode : ExecNode { - SourceNode(ExecPlan* plan, std::string label, std::shared_ptr output_schema, - AsyncGenerator> generator) - : ExecNode(plan, std::move(label), {}, {}, std::move(output_schema), - /*num_outputs=*/1), - generator_(std::move(generator)) {} +ExecFactoryRegistry::AddOnLoad::AddOnLoad(std::string factory_name, Factory factory) + : AddOnLoad(std::move(factory_name), std::move(factory), + default_exec_factory_registry()) {} - const char* kind_name() override { return "SourceNode"; } - - [[noreturn]] static void NoInputs() { - Unreachable("no inputs; this should never be called"); - } - [[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 SourceNode"; - - CallbackOptions options; - if (auto executor = plan()->exec_context()->executor()) { - // These options will transfer execution to the desired Executor if necessary. - // This can happen for in-memory scans where batches didn't require - // any CPU work to decode. Otherwise, parsing etc should have already - // been placed us on the desired Executor and no queues will be pushed to. - options.executor = executor; - options.should_schedule = ShouldSchedule::IfDifferentExecutor; - } - - finished_ = Loop([this, options] { - std::unique_lock lock(mutex_); - int seq = batch_count_++; - if (stop_requested_) { - return Future>::MakeFinished(Break(seq)); - } - lock.unlock(); - - return generator_().Then( - [=](const util::optional& batch) -> ControlFlow { - std::unique_lock lock(mutex_); - if (IsIterationEnd(batch) || stop_requested_) { - stop_requested_ = true; - return Break(seq); - } - lock.unlock(); - - outputs_[0]->InputReceived(this, seq, *batch); - return Continue(); - }, - [=](const Status& error) -> ControlFlow { - // NB: ErrorReceived is independent of InputFinished, but - // ErrorReceived will usually prompt StopProducing which will - // prompt InputFinished. ErrorReceived may still be called from a - // node which was requested to stop (indeed, the request to stop - // may prompt an error). - std::unique_lock lock(mutex_); - stop_requested_ = true; - lock.unlock(); - outputs_[0]->ErrorReceived(this, error); - return Break(seq); - }, - options); - }).Then([&](int seq) { outputs_[0]->InputFinished(this, seq); }); - - 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: - std::mutex mutex_; - bool stop_requested_{false}; - int batch_count_{0}; - Future<> finished_ = Future<>::MakeFinished(); - AsyncGenerator> generator_; -}; - -ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, - std::shared_ptr output_schema, - AsyncGenerator> generator) { - return plan->EmplaceNode(plan, std::move(label), std::move(output_schema), - std::move(generator)); -} - -struct FilterNode : ExecNode { - FilterNode(ExecNode* input, std::string label, Expression filter) - : ExecNode(input->plan(), std::move(label), {input}, {"target"}, - /*output_schema=*/input->output_schema(), - /*num_outputs=*/1), - filter_(std::move(filter)) {} - - const char* kind_name() override { return "FilterNode"; } - - Result DoFilter(const ExecBatch& target) { - ARROW_ASSIGN_OR_RAISE(Expression simplified_filter, - SimplifyWithGuarantee(filter_, target.guarantee)); - - ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target, - plan()->exec_context())); - - if (mask.is_scalar()) { - const auto& mask_scalar = mask.scalar_as(); - if (mask_scalar.is_valid && mask_scalar.value) { - return target; - } - - return target.Slice(0, 0); - } - - // if the values are all scalar then the mask must also be - DCHECK(!std::all_of(target.values.begin(), target.values.end(), - [](const Datum& value) { return value.is_scalar(); })); - - auto values = target.values; - for (auto& value : values) { - if (value.is_scalar()) continue; - ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults())); - } - return ExecBatch::Make(std::move(values)); - } - - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - DCHECK_EQ(input, inputs_[0]); - - auto maybe_filtered = DoFilter(std::move(batch)); - if (ErrorIfNotOk(maybe_filtered.status())) return; - - maybe_filtered->guarantee = batch.guarantee; - outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe()); - } - - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); - outputs_[0]->ErrorReceived(this, std::move(error)); - } - - void InputFinished(ExecNode* input, int seq) override { - DCHECK_EQ(input, inputs_[0]); - outputs_[0]->InputFinished(this, 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]); - StopProducing(); - } - - void StopProducing() override { inputs_[0]->StopProducing(this); } - - Future<> finished() override { return inputs_[0]->finished(); } - - private: - Expression filter_; -}; - -Result MakeFilterNode(ExecNode* input, std::string label, Expression filter) { - if (!filter.IsBound()) { - ARROW_ASSIGN_OR_RAISE(filter, filter.Bind(*input->output_schema())); - } - - if (filter.type()->id() != Type::BOOL) { - return Status::TypeError("Filter expression must evaluate to bool, but ", - filter.ToString(), " evaluates to ", - filter.type()->ToString()); - } - - return input->plan()->EmplaceNode(input, std::move(label), - std::move(filter)); -} - -struct ProjectNode : ExecNode { - ProjectNode(ExecNode* input, std::string label, std::shared_ptr output_schema, - std::vector exprs) - : ExecNode(input->plan(), std::move(label), {input}, {"target"}, - /*output_schema=*/std::move(output_schema), - /*num_outputs=*/1), - exprs_(std::move(exprs)) {} - - const char* kind_name() override { return "ProjectNode"; } - - Result DoProject(const ExecBatch& target) { - std::vector values{exprs_.size()}; - for (size_t i = 0; i < exprs_.size(); ++i) { - ARROW_ASSIGN_OR_RAISE(Expression simplified_expr, - SimplifyWithGuarantee(exprs_[i], target.guarantee)); - - ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target, - plan()->exec_context())); - } - return ExecBatch{std::move(values), target.length}; - } - - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - DCHECK_EQ(input, inputs_[0]); - - auto maybe_projected = DoProject(std::move(batch)); - if (ErrorIfNotOk(maybe_projected.status())) return; - - maybe_projected->guarantee = batch.guarantee; - outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe()); - } - - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); - outputs_[0]->ErrorReceived(this, std::move(error)); - } - - void InputFinished(ExecNode* input, int seq) override { - DCHECK_EQ(input, inputs_[0]); - outputs_[0]->InputFinished(this, 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]); - StopProducing(); - } - - void StopProducing() override { inputs_[0]->StopProducing(this); } - - Future<> finished() override { return inputs_[0]->finished(); } - - private: - std::vector exprs_; -}; - -Result MakeProjectNode(ExecNode* input, std::string label, - std::vector exprs, - std::vector names) { - FieldVector fields(exprs.size()); - - if (names.size() == 0) { - names.resize(exprs.size()); - for (size_t i = 0; i < exprs.size(); ++i) { - names[i] = exprs[i].ToString(); - } - } - - int i = 0; - for (auto& expr : exprs) { - if (!expr.IsBound()) { - ARROW_ASSIGN_OR_RAISE(expr, expr.Bind(*input->output_schema())); - } - fields[i] = field(std::move(names[i]), expr.type()); - ++i; - } - - return input->plan()->EmplaceNode( - input, std::move(label), schema(std::move(fields)), std::move(exprs)); -} - -class AtomicCounter { - public: - AtomicCounter() = default; - - int count() const { return count_.load(); } - - util::optional total() const { - int total = total_.load(); - if (total == -1) return {}; - return total; - } - - // return true if the counter is complete - bool Increment() { - DCHECK_NE(count_.load(), total_.load()); - int count = count_.fetch_add(1) + 1; - if (count != total_.load()) return false; - return DoneOnce(); - } - - // return true if the counter is complete - bool SetTotal(int total) { - total_.store(total); - if (count_.load() != total) return false; - return DoneOnce(); - } - - // return true if the counter has not already been completed - bool Cancel() { return DoneOnce(); } - - private: - // ensure there is only one true return from Increment(), SetTotal(), or Cancel() - bool DoneOnce() { - bool expected = false; - return complete_.compare_exchange_strong(expected, true); - } - - std::atomic count_{0}, total_{-1}; - std::atomic complete_{false}; -}; - -struct SinkNode : ExecNode { - SinkNode(ExecNode* input, std::string label, - AsyncGenerator>* generator) - : ExecNode(input->plan(), std::move(label), {input}, {"collected"}, {}, - /*num_outputs=*/0), - producer_(MakeProducer(generator)) {} - - static PushGenerator>::Producer MakeProducer( - AsyncGenerator>* out_gen) { - PushGenerator> gen; - auto out = gen.producer(); - *out_gen = std::move(gen); - return out; - } - - const char* kind_name() override { return "SinkNode"; } - - Status StartProducing() override { - finished_ = Future<>::Make(); - return Status::OK(); - } - - // sink nodes have no outputs from which to feel backpressure - [[noreturn]] static void NoOutputs() { - Unreachable("no outputs; this should never be called"); - } - [[noreturn]] void ResumeProducing(ExecNode* output) override { NoOutputs(); } - [[noreturn]] void PauseProducing(ExecNode* output) override { NoOutputs(); } - [[noreturn]] void StopProducing(ExecNode* output) override { NoOutputs(); } - - void StopProducing() override { - Finish(); - inputs_[0]->StopProducing(this); - } - - Future<> finished() override { return finished_; } - - void InputReceived(ExecNode* input, int seq_num, ExecBatch batch) override { - DCHECK_EQ(input, inputs_[0]); - - bool did_push = producer_.Push(std::move(batch)); - if (!did_push) return; // producer_ was Closed already - - if (auto total = input_counter_.total()) { - DCHECK_LE(seq_num, *total); - } - - if (input_counter_.Increment()) { - Finish(); - } - } - - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); - - producer_.Push(std::move(error)); - - if (input_counter_.Cancel()) { - Finish(); - } - inputs_[0]->StopProducing(this); - } - - void InputFinished(ExecNode* input, int seq_stop) override { - if (input_counter_.SetTotal(seq_stop)) { - Finish(); - } - } - - private: - void Finish() { - if (producer_.Close()) { - finished_.MarkFinished(); - } - } - - AtomicCounter input_counter_; - Future<> finished_ = Future<>::MakeFinished(); - - PushGenerator>::Producer producer_; -}; - -AsyncGenerator> MakeSinkNode(ExecNode* input, - std::string label) { - AsyncGenerator> out; - (void)input->plan()->EmplaceNode(input, std::move(label), &out); - return out; +ExecFactoryRegistry::AddOnLoad::AddOnLoad(std::string factory_name, Factory factory, + ExecFactoryRegistry* registry) { + DCHECK_OK(registry->AddFactory(std::move(factory_name), std::move(factory))); } std::shared_ptr MakeGeneratorReader( @@ -687,625 +278,72 @@ std::shared_ptr MakeGeneratorReader( return out; } -class ThreadIndexer { - public: - size_t operator()() { - auto id = std::this_thread::get_id(); - - std::unique_lock lock(mutex_); - const auto& id_index = *id_to_index_.emplace(id, id_to_index_.size()).first; - - return Check(id_index.second); - } - - static size_t Capacity() { - static size_t max_size = arrow::internal::ThreadPool::DefaultCapacity(); - return max_size; - } - - private: - size_t Check(size_t thread_index) { - DCHECK_LT(thread_index, Capacity()) << "thread index " << thread_index - << " is out of range [0, " << Capacity() << ")"; - - return thread_index; - } - - std::mutex mutex_; - std::unordered_map id_to_index_; -}; - -struct ScalarAggregateNode : ExecNode { - ScalarAggregateNode(ExecNode* input, std::string label, - std::shared_ptr output_schema, - std::vector kernels, - std::vector argument_indices, - std::vector>> states) - : ExecNode(input->plan(), std::move(label), {input}, {"target"}, - /*output_schema=*/std::move(output_schema), - /*num_outputs=*/1), - kernels_(std::move(kernels)), - argument_indices_(std::move(argument_indices)), - states_(std::move(states)) {} - - const char* kind_name() override { return "ScalarAggregateNode"; } - - Status DoConsume(const ExecBatch& batch, size_t thread_index) { - for (size_t i = 0; i < kernels_.size(); ++i) { - KernelContext batch_ctx{plan()->exec_context()}; - batch_ctx.SetState(states_[i][thread_index].get()); - - ExecBatch single_column_batch{{batch[argument_indices_[i]]}, batch.length}; - RETURN_NOT_OK(kernels_[i]->consume(&batch_ctx, single_column_batch)); - } - return Status::OK(); - } - - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - DCHECK_EQ(input, inputs_[0]); - - auto thread_index = get_thread_index_(); - - if (ErrorIfNotOk(DoConsume(std::move(batch), thread_index))) return; - - if (input_counter_.Increment()) { - ErrorIfNotOk(Finish()); - } - } - - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); - outputs_[0]->ErrorReceived(this, std::move(error)); - } - - void InputFinished(ExecNode* input, int num_total) override { - DCHECK_EQ(input, inputs_[0]); - - if (input_counter_.SetTotal(num_total)) { - ErrorIfNotOk(Finish()); - } - } - - Status StartProducing() override { - finished_ = Future<>::Make(); - // Scalar aggregates will only output a single batch - outputs_[0]->InputFinished(this, 1); - 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 { - if (input_counter_.Cancel()) { - finished_.MarkFinished(); - } - inputs_[0]->StopProducing(this); - } - - Future<> finished() override { return finished_; } - - private: - Status Finish() { - ExecBatch batch{{}, 1}; - batch.values.resize(kernels_.size()); - - for (size_t i = 0; i < kernels_.size(); ++i) { - KernelContext ctx{plan()->exec_context()}; - ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll( - kernels_[i], &ctx, std::move(states_[i]))); - RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); - } - - outputs_[0]->InputReceived(this, 0, std::move(batch)); - finished_.MarkFinished(); - return Status::OK(); - } - - Future<> finished_ = Future<>::MakeFinished(); - const std::vector kernels_; - const std::vector argument_indices_; - - std::vector>> states_; - - ThreadIndexer get_thread_index_; - AtomicCounter input_counter_; -}; - -Result MakeScalarAggregateNode(ExecNode* input, std::string label, - std::vector aggregates, - std::vector arguments, - std::vector out_field_names) { - if (aggregates.size() != arguments.size()) { - return Status::Invalid("Provided ", aggregates.size(), " aggregates but ", - arguments.size(), " arguments."); - } - - if (aggregates.size() != out_field_names.size()) { - return Status::Invalid("Provided ", aggregates.size(), " aggregates but ", - out_field_names.size(), " field names for the output."); - } - - auto exec_ctx = input->plan()->exec_context(); - - std::vector kernels(aggregates.size()); - std::vector>> states(kernels.size()); - FieldVector fields(kernels.size()); - std::vector argument_indices(kernels.size()); - - for (size_t i = 0; i < kernels.size(); ++i) { - if (!arguments[i].IsName()) { - return Status::NotImplemented("Non name field refs"); - } - ARROW_ASSIGN_OR_RAISE(auto match, - arguments[i].FindOneOrNone(*input->output_schema())); - argument_indices[i] = match[0]; - - ARROW_ASSIGN_OR_RAISE(auto function, - exec_ctx->func_registry()->GetFunction(aggregates[i].function)); - - if (function->kind() != Function::SCALAR_AGGREGATE) { - return Status::Invalid("Provided non ScalarAggregateFunction ", - aggregates[i].function); - } - - auto in_type = ValueDescr::Array(input->output_schema()->fields()[i]->type()); - - ARROW_ASSIGN_OR_RAISE(const Kernel* kernel, function->DispatchExact({in_type})); - kernels[i] = static_cast(kernel); +Result Declaration::AddToPlan(ExecPlan* plan, + ExecFactoryRegistry* registry) const { + std::vector inputs(this->inputs.size()); - if (aggregates[i].options == nullptr) { - aggregates[i].options = function->default_options(); + size_t i = 0; + for (const Input& input : this->inputs) { + if (auto node = util::get_if(&input)) { + inputs[i++] = *node; + continue; } - - KernelContext kernel_ctx{exec_ctx}; - states[i].resize(ThreadIndexer::Capacity()); - RETURN_NOT_OK(Kernel::InitAll(&kernel_ctx, - KernelInitArgs{kernels[i], - { - in_type, - }, - aggregates[i].options}, - &states[i])); - - // pick one to resolve the kernel signature - kernel_ctx.SetState(states[i][0].get()); - ARROW_ASSIGN_OR_RAISE( - auto descr, kernels[i]->signature->out_type().Resolve(&kernel_ctx, {in_type})); - - fields[i] = field(std::move(out_field_names[i]), std::move(descr.type)); + ARROW_ASSIGN_OR_RAISE(inputs[i++], + util::get(input).AddToPlan(plan, registry)); } - return input->plan()->EmplaceNode( - input, std::move(label), schema(std::move(fields)), std::move(kernels), - std::move(argument_indices), std::move(states)); + ARROW_ASSIGN_OR_RAISE( + auto node, MakeExecNode(this->factory_name, plan, std::move(inputs), *this->options, + registry)); + node->SetLabel(this->label); + return node; } -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, - const std::vector&& agg_src_field_ids, - 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)), - agg_src_field_ids_(std::move(agg_src_field_ids)), - aggs_(std::move(aggs)), - agg_kernels_(std::move(agg_kernels)) {} - - const char* kind_name() override { return "GroupByNode"; } - - Status Consume(ExecBatch batch) { - 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 - std::vector keys(key_field_ids_.size()); - for (size_t i = 0; i < key_field_ids_.size(); ++i) { - keys[i] = batch.values[key_field_ids_[i]]; - } - ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(keys)); - - // Create a batch with group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, state->grouper->Consume(key_batch)); - - // Execute aggregate kernels - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext kernel_ctx{ctx_}; - kernel_ctx.SetState(state->agg_states[i].get()); +Declaration Declaration::Sequence(std::vector decls) { + DCHECK(!decls.empty()); - ARROW_ASSIGN_OR_RAISE( - auto agg_batch, - ExecBatch::Make({batch.values[agg_src_field_ids_[i]], id_batch})); - - RETURN_NOT_OK(agg_kernels_[i]->resize(&kernel_ctx, state->grouper->num_groups())); - RETURN_NOT_OK(agg_kernels_[i]->consume(&kernel_ctx, agg_batch)); - } + Declaration out = std::move(decls.back()); + decls.pop_back(); + auto receiver = &out; + while (!decls.empty()) { + Declaration input = std::move(decls.back()); + decls.pop_back(); - return Status::OK(); + receiver->inputs.emplace_back(std::move(input)); + receiver = &util::get(receiver->inputs.front()); } + return out; +} - Status Merge() { - ThreadLocalState* state0 = &local_states_[0]; - for (size_t i = 1; i < local_states_.size(); ++i) { - ThreadLocalState* state = &local_states_[i]; - 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(); - - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext batch_ctx{ctx_}; - 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(); +ExecFactoryRegistry* default_exec_factory_registry() { + static class : public ExecFactoryRegistry { + public: + Result GetFactory(const std::string& factory_name) override { + auto it = factories_.find(factory_name); + if (it == factories_.end()) { + return Status::KeyError("ExecNode factory named ", factory_name, + " not present in registry."); } + return it->second; } - return Status::OK(); - } - - Result Finalize() { - ThreadLocalState* state = &local_states_[0]; - ExecBatch out_data{{}, state->grouper->num_groups()}; - out_data.values.resize(agg_kernels_.size() + key_field_ids_.size()); + Status AddFactory(std::string factory_name, Factory factory) override { + auto it_success = factories_.emplace(std::move(factory_name), std::move(factory)); - // Aggregate fields come before key fields to match the behavior of GroupBy function - for (size_t i = 0; i < agg_kernels_.size(); ++i) { - KernelContext batch_ctx{ctx_}; - batch_ctx.SetState(state->agg_states[i].get()); - RETURN_NOT_OK(agg_kernels_[i]->finalize(&batch_ctx, &out_data.values[i])); - state->agg_states[i].reset(); - } - - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); - std::move(out_keys.values.begin(), out_keys.values.end(), - out_data.values.begin() + agg_kernels_.size()); - state->grouper.reset(); - - if (output_counter_.SetTotal( - static_cast(BitUtil::CeilDiv(out_data.length, output_batch_size())))) { - // this will be hit if out_data.length == 0 - finished_.MarkFinished(); - } - return out_data; - } - - void OutputNthBatch(int n) { - // bail if StopProducing was called - if (finished_.is_finished()) return; - - int64_t batch_size = output_batch_size(); - outputs_[0]->InputReceived(this, n, out_data_.Slice(batch_size * n, batch_size)); - - if (output_counter_.Increment()) { - finished_.MarkFinished(); - } - } - - Status OutputResult() { - RETURN_NOT_OK(Merge()); - ARROW_ASSIGN_OR_RAISE(out_data_, Finalize()); - - int num_output_batches = *output_counter_.total(); - outputs_[0]->InputFinished(this, num_output_batches); - - auto executor = ctx_->executor(); - for (int i = 0; i < num_output_batches; ++i) { - if (executor) { - // bail if StopProducing was called - if (finished_.is_finished()) break; - - RETURN_NOT_OK(executor->Spawn([this, i] { OutputNthBatch(i); })); - } else { - OutputNthBatch(i); + if (!it_success.second) { + const auto& factory_name = it_success.first->first; + return Status::KeyError("ExecNode factory named ", factory_name, + " already registered."); } - } - - return Status::OK(); - } - void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { - // bail if StopProducing was called - if (finished_.is_finished()) return; - - DCHECK_EQ(input, inputs_[0]); - - if (ErrorIfNotOk(Consume(std::move(batch)))) return; - - if (input_counter_.Increment()) { - ErrorIfNotOk(OutputResult()); - } - } - - void ErrorReceived(ExecNode* input, Status error) override { - DCHECK_EQ(input, inputs_[0]); - - outputs_[0]->ErrorReceived(this, std::move(error)); - } - - 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)) { - ErrorIfNotOk(OutputResult()); - } - } - - Status StartProducing() override { - finished_ = Future<>::Make(); - - local_states_.resize(ThreadIndexer::Capacity()); - return Status::OK(); - } - - void PauseProducing(ExecNode* output) override {} - - void ResumeProducing(ExecNode* output) override {} - - void StopProducing(ExecNode* output) override { - DCHECK_EQ(output, outputs_[0]); - - if (input_counter_.Cancel()) { - finished_.MarkFinished(); - } else if (output_counter_.Cancel()) { - finished_.MarkFinished(); - } - inputs_[0]->StopProducing(this); - } - - void StopProducing() override { StopProducing(outputs_[0]); } - - Future<> finished() override { return finished_; } - - private: - struct ThreadLocalState { - std::unique_ptr grouper; - std::vector> agg_states; - }; - - ThreadLocalState* GetLocalState() { - size_t thread_index = get_thread_index_(); - return &local_states_[thread_index]; - } - - 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)); - - return Status::OK(); - } - - int output_batch_size() const { - int result = static_cast(ctx_->exec_chunksize()); - if (result < 0) { - result = 32 * 1024; + return Status::OK(); } - return result; - } - - ExecContext* ctx_; - Future<> finished_ = Future<>::MakeFinished(); - - 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_; - AtomicCounter input_counter_, output_counter_; - - std::vector local_states_; - ExecBatch out_data_; -}; -Result MakeGroupByNode(ExecNode* input, std::string label, - std::vector keys, - std::vector agg_srcs, - std::vector aggs) { - // 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) { - ARROW_ASSIGN_OR_RAISE(auto match, FieldRef(keys[i]).FindOne(*input_schema)); - key_field_ids[i] = match[0]; - } - - // Find input field indices for aggregates - std::vector agg_src_field_ids(aggs.size()); - for (size_t i = 0; i < aggs.size(); ++i) { - 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 - 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(), ValueDescr::ARRAY); - } - - auto ctx = input->plan()->exec_context(); - - // 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); - } - - 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(agg_src_field_ids), std::move(aggs), - 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 (const auto& argument : arguments) { - inputs.push_back(argument); - } - for (const auto& key : keys) { - inputs.push_back(key); - } - 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)); - 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"); - - 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 = - 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(); - } + private: + std::unordered_map factories_; + } instance; - 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); + return &instance; } } // namespace compute diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index fc3af92af4a..4a784ceb75b 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -22,7 +22,6 @@ #include #include -#include "arrow/compute/api_aggregate.h" #include "arrow/compute/exec.h" #include "arrow/compute/type_fwd.h" #include "arrow/type_fwd.h" @@ -113,6 +112,7 @@ class ARROW_EXPORT ExecNode { /// /// There is no guarantee that this value is non-empty or unique. const std::string& label() const { return label_; } + void SetLabel(std::string label) { label_ = std::move(label); } Status Validate() const; @@ -218,9 +218,8 @@ class ARROW_EXPORT ExecNode { virtual Future<> finished() = 0; protected: - ExecNode(ExecPlan* plan, std::string label, NodeVector inputs, - std::vector input_labels, std::shared_ptr output_schema, - int num_outputs); + ExecNode(ExecPlan* plan, NodeVector inputs, 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. @@ -237,21 +236,114 @@ class ARROW_EXPORT ExecNode { NodeVector outputs_; }; -/// \brief Adapt an AsyncGenerator as a source node -/// -/// plan->exec_context()->executor() is used to parallelize pushing to -/// outputs, if provided. -ARROW_EXPORT -ExecNode* MakeSourceNode(ExecPlan* plan, std::string label, - std::shared_ptr output_schema, - std::function>()>); +/// \brief An extensible registry for factories of ExecNodes +class ARROW_EXPORT ExecFactoryRegistry { + public: + using Factory = std::function(ExecPlan*, std::vector, + const ExecNodeOptions&)>; -/// \brief Add a sink node which forwards to an AsyncGenerator -/// -/// Emitted batches will not be ordered. + virtual ~ExecFactoryRegistry() = default; + + /// \brief Get the named factory from this registry + /// + /// will raise if factory_name is not found + virtual Result GetFactory(const std::string& factory_name) = 0; + + /// \brief Add a factory to this registry with the provided name + /// + /// will raise if factory_name is already in the registry + virtual Status AddFactory(std::string factory_name, Factory factory) = 0; + + /// Helper for declaring on-load registration of ExecNode factories, + /// including built-in factories. + struct ARROW_EXPORT AddOnLoad { + AddOnLoad(std::string factory_name, Factory factory, ExecFactoryRegistry* registry); + AddOnLoad(std::string factory_name, Factory factory); + }; +}; + +/// The default registry, which includes built-in factories. ARROW_EXPORT -std::function>()> MakeSinkNode(ExecNode* input, - std::string label); +ExecFactoryRegistry* default_exec_factory_registry(); + +/// \brief Construct an ExecNode using the named factory +inline Result MakeExecNode( + const std::string& factory_name, ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options, + ExecFactoryRegistry* registry = default_exec_factory_registry()) { + ARROW_ASSIGN_OR_RAISE(auto factory, registry->GetFactory(factory_name)); + return factory(plan, std::move(inputs), options); +} + +/// \brief Helper class for declaring sets of ExecNodes efficiently +/// +/// A Declaration represents an unconstructed ExecNode (and potentially more since its +/// inputs may also be Declarations). The node can be constructed and added to a plan +/// with Declaration::AddToPlan, which will recursively construct any inputs as necessary. +struct ARROW_EXPORT Declaration { + using Input = util::Variant; + + Declaration(std::string factory_name, std::vector inputs, + std::shared_ptr options, std::string label) + : factory_name{std::move(factory_name)}, + inputs{std::move(inputs)}, + options{std::move(options)}, + label{std::move(label)} {} + + template + Declaration(std::string factory_name, std::vector inputs, Options options) + : factory_name{std::move(factory_name)}, + inputs{std::move(inputs)}, + options{std::make_shared(std::move(options))}, + label{this->factory_name} {} + + template + Declaration(std::string factory_name, Options options) + : factory_name{std::move(factory_name)}, + inputs{}, + options{std::make_shared(std::move(options))}, + label{this->factory_name} {} + + /// \brief Convenience factory for the common case of a simple sequence of nodes. + /// + /// Each of decls will be appended to the inputs of the subsequent declaration, + /// and the final modified declaration will be returned. + /// + /// Without this convenience factory, constructing a sequence would require explicit, + /// difficult-to-read nesting: + /// + /// Declaration{"n3", + /// { + /// Declaration{"n2", + /// { + /// Declaration{"n1", + /// { + /// Declaration{"n0", N0Opts{}}, + /// }, + /// N1Opts{}}, + /// }, + /// N2Opts{}}, + /// }, + /// N3Opts{}}; + /// + /// An equivalent Declaration can be constructed more tersely using Sequence: + /// + /// Declaration::Sequence({ + /// {"n0", N0Opts{}}, + /// {"n1", N1Opts{}}, + /// {"n2", N2Opts{}}, + /// {"n3", N3Opts{}}, + /// }); + static Declaration Sequence(std::vector decls); + + Result AddToPlan(ExecPlan* plan, ExecFactoryRegistry* registry = + default_exec_factory_registry()) const; + + std::string factory_name; + std::vector inputs; + std::shared_ptr options; + std::string label; +}; /// \brief Wrap an ExecBatch generator in a RecordBatchReader. /// @@ -261,47 +353,5 @@ std::shared_ptr MakeGeneratorReader( std::shared_ptr, std::function>()>, MemoryPool*); -/// \brief Make a node which excludes some rows from batches passed through it -/// -/// The filter Expression will be evaluated against each batch which is pushed to -/// this node. Any rows for which the filter does not evaluate to `true` will be excluded -/// in the batch emitted by this node. -/// -/// If the filter is not already bound, it will be bound against the input's schema. -ARROW_EXPORT -Result MakeFilterNode(ExecNode* input, std::string label, Expression filter); - -/// \brief Make a node which executes expressions on input batches, producing new batches. -/// -/// Each expression will be evaluated against each batch which is pushed to -/// this node to produce a corresponding output column. -/// -/// If exprs are not already bound, they will be bound against the input's schema. -/// If names are not provided, the string representations of exprs will be used. -ARROW_EXPORT -Result MakeProjectNode(ExecNode* input, std::string label, - std::vector exprs, - std::vector names = {}); - -ARROW_EXPORT -Result MakeScalarAggregateNode(ExecNode* input, std::string label, - std::vector aggregates, - std::vector arguments, - std::vector out_field_names); - -/// \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, - std::vector aggs); - -ARROW_EXPORT -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/filter_node.cc b/cpp/src/arrow/compute/exec/filter_node.cc new file mode 100644 index 00000000000..1d0e1774cbe --- /dev/null +++ b/cpp/src/arrow/compute/exec/filter_node.cc @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/exec_plan.h" + +#include "arrow/compute/api_vector.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { +namespace { + +class FilterNode : public ExecNode { + public: + FilterNode(ExecPlan* plan, std::vector inputs, + std::shared_ptr output_schema, Expression filter) + : ExecNode(plan, std::move(inputs), /*input_labels=*/{"target"}, + std::move(output_schema), + /*num_outputs=*/1), + filter_(std::move(filter)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "FilterNode")); + auto schema = inputs[0]->output_schema(); + + const auto& filter_options = checked_cast(options); + + auto filter_expression = filter_options.filter_expression; + if (!filter_expression.IsBound()) { + ARROW_ASSIGN_OR_RAISE(filter_expression, filter_expression.Bind(*schema)); + } + + if (filter_expression.type()->id() != Type::BOOL) { + return Status::TypeError("Filter expression must evaluate to bool, but ", + filter_expression.ToString(), " evaluates to ", + filter_expression.type()->ToString()); + } + + return plan->EmplaceNode(plan, std::move(inputs), std::move(schema), + std::move(filter_expression)); + } + + const char* kind_name() override { return "FilterNode"; } + + Result DoFilter(const ExecBatch& target) { + ARROW_ASSIGN_OR_RAISE(Expression simplified_filter, + SimplifyWithGuarantee(filter_, target.guarantee)); + + ARROW_ASSIGN_OR_RAISE(Datum mask, ExecuteScalarExpression(simplified_filter, target, + plan()->exec_context())); + + if (mask.is_scalar()) { + const auto& mask_scalar = mask.scalar_as(); + if (mask_scalar.is_valid && mask_scalar.value) { + return target; + } + + return target.Slice(0, 0); + } + + // if the values are all scalar then the mask must also be + DCHECK(!std::all_of(target.values.begin(), target.values.end(), + [](const Datum& value) { return value.is_scalar(); })); + + auto values = target.values; + for (auto& value : values) { + if (value.is_scalar()) continue; + ARROW_ASSIGN_OR_RAISE(value, Filter(value, mask, FilterOptions::Defaults())); + } + return ExecBatch::Make(std::move(values)); + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + auto maybe_filtered = DoFilter(std::move(batch)); + if (ErrorIfNotOk(maybe_filtered.status())) return; + + maybe_filtered->guarantee = batch.guarantee; + outputs_[0]->InputReceived(this, seq, maybe_filtered.MoveValueUnsafe()); + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + outputs_[0]->ErrorReceived(this, std::move(error)); + } + + void InputFinished(ExecNode* input, int seq) override { + DCHECK_EQ(input, inputs_[0]); + outputs_[0]->InputFinished(this, 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]); + StopProducing(); + } + + void StopProducing() override { inputs_[0]->StopProducing(this); } + + Future<> finished() override { return inputs_[0]->finished(); } + + private: + Expression filter_; +}; + +ExecFactoryRegistry::AddOnLoad kRegisterFilter("filter", FilterNode::Make); + +} // namespace +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h new file mode 100644 index 00000000000..a853a74362d --- /dev/null +++ b/cpp/src/arrow/compute/exec/options.h @@ -0,0 +1,115 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include +#include + +#include "arrow/compute/api_aggregate.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/expression.h" +#include "arrow/util/optional.h" +#include "arrow/util/visibility.h" + +namespace arrow { +namespace compute { + +class ARROW_EXPORT ExecNodeOptions { + public: + virtual ~ExecNodeOptions() = default; +}; + +/// \brief Adapt an AsyncGenerator as a source node +/// +/// plan->exec_context()->executor() will be used to parallelize pushing to +/// outputs, if provided. +class ARROW_EXPORT SourceNodeOptions : public ExecNodeOptions { + public: + SourceNodeOptions(std::shared_ptr output_schema, + std::function>()> generator) + : output_schema(std::move(output_schema)), generator(std::move(generator)) {} + + std::shared_ptr output_schema; + std::function>()> generator; +}; + +/// \brief Make a node which excludes some rows from batches passed through it +/// +/// filter_expression will be evaluated against each batch which is pushed to +/// this node. Any rows for which filter_expression does not evaluate to `true` will be +/// excluded in the batch emitted by this node. +class ARROW_EXPORT FilterNodeOptions : public ExecNodeOptions { + public: + explicit FilterNodeOptions(Expression filter_expression) + : filter_expression(std::move(filter_expression)) {} + + Expression filter_expression; +}; + +/// \brief Make a node which executes expressions on input batches, producing new batches. +/// +/// Each expression will be evaluated against each batch which is pushed to +/// this node to produce a corresponding output column. +/// +/// If names are not provided, the string representations of exprs will be used. +class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { + public: + explicit ProjectNodeOptions(std::vector expressions, + std::vector names = {}) + : expressions(std::move(expressions)), names(std::move(names)) {} + + std::vector expressions; + std::vector names; +}; + +/// \brief Make a node which aggregates input batches, optionally grouped by keys. +class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { + public: + AggregateNodeOptions(std::vector aggregates, + std::vector targets, std::vector names, + std::vector keys = {}) + : aggregates(std::move(aggregates)), + targets(std::move(targets)), + names(std::move(names)), + keys(std::move(keys)) {} + + // aggregations which will be applied to the targetted fields + std::vector aggregates; + // fields to which aggregations will be applied + std::vector targets; + // output field names for aggregations + std::vector names; + // keys by which aggregations will be grouped + std::vector keys; +}; + +/// \brief Add a sink node which forwards to an AsyncGenerator +/// +/// Emitted batches will not be ordered. +class ARROW_EXPORT SinkNodeOptions : public ExecNodeOptions { + public: + explicit SinkNodeOptions(std::function>()>* generator) + : generator(generator) {} + + std::function>()>* generator; +}; + +} // 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 101257f5de8..1ca620304cf 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -23,6 +23,7 @@ #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" #include "arrow/compute/exec/test_util.h" #include "arrow/record_batch.h" #include "arrow/testing/future_util.h" @@ -199,41 +200,39 @@ namespace { struct BatchesWithSchema { std::vector batches; std::shared_ptr schema; -}; -Result MakeTestSourceNode(ExecPlan* plan, std::string label, - BatchesWithSchema batches_with_schema, bool parallel, - bool slow) { - DCHECK_GT(batches_with_schema.batches.size(), 0); + AsyncGenerator> gen(bool parallel, bool slow) const { + DCHECK_GT(batches.size(), 0); - auto opt_batches = ::arrow::internal::MapVector( - [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, - std::move(batches_with_schema.batches)); + auto opt_batches = ::arrow::internal::MapVector( + [](ExecBatch batch) { return util::make_optional(std::move(batch)); }, batches); - AsyncGenerator> gen; + AsyncGenerator> gen; - if (parallel) { - // emulate batches completing initial decode-after-scan on a cpu thread - ARROW_ASSIGN_OR_RAISE( - gen, MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), - ::arrow::internal::GetCpuThreadPool())); + if (parallel) { + // emulate batches completing initial decode-after-scan on a cpu thread + gen = MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), + ::arrow::internal::GetCpuThreadPool()) + .ValueOrDie(); - // ensure that callbacks are not executed immediately on a background thread - gen = MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); - } else { - gen = MakeVectorGenerator(std::move(opt_batches)); - } + // ensure that callbacks are not executed immediately on a background thread + gen = + MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); + } else { + gen = MakeVectorGenerator(std::move(opt_batches)); + } - if (slow) { - gen = MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { - SleepABit(); - return batch; - }); - } + if (slow) { + gen = + MakeMappedGenerator(std::move(gen), [](const util::optional& batch) { + SleepABit(); + return batch; + }); + } - return MakeSourceNode(plan, label, std::move(batches_with_schema.schema), - std::move(gen)); -} + return gen; + } +}; Future> StartAndCollect( ExecPlan* plan, AsyncGenerator> gen) { @@ -284,13 +283,17 @@ TEST(ExecPlanExecution, SourceSink) { SCOPED_TRACE(parallel ? "parallel" : "single threaded"); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic_data = MakeBasicBatches(); - ASSERT_OK_AND_ASSIGN(auto source, MakeTestSourceNode(plan.get(), "source", - basic_data, parallel, slow)); - - auto sink_gen = MakeSinkNode(source, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{basic_data.schema, + basic_data.gen(parallel, slow)}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray(basic_data.batches)))); @@ -300,10 +303,11 @@ TEST(ExecPlanExecution, SourceSink) { TEST(ExecPlanExecution, SourceSinkError) { ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic_data = MakeBasicBatches(); auto it = basic_data.batches.begin(); - AsyncGenerator> gen = + AsyncGenerator> error_source_gen = [&]() -> Result> { if (it == basic_data.batches.end()) { return Status::Invalid("Artificial error"); @@ -311,8 +315,12 @@ TEST(ExecPlanExecution, SourceSinkError) { return util::make_optional(*it++); }; - auto source = MakeSourceNode(plan.get(), "source", {}, gen); - auto sink_gen = MakeSinkNode(source, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{basic_data.schema, error_source_gen}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(Raises(StatusCode::Invalid, HasSubstr("Artificial")))); @@ -328,14 +336,18 @@ TEST(ExecPlanExecution, StressSourceSink) { int num_batches = slow && !parallel ? 30 : 300; ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; auto random_data = MakeRandomBatches( schema({field("a", int32()), field("b", boolean())}), num_batches); - ASSERT_OK_AND_ASSIGN(auto source, MakeTestSourceNode(plan.get(), "source", - random_data, parallel, slow)); - - auto sink_gen = MakeSinkNode(source, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{random_data.schema, + random_data.gen(parallel, slow)}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray(random_data.batches)))); @@ -353,14 +365,18 @@ TEST(ExecPlanExecution, StressSourceSinkStopped) { int num_batches = slow && !parallel ? 30 : 300; ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; auto random_data = MakeRandomBatches( schema({field("a", int32()), field("b", boolean())}), num_batches); - ASSERT_OK_AND_ASSIGN(auto source, MakeTestSourceNode(plan.get(), "source", - random_data, parallel, slow)); - - auto sink_gen = MakeSinkNode(source, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{random_data.schema, + random_data.gen(parallel, slow)}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_OK(plan->Validate()); ASSERT_OK(plan->StartProducing()); @@ -374,18 +390,20 @@ TEST(ExecPlanExecution, StressSourceSinkStopped) { } TEST(ExecPlanExecution, SourceFilterSink) { - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - auto basic_data = MakeBasicBatches(); - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", basic_data, - /*parallel=*/false, /*slow=*/false)); - - ASSERT_OK_AND_ASSIGN( - auto filter, MakeFilterNode(source, "filter", equal(field_ref("i32"), literal(6)))); - - auto sink_gen = MakeSinkNode(filter, "sink"); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; + + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{basic_data.schema, + basic_data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"filter", FilterNodeOptions{equal(field_ref("i32"), literal(6))}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray( @@ -394,26 +412,25 @@ TEST(ExecPlanExecution, SourceFilterSink) { } TEST(ExecPlanExecution, SourceProjectSink) { - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - auto basic_data = MakeBasicBatches(); - ASSERT_OK_AND_ASSIGN(auto source, - 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, {"!bool", "i32 + 1"})); - - auto sink_gen = MakeSinkNode(projection, "sink"); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; + + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{basic_data.schema, + basic_data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"project", + ProjectNodeOptions{{ + not_(field_ref("bool")), + call("add", {field_ref("i32"), literal(1)}), + }, + {"!bool", "i32 + 1"}}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray( @@ -454,6 +471,7 @@ BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { return out; } + } // namespace TEST(ExecPlanExecution, SourceGroupedSum) { @@ -463,14 +481,19 @@ TEST(ExecPlanExecution, SourceGroupedSum) { auto input = MakeGroupableBatches(/*multiplicity=*/parallel ? 100 : 1); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", input, - /*parallel=*/parallel, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN( - auto gby, MakeGroupByNode(source, "gby", /*keys=*/{"str"}, /*targets=*/{"i32"}, - {{"hash_sum", nullptr}})); - auto sink_gen = MakeSinkNode(gby, "sink"); + AsyncGenerator> sink_gen; + + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{input.schema, + input.gen(parallel, /*slow=*/false)}}, + {"aggregate", + AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr}}, + /*targets=*/{"i32"}, /*names=*/{"sum(i32)"}, + /*keys=*/{"str"}}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( @@ -488,32 +511,28 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { auto input = MakeGroupableBatches(/*multiplicity=*/batch_multiplicity); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", input, - /*parallel=*/parallel, /*slow=*/false)); - ASSERT_OK_AND_ASSIGN( - auto 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"}, + AsyncGenerator> sink_gen; + + ASSERT_OK( + Declaration::Sequence( + { + {"source", + SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"filter", + FilterNodeOptions{greater_equal(field_ref("i32"), literal(0))}}, + {"project", ProjectNodeOptions{{ + field_ref("str"), + call("multiply", {field_ref("i32"), literal(2)}), + }}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr}}, /*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"); + /*names=*/{"sum(multiply(i32, 2))"}, + /*keys=*/{"str"}}}, + {"filter", FilterNodeOptions{greater(field_ref("sum(multiply(i32, 2))"), + literal(10 * batch_multiplicity))}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( @@ -524,20 +543,22 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { TEST(ExecPlanExecution, SourceScalarAggSink) { ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic_data = MakeBasicBatches(); - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", basic_data, - /*parallel=*/false, /*slow=*/false)); - - ASSERT_OK_AND_ASSIGN( - auto scalar_agg, - MakeScalarAggregateNode(source, "scalar_agg", {{"sum", nullptr}, {"any", nullptr}}, - /*targets=*/{"i32", "bool"}, - /*out_field_names=*/{"sum(i32)", "any(bool)"})); - - auto sink_gen = MakeSinkNode(scalar_agg, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{basic_data.schema, + basic_data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{ + /*aggregates=*/{{"sum", nullptr}, {"any", nullptr}}, + /*targets=*/{"i32", "bool"}, + /*names=*/{"sum(i32)", "any(bool)"}}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT( StartAndCollect(plan.get(), sink_gen), @@ -549,28 +570,32 @@ TEST(ExecPlanExecution, SourceScalarAggSink) { TEST(ExecPlanExecution, ScalarSourceScalarAggSink) { ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); + AsyncGenerator> sink_gen; - BatchesWithSchema basic_data; - basic_data.batches = { + BatchesWithSchema scalar_data; + scalar_data.batches = { ExecBatchFromJSON({ValueDescr::Scalar(int32()), ValueDescr::Scalar(int32()), ValueDescr::Scalar(int32())}, "[[5, 5, 5], [5, 5, 5], [5, 5, 5]]"), ExecBatchFromJSON({int32(), int32(), int32()}, "[[5, 5, 5], [6, 6, 6], [7, 7, 7]]")}; - basic_data.schema = + scalar_data.schema = schema({field("a", int32()), field("b", int32()), field("c", int32())}); - ASSERT_OK_AND_ASSIGN(auto source, - MakeTestSourceNode(plan.get(), "source", basic_data, - /*parallel=*/false, /*slow=*/false)); - - ASSERT_OK_AND_ASSIGN( - auto scalar_agg, - MakeScalarAggregateNode(source, "scalar_agg", - {{"count", nullptr}, {"sum", nullptr}, {"mean", nullptr}}, - {"a", "b", "c"}, {"sum a", "sum b", "sum c"})); - - auto sink_gen = MakeSinkNode(scalar_agg, "sink"); + ASSERT_OK(Declaration::Sequence( + { + {"source", SourceNodeOptions{scalar_data.schema, + scalar_data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"aggregate", + AggregateNodeOptions{ + /*aggregates=*/{ + {"count", nullptr}, {"sum", nullptr}, {"mean", nullptr}}, + /*targets=*/{"a", "b", "c"}, + /*names=*/{"count(a)", "sum(b)", "mean(c)"}}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); ASSERT_THAT( StartAndCollect(plan.get(), sink_gen), diff --git a/cpp/src/arrow/compute/exec/project_node.cc b/cpp/src/arrow/compute/exec/project_node.cc new file mode 100644 index 00000000000..b0db8c6eb4f --- /dev/null +++ b/cpp/src/arrow/compute/exec/project_node.cc @@ -0,0 +1,133 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/exec_plan.h" + +#include "arrow/compute/api_vector.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { +namespace { + +class ProjectNode : public ExecNode { + public: + ProjectNode(ExecPlan* plan, std::vector inputs, + std::shared_ptr output_schema, std::vector exprs) + : ExecNode(plan, std::move(inputs), /*input_labels=*/{"target"}, + std::move(output_schema), + /*num_outputs=*/1), + exprs_(std::move(exprs)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ProjectNode")); + + const auto& project_options = checked_cast(options); + auto exprs = project_options.expressions; + auto names = project_options.names; + + if (names.size() == 0) { + names.resize(exprs.size()); + for (size_t i = 0; i < exprs.size(); ++i) { + names[i] = exprs[i].ToString(); + } + } + + FieldVector fields(exprs.size()); + int i = 0; + for (auto& expr : exprs) { + if (!expr.IsBound()) { + ARROW_ASSIGN_OR_RAISE(expr, expr.Bind(*inputs[0]->output_schema())); + } + fields[i] = field(std::move(names[i]), expr.type()); + ++i; + } + + return plan->EmplaceNode(plan, std::move(inputs), + schema(std::move(fields)), std::move(exprs)); + } + + const char* kind_name() override { return "ProjectNode"; } + + Result DoProject(const ExecBatch& target) { + std::vector values{exprs_.size()}; + for (size_t i = 0; i < exprs_.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(Expression simplified_expr, + SimplifyWithGuarantee(exprs_[i], target.guarantee)); + + ARROW_ASSIGN_OR_RAISE(values[i], ExecuteScalarExpression(simplified_expr, target, + plan()->exec_context())); + } + return ExecBatch{std::move(values), target.length}; + } + + void InputReceived(ExecNode* input, int seq, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + auto maybe_projected = DoProject(std::move(batch)); + if (ErrorIfNotOk(maybe_projected.status())) return; + + maybe_projected->guarantee = batch.guarantee; + outputs_[0]->InputReceived(this, seq, maybe_projected.MoveValueUnsafe()); + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + outputs_[0]->ErrorReceived(this, std::move(error)); + } + + void InputFinished(ExecNode* input, int seq) override { + DCHECK_EQ(input, inputs_[0]); + outputs_[0]->InputFinished(this, 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]); + StopProducing(); + } + + void StopProducing() override { inputs_[0]->StopProducing(this); } + + Future<> finished() override { return inputs_[0]->finished(); } + + private: + std::vector exprs_; +}; + +ExecFactoryRegistry::AddOnLoad kRegisterProject("project", ProjectNode::Make); + +} // namespace +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc new file mode 100644 index 00000000000..168e4123d14 --- /dev/null +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -0,0 +1,140 @@ + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/exec_plan.h" + +#include + +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/exec_internal.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" +#include "arrow/util/optional.h" +#include "arrow/util/thread_pool.h" +#include "arrow/util/unreachable.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { +namespace { + +class SinkNode : public ExecNode { + public: + SinkNode(ExecPlan* plan, std::vector inputs, + AsyncGenerator>* generator) + : ExecNode(plan, std::move(inputs), {"collected"}, {}, + /*num_outputs=*/0), + producer_(MakeProducer(generator)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "SinkNode")); + + const auto& sink_options = checked_cast(options); + return plan->EmplaceNode(plan, std::move(inputs), sink_options.generator); + } + + static PushGenerator>::Producer MakeProducer( + AsyncGenerator>* out_gen) { + PushGenerator> gen; + auto out = gen.producer(); + *out_gen = std::move(gen); + return out; + } + + const char* kind_name() override { return "SinkNode"; } + + Status StartProducing() override { + finished_ = Future<>::Make(); + return Status::OK(); + } + + // sink nodes have no outputs from which to feel backpressure + [[noreturn]] static void NoOutputs() { + Unreachable("no outputs; this should never be called"); + } + [[noreturn]] void ResumeProducing(ExecNode* output) override { NoOutputs(); } + [[noreturn]] void PauseProducing(ExecNode* output) override { NoOutputs(); } + [[noreturn]] void StopProducing(ExecNode* output) override { NoOutputs(); } + + void StopProducing() override { + Finish(); + inputs_[0]->StopProducing(this); + } + + Future<> finished() override { return finished_; } + + void InputReceived(ExecNode* input, int seq_num, ExecBatch batch) override { + DCHECK_EQ(input, inputs_[0]); + + bool did_push = producer_.Push(std::move(batch)); + if (!did_push) return; // producer_ was Closed already + + if (auto total = input_counter_.total()) { + DCHECK_LE(seq_num, *total); + } + + if (input_counter_.Increment()) { + Finish(); + } + } + + void ErrorReceived(ExecNode* input, Status error) override { + DCHECK_EQ(input, inputs_[0]); + + producer_.Push(std::move(error)); + + if (input_counter_.Cancel()) { + Finish(); + } + inputs_[0]->StopProducing(this); + } + + void InputFinished(ExecNode* input, int seq_stop) override { + if (input_counter_.SetTotal(seq_stop)) { + Finish(); + } + } + + private: + void Finish() { + if (producer_.Close()) { + finished_.MarkFinished(); + } + } + + AtomicCounter input_counter_; + Future<> finished_ = Future<>::MakeFinished(); + + PushGenerator>::Producer producer_; +}; + +ExecFactoryRegistry::AddOnLoad kRegisterSink("sink", SinkNode::Make); + +} // namespace +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/source_node.cc b/cpp/src/arrow/compute/exec/source_node.cc new file mode 100644 index 00000000000..46f753bb12a --- /dev/null +++ b/cpp/src/arrow/compute/exec/source_node.cc @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/exec/exec_plan.h" + +#include + +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/exec_internal.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" +#include "arrow/util/optional.h" +#include "arrow/util/thread_pool.h" +#include "arrow/util/unreachable.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { +namespace { + +struct SourceNode : ExecNode { + SourceNode(ExecPlan* plan, std::shared_ptr output_schema, + AsyncGenerator> generator) + : ExecNode(plan, {}, {}, std::move(output_schema), + /*num_outputs=*/1), + generator_(std::move(generator)) {} + + static Result Make(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { + RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 0, "SourceNode")); + const auto& source_options = checked_cast(options); + return plan->EmplaceNode(plan, source_options.output_schema, + source_options.generator); + } + + const char* kind_name() override { return "SourceNode"; } + + [[noreturn]] static void NoInputs() { + Unreachable("no inputs; this should never be called"); + } + [[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 SourceNode"; + + CallbackOptions options; + if (auto executor = plan()->exec_context()->executor()) { + // These options will transfer execution to the desired Executor if necessary. + // This can happen for in-memory scans where batches didn't require + // any CPU work to decode. Otherwise, parsing etc should have already + // been placed us on the desired Executor and no queues will be pushed to. + options.executor = executor; + options.should_schedule = ShouldSchedule::IfDifferentExecutor; + } + + finished_ = Loop([this, options] { + std::unique_lock lock(mutex_); + int seq = batch_count_++; + if (stop_requested_) { + return Future>::MakeFinished(Break(seq)); + } + lock.unlock(); + + return generator_().Then( + [=](const util::optional& batch) -> ControlFlow { + std::unique_lock lock(mutex_); + if (IsIterationEnd(batch) || stop_requested_) { + stop_requested_ = true; + return Break(seq); + } + lock.unlock(); + + outputs_[0]->InputReceived(this, seq, *batch); + return Continue(); + }, + [=](const Status& error) -> ControlFlow { + // NB: ErrorReceived is independent of InputFinished, but + // ErrorReceived will usually prompt StopProducing which will + // prompt InputFinished. ErrorReceived may still be called from a + // node which was requested to stop (indeed, the request to stop + // may prompt an error). + std::unique_lock lock(mutex_); + stop_requested_ = true; + lock.unlock(); + outputs_[0]->ErrorReceived(this, error); + return Break(seq); + }, + options); + }).Then([&](int seq) { outputs_[0]->InputFinished(this, seq); }); + + 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: + std::mutex mutex_; + bool stop_requested_{false}; + int batch_count_{0}; + Future<> finished_ = Future<>::MakeFinished(); + AsyncGenerator> generator_; +}; + +ExecFactoryRegistry::AddOnLoad kRegisterSource("source", SourceNode::Make); + +} // namespace +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index b47d6087c0b..49f089f3459 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -50,10 +50,9 @@ namespace compute { namespace { struct DummyNode : ExecNode { - DummyNode(ExecPlan* plan, std::string label, NodeVector inputs, int num_outputs, + DummyNode(ExecPlan* plan, NodeVector inputs, int num_outputs, StartProducingFunc start_producing, StopProducingFunc stop_producing) - : ExecNode(plan, std::move(label), std::move(inputs), {}, dummy_schema(), - num_outputs), + : ExecNode(plan, std::move(inputs), {}, dummy_schema(), num_outputs), start_producing_(std::move(start_producing)), stop_producing_(std::move(stop_producing)) { input_labels_.resize(inputs_.size()); @@ -127,9 +126,11 @@ struct DummyNode : ExecNode { ExecNode* MakeDummyNode(ExecPlan* plan, std::string label, std::vector inputs, int num_outputs, StartProducingFunc start_producing, StopProducingFunc stop_producing) { - return plan->EmplaceNode(plan, std::move(label), std::move(inputs), - num_outputs, std::move(start_producing), - std::move(stop_producing)); + auto node = + plan->EmplaceNode(plan, std::move(inputs), num_outputs, + std::move(start_producing), std::move(stop_producing)); + node->SetLabel(std::move(label)); + return node; } ExecBatch ExecBatchFromJSON(const std::vector& descrs, diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index a44676c2f0d..652d58a0d2e 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -17,6 +17,7 @@ #include "arrow/compute/exec/util.h" +#include "arrow/compute/exec/exec_plan.h" #include "arrow/util/bit_util.h" #include "arrow/util/bitmap_ops.h" #include "arrow/util/ubsan.h" @@ -275,4 +276,25 @@ bool BitUtil::are_all_bytes_zero(int64_t hardware_flags, const uint8_t* bytes, } } // namespace util + +namespace compute { + +Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inputs, + int expected_num_inputs, const char* kind_name) { + if (static_cast(inputs.size()) != expected_num_inputs) { + return Status::Invalid(kind_name, " node requires ", expected_num_inputs, " but got ", + inputs.size()); + } + + for (auto input : inputs) { + if (input->plan() != plan) { + return Status::Invalid("Constructing a ", kind_name, + " node in a different plan from its input"); + } + } + + return Status::OK(); +} + +} // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index d8248ceacab..b7cf0aeaa5e 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -17,16 +17,19 @@ #pragma once +#include #include #include #include "arrow/buffer.h" +#include "arrow/compute/type_fwd.h" #include "arrow/memory_pool.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/util/bit_util.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" +#include "arrow/util/optional.h" #if defined(__clang__) || defined(__GNUC__) #define BYTESWAP(x) __builtin_bswap64(x) @@ -179,4 +182,52 @@ class BitUtil { }; } // namespace util +namespace compute { + +ARROW_EXPORT +Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inputs, + int expected_num_inputs, const char* kind_name); + +class AtomicCounter { + public: + AtomicCounter() = default; + + int count() const { return count_.load(); } + + util::optional total() const { + int total = total_.load(); + if (total == -1) return {}; + return total; + } + + // return true if the counter is complete + bool Increment() { + DCHECK_NE(count_.load(), total_.load()); + int count = count_.fetch_add(1) + 1; + if (count != total_.load()) return false; + return DoneOnce(); + } + + // return true if the counter is complete + bool SetTotal(int total) { + total_.store(total); + if (count_.load() != total) return false; + return DoneOnce(); + } + + // return true if the counter has not already been completed + bool Cancel() { return DoneOnce(); } + + private: + // ensure there is only one true return from Increment(), SetTotal(), or Cancel() + bool DoneOnce() { + bool expected = false; + return complete_.compare_exchange_strong(expected, true); + } + + std::atomic count_{0}, total_{-1}; + std::atomic complete_{false}; +}; + +} // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index d37d8f32ac8..b0114eac55c 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -25,6 +25,7 @@ #include #include "arrow/array.h" +#include "arrow/array/concatenate.h" #include "arrow/chunked_array.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_scalar.h" @@ -32,7 +33,9 @@ #include "arrow/compute/cast.h" #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" +#include "arrow/compute/exec/options.h" #include "arrow/compute/exec/test_util.h" +#include "arrow/compute/exec_internal.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/codegen_internal.h" #include "arrow/compute/kernels/test_util.h" @@ -45,12 +48,14 @@ #include "arrow/testing/random.h" #include "arrow/type.h" #include "arrow/type_traits.h" +#include "arrow/util/async_generator.h" #include "arrow/util/bitmap_reader.h" #include "arrow/util/checked_cast.h" #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" +#include "arrow/util/vector.h" using testing::HasSubstr; @@ -114,6 +119,93 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys return StructArray::Make(std::move(out_columns), std::move(out_names)); } +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()); + std::vector names(arguments.size()); + for (size_t i = 0; i < arguments.size(); ++i) { + auto name = std::string("agg_") + std::to_string(i); + names[i] = aggregates[i].function; + scan_fields[i] = field(name, arguments[i].type()); + arguments_str[i] = FieldRef(std::move(name)); + } + for (size_t i = 0; i < keys.size(); ++i) { + auto name = std::string("key_") + std::to_string(i); + scan_fields[arguments.size() + i] = field(name, keys[i].type()); + keys_str[i] = FieldRef(std::move(name)); + } + + std::vector scan_batches; + std::vector inputs; + for (const auto& argument : arguments) { + inputs.push_back(argument); + } + for (const auto& key : keys) { + inputs.push_back(key); + } + 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)); + AsyncGenerator> sink_gen; + RETURN_NOT_OK( + Declaration::Sequence( + { + {"source", SourceNodeOptions{schema(std::move(scan_fields)), + MakeVectorGenerator(arrow::internal::MapVector( + [](ExecBatch batch) { + return util::make_optional( + std::move(batch)); + }, + std::move(scan_batches)))}}, + {"aggregate", + AggregateNodeOptions{std::move(aggregates), std::move(arguments_str), + std::move(names), std::move(keys_str)}}, + {"sink", SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); + + 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 = + start_and_collect.MoveResult().MoveValueUnsafe(); + + ArrayVector out_arrays(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(out_arrays[i], Concatenate(arrays)); + } + + return StructArray::Make(std::move(out_arrays), + plan->sources()[0]->outputs()[0]->output_schema()->fields()); +} + void ValidateGroupBy(const std::vector& aggregates, std::vector arguments, std::vector keys) { ASSERT_OK_AND_ASSIGN(Datum expected, NaiveGroupBy(arguments, keys, aggregates)); diff --git a/cpp/src/arrow/compute/type_fwd.h b/cpp/src/arrow/compute/type_fwd.h index eebc8c1b678..127929ced58 100644 --- a/cpp/src/arrow/compute/type_fwd.h +++ b/cpp/src/arrow/compute/type_fwd.h @@ -43,6 +43,8 @@ struct KernelState; class Expression; class ExecNode; class ExecPlan; +class ExecNodeOptions; +class ExecFactoryRegistry; } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/dataset/dataset_internal.h b/cpp/src/arrow/dataset/dataset_internal.h index a1245b7e2a0..ab0253a9c5a 100644 --- a/cpp/src/arrow/dataset/dataset_internal.h +++ b/cpp/src/arrow/dataset/dataset_internal.h @@ -92,6 +92,10 @@ class FragmentDataset : public Dataset { FragmentDataset(std::shared_ptr schema, FragmentVector fragments) : Dataset(std::move(schema)), fragments_(std::move(fragments)) {} + FragmentDataset(std::shared_ptr schema, + AsyncGenerator> fragments) + : Dataset(std::move(schema)), fragment_gen_(std::move(fragments)) {} + std::string type_name() const override { return "fragment"; } Result> ReplaceSchema( @@ -101,6 +105,14 @@ class FragmentDataset : public Dataset { protected: Result GetFragmentsImpl(compute::Expression predicate) override { + if (fragment_gen_) { + // TODO(ARROW-8163): Async fragment scanning can be forwarded rather than waiting + // for the whole generator here. For now, all Dataset impls have a vector of + // Fragments anyway + auto fragments_fut = CollectAsyncGenerator(std::move(fragment_gen_)); + ARROW_ASSIGN_OR_RAISE(fragments_, fragments_fut.result()); + } + // TODO(ARROW-12891) Provide subtree pruning for any vector of fragments FragmentVector fragments; for (const auto& fragment : fragments_) { @@ -114,7 +126,9 @@ class FragmentDataset : public Dataset { } return MakeVectorIterator(std::move(fragments)); } + FragmentVector fragments_; + AsyncGenerator> fragment_gen_; }; } // namespace dataset diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index d81b9cd1c5c..b7ff01b57e6 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -462,54 +462,6 @@ const FieldVector kAugmentedFields{ field("__last_in_fragment", boolean()), }; -Result MakeScanNode(compute::ExecPlan* plan, - FragmentGenerator fragment_gen, - std::shared_ptr options) { - if (!options->use_async) { - return Status::NotImplemented("ScanNodes without asynchrony"); - } - - ARROW_ASSIGN_OR_RAISE(auto batch_gen_gen, - FragmentsToBatches(std::move(fragment_gen), options)); - - auto merged_batch_gen = - MakeMergedGenerator(std::move(batch_gen_gen), options->fragment_readahead); - - auto batch_gen = - MakeReadaheadGenerator(std::move(merged_batch_gen), options->fragment_readahead); - - auto gen = MakeMappedGenerator( - std::move(batch_gen), - [options](const EnumeratedRecordBatch& partial) - -> Result> { - ARROW_ASSIGN_OR_RAISE( - util::optional batch, - compute::MakeExecBatch(*options->dataset_schema, partial.record_batch.value)); - // TODO(ARROW-13263) fragments may be able to attach more guarantees to batches - // than this, for example parquet's row group stats. Failing to do this leaves - // perf on the table because row group stats could be used to skip kernel execs in - // FilterNode. - // - // Additionally, if a fragment failed to perform projection pushdown there may be - // unnecessarily materialized columns in batch. We could drop them now instead of - // letting them coast through the rest of the plan. - batch->guarantee = partial.fragment.value->partition_expression(); - - // tag rows with fragment- and batch-of-origin - batch->values.emplace_back(partial.fragment.index); - batch->values.emplace_back(partial.record_batch.index); - batch->values.emplace_back(partial.record_batch.last); - return batch; - }); - - 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 { public: OneShotScanTask(RecordBatchIterator batch_it, std::shared_ptr options, @@ -620,22 +572,22 @@ Result AsyncScanner::ScanBatchesUnorderedAsync( std::make_shared(scan_options_->pool, cpu_executor); ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(exec_context.get())); - - ARROW_ASSIGN_OR_RAISE(auto scan, MakeScanNode(plan.get(), dataset_, scan_options_)); - - ARROW_ASSIGN_OR_RAISE(auto filter, - compute::MakeFilterNode(scan, "filter", scan_options_->filter)); + AsyncGenerator> sink_gen; auto exprs = scan_options_->projection.call()->arguments; auto names = checked_cast( scan_options_->projection.call()->options.get()) ->field_names; - ARROW_ASSIGN_OR_RAISE( - auto project, - MakeAugmentedProjectNode(filter, "project", std::move(exprs), std::move(names))); - AsyncGenerator> sink_gen = - compute::MakeSinkNode(project, "sink"); + RETURN_NOT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{dataset_, scan_options_}}, + {"filter", compute::FilterNodeOptions{scan_options_->filter}}, + {"augmented_project", + compute::ProjectNodeOptions{std::move(exprs), std::move(names)}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); RETURN_NOT_OK(plan->StartProducing()); @@ -790,7 +742,7 @@ Result AsyncScanner::CountRows() { 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_); + const auto options = std::make_shared(*scan_options_); RETURN_NOT_OK(SetProjection(options.get(), std::vector())); std::atomic total{0}; @@ -813,21 +765,23 @@ Result AsyncScanner::CountRows() { }); }); - ARROW_ASSIGN_OR_RAISE(auto scan, - MakeScanNode(plan.get(), std::move(fragment_gen), options)); - - ARROW_ASSIGN_OR_RAISE(auto get_selection, - compute::MakeProjectNode(scan, "get_selection", {options->filter}, - {"selection_mask"})); - - ARROW_ASSIGN_OR_RAISE( - auto sum_selection, - compute::MakeScalarAggregateNode(get_selection, "sum_selection", - {compute::internal::Aggregate{"sum", nullptr}}, - {"selection_mask"}, {"sum"})); - - AsyncGenerator> sink_gen = - compute::MakeSinkNode(sum_selection, "sink"); + AsyncGenerator> sink_gen; + + RETURN_NOT_OK( + compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{std::make_shared( + scan_options_->dataset_schema, + std::move(fragment_gen)), + options}}, + {"project", compute::ProjectNodeOptions{{options->filter}, {"mask"}}}, + {"aggregate", compute::AggregateNodeOptions{{compute::internal::Aggregate{ + "sum", nullptr}}, + /*targets=*/{"mask"}, + /*names=*/{"selected_count"}}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); RETURN_NOT_OK(plan->StartProducing()); auto maybe_slow_count = sink_gen().result(); @@ -1146,9 +1100,19 @@ Result SyncScanner::CountRows() { return count; } +namespace { + Result MakeScanNode(compute::ExecPlan* plan, - std::shared_ptr dataset, - std::shared_ptr scan_options) { + std::vector inputs, + const compute::ExecNodeOptions& options) { + const auto& scan_node_options = checked_cast(options); + auto scan_options = scan_node_options.scan_options; + auto dataset = scan_node_options.dataset; + + if (!scan_options->use_async) { + return Status::NotImplemented("ScanNodes without asynchrony"); + } + if (scan_options->dataset_schema == nullptr) { scan_options->dataset_schema = dataset->schema(); } @@ -1171,14 +1135,59 @@ Result MakeScanNode(compute::ExecPlan* plan, // 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()); - auto fragments_gen = MakeVectorGenerator(std::move(fragments_vec)); + auto fragment_gen = MakeVectorGenerator(std::move(fragments_vec)); + + ARROW_ASSIGN_OR_RAISE(auto batch_gen_gen, + FragmentsToBatches(std::move(fragment_gen), scan_options)); + + auto merged_batch_gen = + MakeMergedGenerator(std::move(batch_gen_gen), scan_options->fragment_readahead); + + auto batch_gen = MakeReadaheadGenerator(std::move(merged_batch_gen), + scan_options->fragment_readahead); + + auto gen = MakeMappedGenerator( + std::move(batch_gen), + [scan_options](const EnumeratedRecordBatch& partial) + -> Result> { + ARROW_ASSIGN_OR_RAISE(util::optional batch, + compute::MakeExecBatch(*scan_options->dataset_schema, + partial.record_batch.value)); + // TODO(ARROW-13263) fragments may be able to attach more guarantees to batches + // than this, for example parquet's row group stats. Failing to do this leaves + // perf on the table because row group stats could be used to skip kernel execs in + // FilterNode. + // + // Additionally, if a fragment failed to perform projection pushdown there may be + // unnecessarily materialized columns in batch. We could drop them now instead of + // letting them coast through the rest of the plan. + batch->guarantee = partial.fragment.value->partition_expression(); + + // tag rows with fragment- and batch-of-origin + batch->values.emplace_back(partial.fragment.index); + batch->values.emplace_back(partial.record_batch.index); + batch->values.emplace_back(partial.record_batch.last); + return batch; + }); + + auto fields = scan_options->dataset_schema->fields(); + for (const auto& aug_field : kAugmentedFields) { + fields.push_back(aug_field); + } - return MakeScanNode(plan, std::move(fragments_gen), std::move(scan_options)); + return compute::MakeExecNode( + "source", plan, {}, + compute::SourceNodeOptions{schema(std::move(fields)), std::move(gen)}); } +compute::ExecFactoryRegistry::AddOnLoad kRegisterScan("scan", MakeScanNode); Result MakeAugmentedProjectNode( - compute::ExecNode* input, std::string label, std::vector exprs, - std::vector names) { + compute::ExecPlan* plan, std::vector inputs, + const compute::ExecNodeOptions& options) { + const auto& project_options = checked_cast(options); + auto exprs = project_options.expressions; + auto names = project_options.names; + if (names.size() == 0) { names.resize(exprs.size()); for (size_t i = 0; i < exprs.size(); ++i) { @@ -1190,13 +1199,26 @@ Result MakeAugmentedProjectNode( 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)); + return compute::MakeExecNode( + "project", plan, std::move(inputs), + compute::ProjectNodeOptions{std::move(exprs), std::move(names)}); } +compute::ExecFactoryRegistry::AddOnLoad kRegisterProject("augmented_project", + MakeAugmentedProjectNode); + +Result MakeOrderedSinkNode(compute::ExecPlan* plan, + std::vector inputs, + const compute::ExecNodeOptions& options) { + if (inputs.size() != 1) { + return Status::Invalid("Ordered SinkNode requires exactly 1 input, got ", + inputs.size()); + } + auto input = inputs[0]; -Result>> MakeOrderedSinkNode( - compute::ExecNode* input, std::string label) { - auto unordered = compute::MakeSinkNode(input, std::move(label)); + AsyncGenerator> unordered; + ARROW_ASSIGN_OR_RAISE(auto node, + compute::MakeExecNode("sink", plan, std::move(inputs), + compute::SinkNodeOptions{&unordered})); const Schema& schema = *input->output_schema(); ARROW_ASSIGN_OR_RAISE(FieldPath match, FieldRef("__fragment_index").FindOne(schema)); @@ -1256,9 +1278,16 @@ Result>> MakeOrderedSinkNode( last_in_fragment(*prev) && batch_index(*next) == 0; }; - return MakeSequencingGenerator(std::move(unordered), left_after_right, is_next, - util::make_optional(std::move(before_any))); + const auto& sink_options = checked_cast(options); + *sink_options.generator = + MakeSequencingGenerator(std::move(unordered), left_after_right, is_next, + util::make_optional(std::move(before_any))); + + return node; } +compute::ExecFactoryRegistry::AddOnLoad kRegisterSink("ordered_sink", + MakeOrderedSinkNode); +} // namespace } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/scanner.h b/cpp/src/arrow/dataset/scanner.h index fc715206d7d..445303d5fc1 100644 --- a/cpp/src/arrow/dataset/scanner.h +++ b/cpp/src/arrow/dataset/scanner.h @@ -26,6 +26,7 @@ #include #include "arrow/compute/exec/expression.h" +#include "arrow/compute/exec/options.h" #include "arrow/compute/type_fwd.h" #include "arrow/dataset/dataset.h" #include "arrow/dataset/projector.h" @@ -412,21 +413,15 @@ class ARROW_DS_EXPORT ScannerBuilder { /// Does not construct associated filter or project nodes. /// Yielded batches will be augmented with fragment/batch indices to enable stable /// ordering for simple ExecPlans. -ARROW_DS_EXPORT Result MakeScanNode(compute::ExecPlan*, - std::shared_ptr, - std::shared_ptr); - -/// \brief Construct a ProjectNode which preserves fragment/batch indices. -ARROW_DS_EXPORT Result MakeAugmentedProjectNode( - compute::ExecNode* input, std::string label, std::vector exprs, - std::vector names = {}); +class ARROW_DS_EXPORT ScanNodeOptions : public compute::ExecNodeOptions { + public: + explicit ScanNodeOptions(std::shared_ptr dataset, + std::shared_ptr scan_options) + : dataset(std::move(dataset)), scan_options(std::move(scan_options)) {} -/// \brief Add a sink node which forwards to an AsyncGenerator -/// -/// Emitted batches will be ordered by fragment and batch indices, or an error -/// will be raised if those fields are not available in the input. -ARROW_DS_EXPORT Result>> -MakeOrderedSinkNode(compute::ExecNode*, std::string label); + std::shared_ptr dataset; + std::shared_ptr scan_options; +}; /// @} diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 34fa1486ef2..4971ecfd29a 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -1202,7 +1202,9 @@ TEST(ScanNode, Schema) { options->use_async = true; options->projection = Materialize({}); // set an empty projection - ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); + ASSERT_OK_AND_ASSIGN(auto scan, + compute::MakeExecNode("scan", plan.get(), {}, + ScanNodeOptions{basic.dataset, options})); auto fields = basic.dataset->schema()->fields(); fields.push_back(field("__fragment_index", int32())); @@ -1215,6 +1217,7 @@ TEST(ScanNode, Schema) { TEST(ScanNode, Trivial) { ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic = MakeBasicDataset(); @@ -1223,8 +1226,12 @@ TEST(ScanNode, Trivial) { // 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"); + ASSERT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{basic.dataset, options}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); // trivial scan: the batches are returned unmodified auto expected = basic.batches; @@ -1234,6 +1241,7 @@ TEST(ScanNode, Trivial) { TEST(ScanNode, FilteredOnVirtualColumn) { ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic = MakeBasicDataset(); @@ -1243,9 +1251,12 @@ TEST(ScanNode, FilteredOnVirtualColumn) { // 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"); + ASSERT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{basic.dataset, options}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); auto expected = basic.batches; @@ -1259,6 +1270,7 @@ TEST(ScanNode, FilteredOnVirtualColumn) { TEST(ScanNode, DeferredFilterOnPhysicalColumn) { ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic = MakeBasicDataset(); @@ -1268,9 +1280,12 @@ TEST(ScanNode, DeferredFilterOnPhysicalColumn) { // 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"); + ASSERT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{basic.dataset, options}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); // No post filtering is performed by ScanNode: all batches will be yielded whole. // To filter out rows from individual batches, construct a FilterNode. @@ -1283,6 +1298,7 @@ TEST(ScanNode, DeferredFilterOnPhysicalColumn) { TEST(ScanNode, DISABLED_ProjectionPushdown) { // ARROW-13263 ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic = MakeBasicDataset(); @@ -1290,9 +1306,12 @@ TEST(ScanNode, DISABLED_ProjectionPushdown) { options->use_async = true; options->projection = Materialize({"b"}, /*include_aug_fields=*/true); - ASSERT_OK_AND_ASSIGN(auto scan, MakeScanNode(plan.get(), basic.dataset, options)); - - auto sink_gen = MakeSinkNode(scan, "sink"); + ASSERT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{basic.dataset, options}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); auto expected = basic.batches; @@ -1310,6 +1329,7 @@ TEST(ScanNode, DISABLED_ProjectionPushdown) { TEST(ScanNode, MaterializationOfVirtualColumn) { ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + AsyncGenerator> sink_gen; auto basic = MakeBasicDataset(); @@ -1317,21 +1337,22 @@ TEST(ScanNode, MaterializationOfVirtualColumn) { options->use_async = true; 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, - dataset::MakeAugmentedProjectNode( - scan, "project", {field_ref("a"), field_ref("b"), field_ref("c")})); - - auto sink_gen = MakeSinkNode(project, "sink"); + ASSERT_OK(compute::Declaration::Sequence( + { + {"scan", ScanNodeOptions{basic.dataset, options}}, + {"augmented_project", + compute::ProjectNodeOptions{ + {field_ref("a"), field_ref("b"), field_ref("c")}}}, + {"sink", compute::SinkNodeOptions{&sink_gen}}, + }) + .AddToPlan(plan.get())); auto expected = basic.batches; for (auto& batch : expected) { // ProjectNode overwrites "c" placeholder with non-null drawn from guarantee const auto& value = *batch.guarantee.call()->arguments[1].literal(); - batch.values[project->output_schema()->GetFieldIndex("c")] = value; + batch.values[2] = value; } ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), @@ -1377,23 +1398,30 @@ TEST(ScanNode, MinimalEndToEnd) { call("make_struct", {a_times_2}, compute::MakeStructOptions{{"a * 2"}}); // construct the scan node - ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, - dataset::MakeScanNode(plan.get(), dataset, options)); + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * scan, + compute::MakeExecNode("scan", plan.get(), {}, ScanNodeOptions{dataset, options})); // pipe the scan node into a filter node ASSERT_OK_AND_ASSIGN(compute::ExecNode * filter, - compute::MakeFilterNode(scan, "filter", b_is_true)); + compute::MakeExecNode("filter", plan.get(), {scan}, + compute::FilterNodeOptions{b_is_true})); // pipe the filter node into a project node // NB: we're using the project node factory which preserves fragment/batch index // tagging, so we *can* reorder later if we choose. The tags will not appear in // our output. ASSERT_OK_AND_ASSIGN(compute::ExecNode * project, - dataset::MakeAugmentedProjectNode(filter, "project", {a_times_2})); + compute::MakeExecNode("augmented_project", plan.get(), {filter}, + compute::ProjectNodeOptions{{a_times_2}})); // finally, pipe the project node into a sink node - // NB: if we don't need ordering, we could use compute::MakeSinkNode instead - ASSERT_OK_AND_ASSIGN(auto sink_gen, dataset::MakeOrderedSinkNode(project, "sink")); + AsyncGenerator> sink_gen; + ASSERT_OK_AND_ASSIGN(compute::ExecNode * sink, + compute::MakeExecNode("ordered_sink", plan.get(), {project}, + compute::SinkNodeOptions{&sink_gen})); + + ASSERT_THAT(plan->sinks(), ElementsAre(sink)); // translate sink_gen (async) to sink_reader (sync) std::shared_ptr sink_reader = compute::MakeGeneratorReader( @@ -1456,26 +1484,37 @@ TEST(ScanNode, MinimalScalarAggEndToEnd) { call("make_struct", {a_times_2}, compute::MakeStructOptions{{"a * 2"}}); // construct the scan node - ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, - dataset::MakeScanNode(plan.get(), dataset, options)); + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * scan, + compute::MakeExecNode("scan", plan.get(), {}, ScanNodeOptions{dataset, options})); // pipe the scan node into a filter node ASSERT_OK_AND_ASSIGN(compute::ExecNode * filter, - compute::MakeFilterNode(scan, "filter", b_is_true)); + compute::MakeExecNode("filter", plan.get(), {scan}, + compute::FilterNodeOptions{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})); + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * project, + compute::MakeExecNode("project", plan.get(), {filter}, + compute::ProjectNodeOptions{{a_times_2}, {"a * 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}}, - {a_times_2.ToString()}, {"a*2 sum"})); + compute::ExecNode * aggregate, + compute::MakeExecNode( + "aggregate", plan.get(), {project}, + compute::AggregateNodeOptions{{compute::internal::Aggregate{"sum", nullptr}}, + /*targets=*/{"a * 2"}, + /*names=*/{"sum(a * 2)"}})); - // finally, pipe the project node into a sink node - auto sink_gen = compute::MakeSinkNode(sum, "sink"); + // finally, pipe the aggregate node into a sink node + AsyncGenerator> sink_gen; + ASSERT_OK_AND_ASSIGN(compute::ExecNode * sink, + compute::MakeExecNode("sink", plan.get(), {aggregate}, + compute::SinkNodeOptions{&sink_gen})); + + ASSERT_THAT(plan->sinks(), ElementsAre(sink)); // translate sink_gen (async) to sink_reader (sync) std::shared_ptr sink_reader = @@ -1538,26 +1577,37 @@ TEST(ScanNode, MinimalGroupedAggEndToEnd) { call("make_struct", {a_times_2, b}, compute::MakeStructOptions{{"a * 2", "b"}}); // construct the scan node - ASSERT_OK_AND_ASSIGN(compute::ExecNode * scan, - dataset::MakeScanNode(plan.get(), dataset, options)); + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * scan, + compute::MakeExecNode("scan", plan.get(), {}, ScanNodeOptions{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"})); + compute::MakeExecNode("project", plan.get(), {scan}, + compute::ProjectNodeOptions{{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"); + ASSERT_OK_AND_ASSIGN( + compute::ExecNode * aggregate, + compute::MakeExecNode("aggregate", plan.get(), {project}, + compute::AggregateNodeOptions{ + {compute::internal::Aggregate{"hash_sum", nullptr}}, + /*targets=*/{"a * 2"}, + /*names=*/{"sum(a * 2)"}, + /*keys=*/{"b"}})); + + // finally, pipe the aggregate node into a sink node + AsyncGenerator> sink_gen; + ASSERT_OK_AND_ASSIGN(compute::ExecNode * sink, + compute::MakeExecNode("sink", plan.get(), {aggregate}, + compute::SinkNodeOptions{&sink_gen})); + + ASSERT_THAT(plan->sinks(), ElementsAre(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), + schema({field("sum(a * 2)", int64()), field("b", boolean())}), std::move(sink_gen), exec_context.memory_pool()); // start the ExecPlan @@ -1569,13 +1619,13 @@ TEST(ScanNode, MinimalGroupedAggEndToEnd) { // 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} - ])"}); + auto expected = TableFromJSON( + schema({field("sum(a * 2)", int64()), field("b", boolean())}), { + R"JSON([ + {"sum(a * 2)": 12, "b": null}, + {"sum(a * 2)": 4, "b": true}, + {"sum(a * 2)": 40, "b": false} + ])JSON"}); AssertTablesEqual(*expected, *collected, /*same_chunk_layout=*/false); } diff --git a/cpp/src/arrow/util/unreachable.h b/cpp/src/arrow/util/unreachable.h index 027f76e84d2..55263598134 100644 --- a/cpp/src/arrow/util/unreachable.h +++ b/cpp/src/arrow/util/unreachable.h @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +#pragma once + namespace arrow { [[noreturn]] void Unreachable(const char* message = "Unreachable"); From 73e5ef7f7ed18cff71440f32df2d022ee784cd9f Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 4 Aug 2021 16:45:25 -0400 Subject: [PATCH 2/8] repair R binding --- cpp/src/arrow/compute/exec/util.cc | 4 +- r/R/arrowExports.R | 8 +--- r/R/dplyr-summarize.R | 27 ++++------- r/R/query-engine.R | 7 +-- r/src/arrowExports.cpp | 32 +++---------- r/src/compute-exec.cpp | 74 +++++++++++++----------------- 6 files changed, 55 insertions(+), 97 deletions(-) diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/compute/exec/util.cc index 652d58a0d2e..eecc617c9c0 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/compute/exec/util.cc @@ -282,8 +282,8 @@ namespace compute { Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inputs, int expected_num_inputs, const char* kind_name) { if (static_cast(inputs.size()) != expected_num_inputs) { - return Status::Invalid(kind_name, " node requires ", expected_num_inputs, " but got ", - inputs.size()); + return Status::Invalid(kind_name, " node requires ", expected_num_inputs, + " inputs but got ", inputs.size()); } for (auto input : inputs) { diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R index 73f3ba67de6..a04c0efcdae 100644 --- a/r/R/arrowExports.R +++ b/r/R/arrowExports.R @@ -300,12 +300,8 @@ ExecNode_Project <- function(input, exprs, names) { .Call(`_arrow_ExecNode_Project`, input, exprs, names) } -ExecNode_ScalarAggregate <- function(input, options, target_names, out_field_names) { - .Call(`_arrow_ExecNode_ScalarAggregate`, input, options, target_names, out_field_names) -} - -ExecNode_GroupByAggregate <- function(input, group_vars, agg_srcs, aggregations) { - .Call(`_arrow_ExecNode_GroupByAggregate`, input, group_vars, agg_srcs, aggregations) +ExecNode_Aggregate <- function(input, options, target_names, out_field_names, key_names){ + .Call(`_arrow_ExecNode_Aggregate`, input, options, target_names, out_field_names, key_names) } RecordBatch__cast <- function(batch, schema, options) { diff --git a/r/R/dplyr-summarize.R b/r/R/dplyr-summarize.R index 87e1157dfc7..394e5fe2ac9 100644 --- a/r/R/dplyr-summarize.R +++ b/r/R/dplyr-summarize.R @@ -110,26 +110,19 @@ do_exec_plan <- function(.data, group_vars = NULL) { # If any columns are derived we need to Project (otherwise this may be no-op) project_node <- start_node$Project(.data$selected_columns) + final_node <- project_node$Aggregate( + options = .data$aggregations, + target_names = target_names, + out_field_names = names(.data$aggregations), + key_names = group_vars + ) + + out <- plan$Run(final_node) if (grouped) { - final_node <- project_node$GroupByAggregate( - group_vars, - target_names = target_names, - aggregations = .data$aggregations - ) - out <- plan$Run(final_node) - # The result will have result columns first (named by their function) - # then the grouping cols. dplyr orders group cols first, and it accepts - # names for the result cols. Adapt the result to meet that expectation. + # The result will have result columns first then the grouping cols. + # dplyr orders group cols first, so adapt the result to meet that expectation. n_results <- length(.data$aggregations) - names(out)[seq_along(.data$aggregations)] <- names(.data$aggregations) out <- out[c((n_results + 1):ncol(out), seq_along(.data$aggregations))] - } else { - final_node <- project_node$ScalarAggregate( - options = .data$aggregations, - target_names = target_names, - out_field_names = names(.data$aggregations) - ) - out <- plan$Run(final_node) } out } diff --git a/r/R/query-engine.R b/r/R/query-engine.R index 72c35c515db..4de2f87165b 100644 --- a/r/R/query-engine.R +++ b/r/R/query-engine.R @@ -65,11 +65,8 @@ ExecNode <- R6Class("ExecNode", assert_is(expr, "Expression") ExecNode_Filter(self, expr) }, - ScalarAggregate = function(options, target_names, out_field_names) { - ExecNode_ScalarAggregate(self, options, target_names, out_field_names) - }, - GroupByAggregate = function(group_vars, target_names, aggregations) { - ExecNode_GroupByAggregate(self, group_vars, target_names, aggregations) + Aggregate = function(options, target_names, out_field_names, key_names) { + ExecNode_Aggregate(self, options, target_names, out_field_names, key_names) } ) ) diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp index 92ddbae23fd..0645a64eac9 100644 --- a/r/src/arrowExports.cpp +++ b/r/src/arrowExports.cpp @@ -1176,37 +1176,20 @@ extern "C" SEXP _arrow_ExecNode_Project(SEXP input_sexp, SEXP exprs_sexp, SEXP n // compute-exec.cpp #if defined(ARROW_R_WITH_ARROW) -std::shared_ptr ExecNode_ScalarAggregate(const std::shared_ptr& input, cpp11::list options, std::vector target_names, std::vector out_field_names); -extern "C" SEXP _arrow_ExecNode_ScalarAggregate(SEXP input_sexp, SEXP options_sexp, SEXP target_names_sexp, SEXP out_field_names_sexp){ +std::shared_ptr ExecNode_Aggregate(const std::shared_ptr& input, cpp11::list options, std::vector target_names, std::vector out_field_names, std::vector key_names); +extern "C" SEXP _arrow_ExecNode_Aggregate(SEXP input_sexp, SEXP options_sexp, SEXP target_names_sexp, SEXP out_field_names_sexp, SEXP key_names_sexp){ BEGIN_CPP11 arrow::r::Input&>::type input(input_sexp); arrow::r::Input::type options(options_sexp); arrow::r::Input>::type target_names(target_names_sexp); arrow::r::Input>::type out_field_names(out_field_names_sexp); - return cpp11::as_sexp(ExecNode_ScalarAggregate(input, options, target_names, out_field_names)); + arrow::r::Input>::type key_names(key_names_sexp); + return cpp11::as_sexp(ExecNode_Aggregate(input, options, target_names, out_field_names, key_names)); END_CPP11 } #else -extern "C" SEXP _arrow_ExecNode_ScalarAggregate(SEXP input_sexp, SEXP options_sexp, SEXP target_names_sexp, SEXP out_field_names_sexp){ - Rf_error("Cannot call ExecNode_ScalarAggregate(). See https://arrow.apache.org/docs/r/articles/install.html for help installing Arrow C++ libraries. "); -} -#endif - -// compute-exec.cpp -#if defined(ARROW_R_WITH_ARROW) -std::shared_ptr ExecNode_GroupByAggregate(const std::shared_ptr& input, std::vector group_vars, std::vector agg_srcs, cpp11::list aggregations); -extern "C" SEXP _arrow_ExecNode_GroupByAggregate(SEXP input_sexp, SEXP group_vars_sexp, SEXP agg_srcs_sexp, SEXP aggregations_sexp){ -BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); - arrow::r::Input>::type group_vars(group_vars_sexp); - arrow::r::Input>::type agg_srcs(agg_srcs_sexp); - arrow::r::Input::type aggregations(aggregations_sexp); - return cpp11::as_sexp(ExecNode_GroupByAggregate(input, group_vars, agg_srcs, aggregations)); -END_CPP11 -} -#else -extern "C" SEXP _arrow_ExecNode_GroupByAggregate(SEXP input_sexp, SEXP group_vars_sexp, SEXP agg_srcs_sexp, SEXP aggregations_sexp){ - Rf_error("Cannot call ExecNode_GroupByAggregate(). See https://arrow.apache.org/docs/r/articles/install.html for help installing Arrow C++ libraries. "); +extern "C" SEXP _arrow_ExecNode_Aggregate(SEXP input_sexp, SEXP options_sexp, SEXP target_names_sexp, SEXP out_field_names_sexp, SEXP key_names_sexp){ + Rf_error("Cannot call ExecNode_Aggregate(). See https://arrow.apache.org/docs/r/articles/install.html for help installing Arrow C++ libraries. "); } #endif @@ -7149,8 +7132,7 @@ static const R_CallMethodDef CallEntries[] = { { "_arrow_ExecNode_Scan", (DL_FUNC) &_arrow_ExecNode_Scan, 4}, { "_arrow_ExecNode_Filter", (DL_FUNC) &_arrow_ExecNode_Filter, 2}, { "_arrow_ExecNode_Project", (DL_FUNC) &_arrow_ExecNode_Project, 3}, - { "_arrow_ExecNode_ScalarAggregate", (DL_FUNC) &_arrow_ExecNode_ScalarAggregate, 4}, - { "_arrow_ExecNode_GroupByAggregate", (DL_FUNC) &_arrow_ExecNode_GroupByAggregate, 4}, + { "_arrow_ExecNode_Aggregate", (DL_FUNC) &_arrow_ExecNode_Aggregate, 5}, { "_arrow_RecordBatch__cast", (DL_FUNC) &_arrow_RecordBatch__cast, 3}, { "_arrow_Table__cast", (DL_FUNC) &_arrow_Table__cast, 3}, { "_arrow_compute__CallFunction", (DL_FUNC) &_arrow_compute__CallFunction, 3}, diff --git a/r/src/compute-exec.cpp b/r/src/compute-exec.cpp index 61a79bf462e..736b499b086 100644 --- a/r/src/compute-exec.cpp +++ b/r/src/compute-exec.cpp @@ -22,7 +22,9 @@ #include #include #include +#include #include +#include #include #include @@ -42,13 +44,25 @@ std::shared_ptr ExecPlan_create(bool use_threads) { return plan; } +std::shared_ptr MakeExecNodeOrStop( + const std::string& factory_name, compute::ExecPlan* plan, + std::vector inputs, const compute::ExecNodeOptions& options) { + return std::shared_ptr( + ValueOrStop(compute::MakeExecNode(factory_name, plan, std::move(inputs), options)), + [](...) { + // empty destructor: ExecNode lifetime is managed by an ExecPlan + }); +} + // [[arrow::export]] std::shared_ptr ExecPlan_run( const std::shared_ptr& plan, const std::shared_ptr& final_node) { // For now, don't require R to construct SinkNodes. // Instead, just pass the node we should collect as an argument. - auto sink_gen = compute::MakeSinkNode(final_node.get(), "sink"); + arrow::AsyncGenerator> sink_gen; + MakeExecNodeOrStop("sink", plan.get(), {final_node.get()}, + compute::SinkNodeOptions{&sink_gen}); StopIfNotOk(plan->Validate()); StopIfNotOk(plan->StartProducing()); @@ -60,13 +74,6 @@ std::shared_ptr ExecPlan_run( return ValueOrStop(arrow::Table::FromRecordBatchReader(sink_reader.get())); } -std::shared_ptr ExecNodeOrStop( - arrow::Result maybe_node) { - return std::shared_ptr(ValueOrStop(maybe_node), [](...) { - // empty destructor: ExecNode lifetime is managed by an ExecPlan - }); -} - #if defined(ARROW_R_WITH_DATASET) #include @@ -98,7 +105,8 @@ std::shared_ptr ExecNode_Scan( compute::MakeStructOptions{std::move(materialized_field_names)}) .Bind(*dataset->schema())); - return ExecNodeOrStop(arrow::dataset::MakeScanNode(plan.get(), dataset, options)); + return MakeExecNodeOrStop("scan", plan.get(), {}, + arrow::dataset::ScanNodeOptions{dataset, options}); } #endif @@ -107,8 +115,8 @@ std::shared_ptr ExecNode_Scan( std::shared_ptr ExecNode_Filter( const std::shared_ptr& input, const std::shared_ptr& filter) { - return ExecNodeOrStop( - compute::MakeFilterNode(input.get(), /*label=*/"filter", *filter)); + return MakeExecNodeOrStop("filter", input->plan(), {input.get()}, + compute::FilterNodeOptions{*filter}); } // [[arrow::export]] @@ -121,14 +129,16 @@ std::shared_ptr ExecNode_Project( for (auto expr : exprs) { expressions.push_back(*expr); } - return ExecNodeOrStop(compute::MakeProjectNode( - input.get(), /*label=*/"project", std::move(expressions), std::move(names))); + return MakeExecNodeOrStop( + "project", input->plan(), {input.get()}, + compute::ProjectNodeOptions{std::move(expressions), std::move(names)}); } // [[arrow::export]] -std::shared_ptr ExecNode_ScalarAggregate( +std::shared_ptr ExecNode_Aggregate( const std::shared_ptr& input, cpp11::list options, - std::vector target_names, std::vector out_field_names) { + std::vector target_names, std::vector out_field_names, + std::vector key_names) { std::vector aggregates; std::vector> keep_alives; @@ -141,37 +151,17 @@ std::shared_ptr ExecNode_ScalarAggregate( keep_alives.push_back(std::move(opts)); } - std::vector targets; + std::vector targets, keys; for (auto&& name : target_names) { targets.emplace_back(std::move(name)); } - return ExecNodeOrStop(compute::MakeScalarAggregateNode( - input.get(), /*label=*/"scalar_agg", std::move(aggregates), std::move(targets), - std::move(out_field_names))); -} - -// [[arrow::export]] -std::shared_ptr ExecNode_GroupByAggregate( - const std::shared_ptr& input, std::vector group_vars, - std::vector agg_srcs, cpp11::list aggregations) { - std::vector aggs; - std::vector> keep_alives; - - for (cpp11::list name_opts : aggregations) { - auto name = cpp11::as_cpp(name_opts[0]); - auto opts = make_compute_options(name, name_opts[1]); - - aggs.push_back(arrow::compute::internal::Aggregate{std::move(name), opts.get()}); - keep_alives.push_back(std::move(opts)); + for (auto&& name : key_names) { + keys.emplace_back(std::move(name)); } - - return ExecNodeOrStop(compute::MakeGroupByNode(input.get(), /*label=*/"group_agg", - /*keys=*/std::move(group_vars), - std::move(agg_srcs), std::move(aggs))); + return MakeExecNodeOrStop( + "aggregate", input->plan(), {input.get()}, + compute::AggregateNodeOptions{std::move(aggregates), std::move(targets), + std::move(out_field_names), std::move(keys)}); } -// Result MakeGroupByNode(ExecNode* input, std::string label, -// std::vector keys, -// std::vector agg_srcs, -// std::vector aggs); #endif From 3c74503864e275774742c4dea40694a34a06f5e1 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 6 Aug 2021 16:31:41 -0400 Subject: [PATCH 3/8] msvc: try ensuring that the generator is const callable --- cpp/src/arrow/compute/exec/sink_node.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 168e4123d14..95e11d0d0c1 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -60,9 +60,9 @@ class SinkNode : public ExecNode { static PushGenerator>::Producer MakeProducer( AsyncGenerator>* out_gen) { - PushGenerator> gen; - auto out = gen.producer(); - *out_gen = std::move(gen); + auto gen = std::make_shared>>(); + auto out = gen->producer(); + *out_gen = [gen] { return (*gen)(); }; return out; } From dec08c3beef0dbaaad654e690d9a7a52f09d5ce1 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 6 Aug 2021 16:48:20 -0400 Subject: [PATCH 4/8] msvc: try explicit construction again --- cpp/src/arrow/compute/exec/sink_node.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 95e11d0d0c1..25154b9107d 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -62,7 +62,7 @@ class SinkNode : public ExecNode { AsyncGenerator>* out_gen) { auto gen = std::make_shared>>(); auto out = gen->producer(); - *out_gen = [gen] { return (*gen)(); }; + *out_gen = AsyncGenerator>([gen] { return (*gen)(); }); return out; } From 64906579048c1a4621561e959a849e7ef3ea8960 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 6 Aug 2021 17:09:58 -0400 Subject: [PATCH 5/8] msvc: try adding const to PushGenerator::operator() --- cpp/src/arrow/compute/exec/sink_node.cc | 6 +++--- cpp/src/arrow/util/async_generator.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 25154b9107d..168e4123d14 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -60,9 +60,9 @@ class SinkNode : public ExecNode { static PushGenerator>::Producer MakeProducer( AsyncGenerator>* out_gen) { - auto gen = std::make_shared>>(); - auto out = gen->producer(); - *out_gen = AsyncGenerator>([gen] { return (*gen)(); }); + PushGenerator> gen; + auto out = gen.producer(); + *out_gen = std::move(gen); return out; } diff --git a/cpp/src/arrow/util/async_generator.h b/cpp/src/arrow/util/async_generator.h index 9d1021edff5..bfe4937cbc6 100644 --- a/cpp/src/arrow/util/async_generator.h +++ b/cpp/src/arrow/util/async_generator.h @@ -850,7 +850,7 @@ class PushGenerator { PushGenerator() : state_(std::make_shared()) {} /// Read an item from the queue - Future operator()() { + Future operator()() const { auto lock = state_->mutex.Lock(); assert(!state_->consumer_fut.has_value()); // Non-reentrant if (!state_->result_q.empty()) { From 82d80a8731428ec064e4ca68c648fb4c53fee655 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 6 Aug 2021 17:11:13 -0400 Subject: [PATCH 6/8] msvc: try explict construction which isn't functional-style cast --- cpp/src/arrow/compute/exec/sink_node.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 168e4123d14..1d987475a5c 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -60,8 +60,9 @@ class SinkNode : public ExecNode { static PushGenerator>::Producer MakeProducer( AsyncGenerator>* out_gen) { - PushGenerator> gen; - auto out = gen.producer(); + PushGenerator> push_gen; + auto out = push_gen.producer(); + AsyncGenerator> gen{std::move(push_gen)}; *out_gen = std::move(gen); return out; } From c44be29686af6fab2132097aa3cbd430d6ac71fe Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 9 Aug 2021 11:22:17 -0400 Subject: [PATCH 7/8] msvc: explicitly convertible wrapper --- cpp/src/arrow/compute/exec/sink_node.cc | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index 1d987475a5c..e4a06e0d224 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -62,8 +62,22 @@ class SinkNode : public ExecNode { AsyncGenerator>* out_gen) { PushGenerator> push_gen; auto out = push_gen.producer(); - AsyncGenerator> gen{std::move(push_gen)}; - *out_gen = std::move(gen); + *out_gen = [push_gen] { + // Awful workaround for MSVC 19.0 (Visual Studio 2015) bug. + // For some types including Future>, + // std::is_convertible::value will be false causing + // SFINAE exclusion of the std::function constructor we need. + // Definining a convertible (but distinct) type soothes the + // faulty trait. + struct ConvertibleToFuture { + operator Future>() && { // NOLINT runtime/explicit + return std::move(ret); + } + Future> ret; + }; + + return ConvertibleToFuture{push_gen()}; + }; return out; } From 257a60b19b89df560e8dc9b0ff096b84733f7047 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 9 Aug 2021 13:12:27 -0400 Subject: [PATCH 8/8] remove unused class --- cpp/src/arrow/compute/exec/aggregate_node.cc | 27 -------------------- 1 file changed, 27 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 79444dc575c..de9078cd07e 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -581,33 +581,6 @@ struct GroupByNode : ExecNode { ExecBatch out_data_; }; -class DefaultExecFactoryRegistry : public ExecFactoryRegistry { - public: - Result GetFactory(const std::string& factory_name) override { - auto it = factories_.find(factory_name); - if (it == factories_.end()) { - return Status::KeyError("ExecNode factory named ", factory_name, - " not present in registry."); - } - return it->second; - } - - Status AddFactory(std::string factory_name, Factory factory) override { - auto it_success = factories_.emplace(std::move(factory_name), std::move(factory)); - - if (!it_success.second) { - const auto& factory_name = it_success.first->first; - return Status::KeyError("ExecNode factory named ", factory_name, - " already registered."); - } - - return Status::OK(); - } - - private: - std::unordered_map factories_; -}; - ExecFactoryRegistry::AddOnLoad kRegisterAggregate( "aggregate", [](ExecPlan* plan, std::vector inputs,