diff --git a/cpp/examples/arrow/execution_plan_documentation_examples.cc b/cpp/examples/arrow/execution_plan_documentation_examples.cc index 10bd19d3044..4aed041be33 100644 --- a/cpp/examples/arrow/execution_plan_documentation_examples.cc +++ b/cpp/examples/arrow/execution_plan_documentation_examples.cc @@ -256,182 +256,122 @@ arrow::Result MakeGroupableBatches(int multiplicity = 1) { return out; } -arrow::Status ExecutePlanAndCollectAsTable( - cp::ExecContext& exec_context, std::shared_ptr plan, - std::shared_ptr schema, - arrow::AsyncGenerator> sink_gen) { - // translate sink_gen (async) to sink_reader (sync) - std::shared_ptr sink_reader = - cp::MakeGeneratorReader(schema, std::move(sink_gen), exec_context.memory_pool()); - - // validate the ExecPlan - ARROW_RETURN_NOT_OK(plan->Validate()); - std::cout << "ExecPlan created : " << plan->ToString() << std::endl; - // start the ExecPlan - ARROW_RETURN_NOT_OK(plan->StartProducing()); - +arrow::Status ExecutePlanAndCollectAsTable(cp::Declaration plan) { // collect sink_reader into a Table std::shared_ptr response_table; - - ARROW_ASSIGN_OR_RAISE(response_table, - arrow::Table::FromRecordBatchReader(sink_reader.get())); + ARROW_ASSIGN_OR_RAISE(response_table, cp::DeclarationToTable(std::move(plan))); std::cout << "Results : " << response_table->ToString() << std::endl; - // stop producing - plan->StopProducing(); - // plan mark finished - auto future = plan->finished(); - return future.status(); + return arrow::Status::OK(); } // (Doc section: Scan Example) /// \brief An example demonstrating a scan and sink node -/// \param exec_context The execution context to run the plan in /// -/// Scan-Sink +/// Scan-Table /// This example shows how scan operation can be applied on a dataset. /// There are operations that can be applied on the scan (project, filter) /// and the input data can be processed. The output is obtained as a table -/// via the sink node. -arrow::Status ScanSinkExample(cp::ExecContext& exec_context) { - // Execution plan created - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +arrow::Status ScanSinkExample() { ARROW_ASSIGN_OR_RAISE(std::shared_ptr dataset, GetDataset()); auto options = std::make_shared(); options->projection = cp::project({}, {}); // create empty projection // construct the scan node - cp::ExecNode* scan; auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - ARROW_ASSIGN_OR_RAISE(scan, - cp::MakeExecNode("scan", plan.get(), {}, scan_node_options)); - - arrow::AsyncGenerator> sink_gen; + cp::Declaration scan{"scan", std::move(scan_node_options)}; - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {scan}, cp::SinkNodeOptions{&sink_gen})); - - return ExecutePlanAndCollectAsTable(exec_context, plan, dataset->schema(), sink_gen); + return ExecutePlanAndCollectAsTable(std::move(scan)); } // (Doc section: Scan Example) // (Doc section: Source Example) /// \brief An example demonstrating a source and sink node -/// \param exec_context The execution context to run the plan in /// -/// Source-Sink Example -/// This example shows how a source and sink can be used -/// in an execution plan. This includes source node receiving data -/// and the sink node emits the data as an output represented in -/// a table. -arrow::Status SourceSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// Source-Table Example +/// This example shows how a custom source can be used +/// in an execution plan. This includes source node using pregenerated +/// data and collecting it into a table. +/// +/// This sort of custom souce is often not needed. In most cases you can +/// use a scan (for a dataset source) or a source like table_source, array_vector_source, +/// exec_batch_source, or record_batch_source (for in-memory data) +arrow::Status SourceSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - arrow::AsyncGenerator> sink_gen; - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - ARROW_ASSIGN_OR_RAISE(cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, source_node_options)); - - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {source}, cp::SinkNodeOptions{&sink_gen})); + cp::Declaration source{"source", std::move(source_node_options)}; - return ExecutePlanAndCollectAsTable(exec_context, plan, basic_data.schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(source)); } // (Doc section: Source Example) // (Doc section: Table Source Example) /// \brief An example showing a table source node -/// \param exec_context The execution context to run the plan in /// -/// TableSource-Sink Example -/// This example shows how a table_source and sink can be used +/// TableSource-Table Example +/// This example shows how a table_source can be used /// in an execution plan. This includes a table source node -/// receiving data from a table and the sink node emits -/// the data to a generator which we collect into a table. -arrow::Status TableSourceSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// receiving data from a table. This plan simply collects the +/// data back into a table but nodes could be added that modify +/// or transform the data as well (as is shown in later examples) +arrow::Status TableSourceSinkExample() { ARROW_ASSIGN_OR_RAISE(auto table, GetTable()); arrow::AsyncGenerator> sink_gen; int max_batch_size = 2; auto table_source_options = cp::TableSourceNodeOptions{table, max_batch_size}; - ARROW_ASSIGN_OR_RAISE( - cp::ExecNode * source, - cp::MakeExecNode("table_source", plan.get(), {}, table_source_options)); - - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {source}, cp::SinkNodeOptions{&sink_gen})); + cp::Declaration source{"table_source", std::move(table_source_options)}; - return ExecutePlanAndCollectAsTable(exec_context, plan, table->schema(), sink_gen); + return ExecutePlanAndCollectAsTable(std::move(source)); } // (Doc section: Table Source Example) // (Doc section: Filter Example) /// \brief An example showing a filter node -/// \param exec_context The execution context to run the plan in /// -/// Source-Filter-Sink +/// Source-Filter-Table /// This example shows how a filter can be used in an execution plan, -/// along with the source and sink operations. The output from the -/// exeuction plan is obtained as a table via the sink node. -arrow::Status ScanFilterSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// to filter data from a source. The output from the exeuction plan +/// is collected into a table. +arrow::Status ScanFilterSinkExample() { ARROW_ASSIGN_OR_RAISE(std::shared_ptr dataset, GetDataset()); auto options = std::make_shared(); // specify the filter. This filter removes all rows where the // value of the "a" column is greater than 3. - cp::Expression filter_opt = cp::greater(cp::field_ref("a"), cp::literal(3)); + cp::Expression filter_expr = cp::greater(cp::field_ref("a"), cp::literal(3)); // set filter for scanner : on-disk / push-down filtering. // This step can be skipped if you are not reading from disk. - options->filter = filter_opt; + options->filter = filter_expr; // empty projection options->projection = cp::project({}, {}); // construct the scan node std::cout << "Initialized Scanning Options" << std::endl; - cp::ExecNode* scan; - auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; std::cout << "Scan node options created" << std::endl; - ARROW_ASSIGN_OR_RAISE(scan, - cp::MakeExecNode("scan", plan.get(), {}, scan_node_options)); + cp::Declaration scan{"scan", std::move(scan_node_options)}; // pipe the scan node into the filter node // Need to set the filter in scan node options and filter node options. // At scan node it is used for on-disk / push-down filtering. // At filter node it is used for in-memory filtering. - cp::ExecNode* filter; - ARROW_ASSIGN_OR_RAISE(filter, cp::MakeExecNode("filter", plan.get(), {scan}, - cp::FilterNodeOptions{filter_opt})); + cp::Declaration filter{ + "filter", {std::move(scan)}, cp::FilterNodeOptions(std::move(filter_expr))}; - // finally, pipe the filter node into a sink node - arrow::AsyncGenerator> sink_gen; - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {filter}, cp::SinkNodeOptions{&sink_gen})); - - return ExecutePlanAndCollectAsTable(exec_context, plan, dataset->schema(), sink_gen); + return ExecutePlanAndCollectAsTable(std::move(filter)); } // (Doc section: Filter Example) @@ -439,16 +379,12 @@ arrow::Status ScanFilterSinkExample(cp::ExecContext& exec_context) { // (Doc section: Project Example) /// \brief An example showing a project node -/// \param exec_context The execution context to run the plan in /// -/// Scan-Project-Sink -/// This example shows how Scan operation can be used to load the data -/// into the execution plan, how project operation can be applied on the -/// data stream and how the output is obtained as a table via the sink node. -arrow::Status ScanProjectSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// Scan-Project-Table +/// This example shows how a Scan operation can be used to load the data +/// into the execution plan, how a project operation can be applied on the +/// data stream and how the output is collected into a table +arrow::Status ScanProjectSinkExample() { ARROW_ASSIGN_OR_RAISE(std::shared_ptr dataset, GetDataset()); auto options = std::make_shared(); @@ -456,26 +392,13 @@ arrow::Status ScanProjectSinkExample(cp::ExecContext& exec_context) { cp::Expression a_times_2 = cp::call("multiply", {cp::field_ref("a"), cp::literal(2)}); options->projection = cp::project({}, {}); - cp::ExecNode* scan; - auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - ARROW_ASSIGN_OR_RAISE(scan, - cp::MakeExecNode("scan", plan.get(), {}, scan_node_options)); - - cp::ExecNode* project; - ARROW_ASSIGN_OR_RAISE(project, cp::MakeExecNode("project", plan.get(), {scan}, - cp::ProjectNodeOptions{{a_times_2}})); - // schema after projection => multiply(a, 2): int64 - std::cout << "Schema after projection : \n" - << project->output_schema()->ToString() << std::endl; - - arrow::AsyncGenerator> sink_gen; - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {project}, cp::SinkNodeOptions{&sink_gen})); - auto schema = arrow::schema({arrow::field("a * 2", arrow::int32())}); + cp::Declaration scan{"scan", std::move(scan_node_options)}; + cp::Declaration project{ + "project", {std::move(scan)}, cp::ProjectNodeOptions({a_times_2})}; - return ExecutePlanAndCollectAsTable(exec_context, plan, schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(project)); } // (Doc section: Project Example) @@ -483,98 +406,70 @@ arrow::Status ScanProjectSinkExample(cp::ExecContext& exec_context) { // (Doc section: Scalar Aggregate Example) /// \brief An example showing an aggregation node to aggregate an entire table -/// \param exec_context The execution context to run the plan in /// -/// Source-Aggregation-Sink +/// Source-Aggregation-Table /// This example shows how an aggregation operation can be applied on a -/// execution plan resulting a scalar output. The source node loads the +/// execution plan resulting in a scalar output. The source node loads the /// data and the aggregation (counting unique types in column 'a') -/// is applied on this data. The output is obtained from the sink node as a table. -arrow::Status SourceScalarAggregateSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// is applied on this data. The output is collected into a table (that will +/// have exactly one row) +arrow::Status SourceScalarAggregateSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - arrow::AsyncGenerator> sink_gen; - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - ARROW_ASSIGN_OR_RAISE(cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, source_node_options)); + cp::Declaration source{"source", std::move(source_node_options)}; auto aggregate_options = cp::AggregateNodeOptions{/*aggregates=*/{{"sum", nullptr, "a", "sum(a)"}}}; - ARROW_ASSIGN_OR_RAISE( - cp::ExecNode * aggregate, - cp::MakeExecNode("aggregate", plan.get(), {source}, std::move(aggregate_options))); - - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {aggregate}, cp::SinkNodeOptions{&sink_gen})); - auto schema = arrow::schema({arrow::field("sum(a)", arrow::int32())}); + cp::Declaration aggregate{ + "aggregate", {std::move(source)}, std::move(aggregate_options)}; - return ExecutePlanAndCollectAsTable(exec_context, plan, schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(aggregate)); } // (Doc section: Scalar Aggregate Example) // (Doc section: Group Aggregate Example) /// \brief An example showing an aggregation node to perform a group-by operation -/// \param exec_context The execution context to run the plan in /// -/// Source-Aggregation-Sink +/// Source-Aggregation-Table /// This example shows how an aggregation operation can be applied on a -/// execution plan resulting a grouped output. The source node loads the +/// execution plan resulting in grouped output. The source node loads the /// data and the aggregation (counting unique types in column 'a') is -/// applied on this data. The output is obtained from the sink node as a table. -arrow::Status SourceGroupAggregateSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +/// applied on this data. The output is collected into a table that will contain +/// one row for each unique combination of group keys. +arrow::Status SourceGroupAggregateSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); arrow::AsyncGenerator> sink_gen; auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - ARROW_ASSIGN_OR_RAISE(cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, source_node_options)); + cp::Declaration source{"source", std::move(source_node_options)}; auto options = std::make_shared(cp::CountOptions::ONLY_VALID); auto aggregate_options = cp::AggregateNodeOptions{/*aggregates=*/{{"hash_count", options, "a", "count(a)"}}, /*keys=*/{"b"}}; - ARROW_ASSIGN_OR_RAISE( - cp::ExecNode * aggregate, - cp::MakeExecNode("aggregate", plan.get(), {source}, aggregate_options)); - - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {aggregate}, cp::SinkNodeOptions{&sink_gen})); - auto schema = arrow::schema({ - arrow::field("count(a)", arrow::int32()), - arrow::field("b", arrow::boolean()), - }); + cp::Declaration aggregate{ + "aggregate", {std::move(source)}, std::move(aggregate_options)}; - return ExecutePlanAndCollectAsTable(exec_context, plan, schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(aggregate)); } // (Doc section: Group Aggregate Example) // (Doc section: ConsumingSink Example) /// \brief An example showing a consuming sink node -/// \param exec_context The execution context to run the plan in /// /// Source-Consuming-Sink /// This example shows how the data can be consumed within the execution plan /// by using a ConsumingSink node. There is no data output from this execution plan. -arrow::Status SourceConsumingSinkExample(cp::ExecContext& exec_context) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +arrow::Status SourceConsumingSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - ARROW_ASSIGN_OR_RAISE(cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, source_node_options)); + cp::Declaration source{"source", std::move(source_node_options)}; std::atomic batches_seen{0}; arrow::Future<> finish = arrow::Future<>::Make(); @@ -585,6 +480,10 @@ arrow::Status SourceConsumingSinkExample(cp::ExecContext& exec_context) { arrow::Status Init(const std::shared_ptr& schema, cp::BackpressureControl* backpressure_control, cp::ExecPlan* plan) override { + // This will be called as the plan is started (before the first call to Consume) + // and provides the schema of the data coming into the node, controls for pausing / + // resuming input, and a pointer to the plan itself which can be used to access + // other utilities such as the thread indexer or async task scheduler. return arrow::Status::OK(); } @@ -593,7 +492,11 @@ arrow::Status SourceConsumingSinkExample(cp::ExecContext& exec_context) { return arrow::Status::OK(); } - arrow::Future<> Finish() override { return finish; } + arrow::Future<> Finish() override { + // Here you can perform whatever (possibly async) cleanup is needed, e.g. closing + // output file handles and flushing remaining work + return arrow::Future<>::MakeFinished(); + } std::atomic* batches_seen; arrow::Future<> finish; @@ -601,41 +504,60 @@ arrow::Status SourceConsumingSinkExample(cp::ExecContext& exec_context) { std::shared_ptr consumer = std::make_shared(&batches_seen, finish); - cp::ExecNode* consuming_sink; + cp::Declaration consuming_sink{"consuming_sink", + {std::move(source)}, + cp::ConsumingSinkNodeOptions(std::move(consumer))}; - ARROW_ASSIGN_OR_RAISE(consuming_sink, - MakeExecNode("consuming_sink", plan.get(), {source}, - cp::ConsumingSinkNodeOptions(consumer))); + // Since we are consuming the data within the plan there is no output and we simply + // run the plan to completion instead of collecting into a table. + ARROW_RETURN_NOT_OK(cp::DeclarationToStatus(std::move(consuming_sink))); - ARROW_RETURN_NOT_OK(consuming_sink->Validate()); - - ARROW_RETURN_NOT_OK(plan->Validate()); - std::cout << "Exec Plan created: " << plan->ToString() << std::endl; - // plan start producing - ARROW_RETURN_NOT_OK(plan->StartProducing()); - // Source should finish fairly quickly - ARROW_RETURN_NOT_OK(source->finished().status()); - std::cout << "Source Finished!" << std::endl; - // Mark consumption complete, plan should finish - finish.MarkFinished(arrow::Status::OK()); - ARROW_RETURN_NOT_OK(plan->finished().status()); + std::cout << "The consuming sink node saw " << batches_seen.load() << " batches" + << std::endl; return arrow::Status::OK(); } // (Doc section: ConsumingSink Example) // (Doc section: OrderBySink Example) +arrow::Status ExecutePlanAndCollectAsTableWithCustomSink( + std::shared_ptr plan, std::shared_ptr schema, + arrow::AsyncGenerator> sink_gen) { + // translate sink_gen (async) to sink_reader (sync) + std::shared_ptr sink_reader = + cp::MakeGeneratorReader(schema, std::move(sink_gen), arrow::default_memory_pool()); + + // validate the ExecPlan + ARROW_RETURN_NOT_OK(plan->Validate()); + std::cout << "ExecPlan created : " << plan->ToString() << std::endl; + // start the ExecPlan + ARROW_RETURN_NOT_OK(plan->StartProducing()); + + // collect sink_reader into a Table + std::shared_ptr response_table; + + ARROW_ASSIGN_OR_RAISE(response_table, + arrow::Table::FromRecordBatchReader(sink_reader.get())); + + std::cout << "Results : " << response_table->ToString() << std::endl; + + // stop producing + plan->StopProducing(); + // plan mark finished + auto future = plan->finished(); + return future.status(); +} + /// \brief An example showing an order-by node -/// \param exec_context The execution context to run the plan in /// /// Source-OrderBy-Sink /// In this example, the data enters through the source node /// and the data is ordered in the sink node. The order can be /// ASCENDING or DESCENDING and it is configurable. The output /// is obtained as a table from the sink node. -arrow::Status SourceOrderBySinkExample(cp::ExecContext& exec_context) { +arrow::Status SourceOrderBySinkExample() { ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); + cp::ExecPlan::Make(*cp::threaded_exec_context())); ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeSortTestBasicBatches()); @@ -652,7 +574,7 @@ arrow::Status SourceOrderBySinkExample(cp::ExecContext& exec_context) { cp::OrderBySinkNodeOptions{ cp::SortOptions{{cp::SortKey{"a", cp::SortOrder::Descending}}}, &sink_gen})); - return ExecutePlanAndCollectAsTable(exec_context, plan, basic_data.schema, sink_gen); + return ExecutePlanAndCollectAsTableWithCustomSink(plan, basic_data.schema, sink_gen); } // (Doc section: OrderBySink Example) @@ -660,44 +582,26 @@ arrow::Status SourceOrderBySinkExample(cp::ExecContext& exec_context) { // (Doc section: HashJoin Example) /// \brief An example showing a hash join node -/// \param exec_context The execution context to run the plan in /// -/// Source-HashJoin-Sink +/// Source-HashJoin-Table /// This example shows how source node gets the data and how a self-join /// is applied on the data. The join options are configurable. The output -/// is obtained as a table via the sink node. -arrow::Status SourceHashJoinSinkExample(cp::ExecContext& exec_context) { +/// is collected into a table. +arrow::Status SourceHashJoinSinkExample() { ARROW_ASSIGN_OR_RAISE(auto input, MakeGroupableBatches()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - - arrow::AsyncGenerator> sink_gen; - cp::ExecNode* left_source; - cp::ExecNode* right_source; - for (auto source : {&left_source, &right_source}) { - ARROW_ASSIGN_OR_RAISE(*source, - MakeExecNode("source", plan.get(), {}, - cp::SourceNodeOptions{input.schema, input.gen()})); - } + cp::Declaration left{"source", cp::SourceNodeOptions{input.schema, input.gen()}}; + cp::Declaration right{"source", cp::SourceNodeOptions{input.schema, input.gen()}}; cp::HashJoinNodeOptions join_opts{ cp::JoinType::INNER, /*left_keys=*/{"str"}, /*right_keys=*/{"str"}, cp::literal(true), "l_", "r_"}; - ARROW_ASSIGN_OR_RAISE( - auto hashjoin, - cp::MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); + cp::Declaration hashjoin{ + "hashjoin", {std::move(left), std::move(right)}, std::move(join_opts)}; - ARROW_RETURN_NOT_OK( - cp::MakeExecNode("sink", plan.get(), {hashjoin}, cp::SinkNodeOptions{&sink_gen})); - // expected columns i32, str, l_str, r_str - auto schema = arrow::schema( - {arrow::field("i32", arrow::int32()), arrow::field("str", arrow::utf8()), - arrow::field("l_str", arrow::utf8()), arrow::field("r_str", arrow::utf8())}); - - return ExecutePlanAndCollectAsTable(exec_context, plan, schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(hashjoin)); } // (Doc section: HashJoin Example) @@ -705,16 +609,15 @@ arrow::Status SourceHashJoinSinkExample(cp::ExecContext& exec_context) { // (Doc section: KSelect Example) /// \brief An example showing a select-k node -/// \param exec_context The execution context to run the plan in /// /// Source-KSelect /// This example shows how K number of elements can be selected /// either from the top or bottom. The output node is a modified /// sink node where output can be obtained as a table. -arrow::Status SourceKSelectExample(cp::ExecContext& exec_context) { +arrow::Status SourceKSelectExample() { ARROW_ASSIGN_OR_RAISE(auto input, MakeGroupableBatches()); ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); + cp::ExecPlan::Make(*cp::threaded_exec_context())); arrow::AsyncGenerator> sink_gen; ARROW_ASSIGN_OR_RAISE( @@ -730,7 +633,7 @@ arrow::Status SourceKSelectExample(cp::ExecContext& exec_context) { auto schema = arrow::schema( {arrow::field("i32", arrow::int32()), arrow::field("str", arrow::utf8())}); - return ExecutePlanAndCollectAsTable(exec_context, plan, schema, sink_gen); + return ExecutePlanAndCollectAsTableWithCustomSink(plan, schema, sink_gen); } // (Doc section: KSelect Example) @@ -738,29 +641,21 @@ arrow::Status SourceKSelectExample(cp::ExecContext& exec_context) { // (Doc section: Write Example) /// \brief An example showing a write node -/// \param exec_context The execution context to run the plan in /// \param file_path The destination to write to /// /// Scan-Filter-Write /// This example shows how scan node can be used to load the data /// and after processing how it can be written to disk. -arrow::Status ScanFilterWriteExample(cp::ExecContext& exec_context, - const std::string& file_path) { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - +arrow::Status ScanFilterWriteExample(const std::string& file_path) { ARROW_ASSIGN_OR_RAISE(std::shared_ptr dataset, GetDataset()); auto options = std::make_shared(); // empty projection options->projection = cp::project({}, {}); - cp::ExecNode* scan; - auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - ARROW_ASSIGN_OR_RAISE(scan, - cp::MakeExecNode("scan", plan.get(), {}, scan_node_options)); + cp::Declaration scan{"scan", std::move(scan_node_options)}; arrow::AsyncGenerator> sink_gen; @@ -793,15 +688,13 @@ arrow::Status ScanFilterWriteExample(cp::ExecContext& exec_context, arrow::dataset::WriteNodeOptions write_node_options{write_options}; - ARROW_RETURN_NOT_OK(cp::MakeExecNode("write", plan.get(), {scan}, write_node_options)); + cp::Declaration write{"write", {std::move(scan)}, std::move(write_node_options)}; - ARROW_RETURN_NOT_OK(plan->Validate()); - std::cout << "Execution Plan Created : " << plan->ToString() << std::endl; - // // // start the ExecPlan - ARROW_RETURN_NOT_OK(plan->StartProducing()); - auto future = plan->finished(); - ARROW_RETURN_NOT_OK(future.status()); - future.Wait(); + // Since the write node has no output we simply run the plan to completion and the + // data should be written + ARROW_RETURN_NOT_OK(cp::DeclarationToStatus(std::move(write))); + + std::cout << "Dataset written to " << base_path << std::endl; return arrow::Status::OK(); } @@ -810,41 +703,23 @@ arrow::Status ScanFilterWriteExample(cp::ExecContext& exec_context, // (Doc section: Union Example) /// \brief An example showing a union node -/// \param exec_context The execution context to run the plan in /// -/// Source-Union-Sink +/// Source-Union-Table /// This example shows how a union operation can be applied on two -/// data sources. The output is obtained as a table via the sink -/// node. -arrow::Status SourceUnionSinkExample(cp::ExecContext& exec_context) { +/// data sources. The output is collected into a table. +arrow::Status SourceUnionSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - arrow::AsyncGenerator> sink_gen; - - cp::Declaration union_node{"union", cp::ExecNodeOptions{}}; cp::Declaration lhs{"source", cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; lhs.label = "lhs"; cp::Declaration rhs{"source", cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; rhs.label = "rhs"; - union_node.inputs.emplace_back(lhs); - union_node.inputs.emplace_back(rhs); + cp::Declaration union_plan{ + "union", {std::move(lhs), std::move(rhs)}, cp::ExecNodeOptions{}}; - cp::CountOptions options(cp::CountOptions::ONLY_VALID); - ARROW_ASSIGN_OR_RAISE( - auto declr, cp::Declaration::Sequence({ - union_node, - {"sink", cp::SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); - - ARROW_RETURN_NOT_OK(declr->Validate()); - - ARROW_RETURN_NOT_OK(plan->Validate()); - return ExecutePlanAndCollectAsTable(exec_context, plan, basic_data.schema, sink_gen); + return ExecutePlanAndCollectAsTable(std::move(union_plan)); } // (Doc section: Union Example) @@ -852,16 +727,15 @@ arrow::Status SourceUnionSinkExample(cp::ExecContext& exec_context) { // (Doc section: Table Sink Example) /// \brief An example showing a table sink node -/// \param exec_context The execution context to run the plan in /// /// TableSink Example /// This example shows how a table_sink can be used /// in an execution plan. This includes a source node /// receiving data as batches and the table sink node /// which emits the output as a table. -arrow::Status TableSinkExample(cp::ExecContext& exec_context) { +arrow::Status TableSinkExample() { ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); + cp::ExecPlan::Make(*cp::threaded_exec_context())); ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); @@ -907,7 +781,7 @@ enum ExampleMode { }; int main(int argc, char** argv) { - if (argc < 2) { + if (argc < 3) { // Fake success for CI purposes. return EXIT_SUCCESS; } @@ -917,64 +791,62 @@ int main(int argc, char** argv) { arrow::Status status; // ensure arrow::dataset node factories are in the registry arrow::dataset::internal::Initialize(); - // execution context - cp::ExecContext exec_context; switch (mode) { case SOURCE_SINK: PrintBlock("Source Sink Example"); - status = SourceSinkExample(exec_context); + status = SourceSinkExample(); break; case TABLE_SOURCE_SINK: PrintBlock("Table Source Sink Example"); - status = TableSourceSinkExample(exec_context); + status = TableSourceSinkExample(); break; case SCAN: PrintBlock("Scan Example"); - status = ScanSinkExample(exec_context); + status = ScanSinkExample(); break; case FILTER: PrintBlock("Filter Example"); - status = ScanFilterSinkExample(exec_context); + status = ScanFilterSinkExample(); break; case PROJECT: PrintBlock("Project Example"); - status = ScanProjectSinkExample(exec_context); + status = ScanProjectSinkExample(); break; case GROUP_AGGREGATION: PrintBlock("Aggregate Example"); - status = SourceGroupAggregateSinkExample(exec_context); + status = SourceGroupAggregateSinkExample(); break; case SCALAR_AGGREGATION: PrintBlock("Aggregate Example"); - status = SourceScalarAggregateSinkExample(exec_context); + status = SourceScalarAggregateSinkExample(); break; case CONSUMING_SINK: PrintBlock("Consuming-Sink Example"); - status = SourceConsumingSinkExample(exec_context); + status = SourceConsumingSinkExample(); break; case ORDER_BY_SINK: PrintBlock("OrderBy Example"); - status = SourceOrderBySinkExample(exec_context); + status = SourceOrderBySinkExample(); break; case HASHJOIN: PrintBlock("HashJoin Example"); - status = SourceHashJoinSinkExample(exec_context); + status = SourceHashJoinSinkExample(); break; case KSELECT: PrintBlock("KSelect Example"); - status = SourceKSelectExample(exec_context); + status = SourceKSelectExample(); break; case WRITE: PrintBlock("Write Example"); - status = ScanFilterWriteExample(exec_context, base_save_path); + status = ScanFilterWriteExample(base_save_path); break; case UNION: PrintBlock("Union Example"); - status = SourceUnionSinkExample(exec_context); + status = SourceUnionSinkExample(); break; case TABLE_SOURCE_TABLE_SINK: PrintBlock("TableSink Example"); - status = TableSinkExample(exec_context); + status = TableSinkExample(); break; default: break; diff --git a/cpp/examples/arrow/join_example.cc b/cpp/examples/arrow/join_example.cc index c29f5e5dbbd..eb7a8678a6e 100644 --- a/cpp/examples/arrow/join_example.cc +++ b/cpp/examples/arrow/join_example.cc @@ -82,18 +82,8 @@ arrow::Result> CreateDataSetFromCSVData } arrow::Status DoHashJoin() { - cp::ExecContext exec_context; - arrow::dataset::internal::Initialize(); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(&exec_context)); - - arrow::AsyncGenerator> sink_gen; - - cp::ExecNode* left_source; - cp::ExecNode* right_source; - ARROW_ASSIGN_OR_RAISE(auto l_dataset, CreateDataSetFromCSVData(true)); ARROW_ASSIGN_OR_RAISE(auto r_dataset, CreateDataSetFromCSVData(false)); @@ -111,10 +101,8 @@ arrow::Status DoHashJoin() { auto l_scan_node_options = arrow::dataset::ScanNodeOptions{l_dataset, l_options}; auto r_scan_node_options = arrow::dataset::ScanNodeOptions{r_dataset, r_options}; - ARROW_ASSIGN_OR_RAISE(left_source, - cp::MakeExecNode("scan", plan.get(), {}, l_scan_node_options)); - ARROW_ASSIGN_OR_RAISE(right_source, - cp::MakeExecNode("scan", plan.get(), {}, r_scan_node_options)); + arrow::compute::Declaration left{"scan", std::move(l_scan_node_options)}; + arrow::compute::Declaration right{"scan", std::move(r_scan_node_options)}; arrow::compute::HashJoinNodeOptions join_opts{arrow::compute::JoinType::INNER, /*in_left_keys=*/{"lkey"}, @@ -123,26 +111,12 @@ arrow::Status DoHashJoin() { /*output_suffix_for_left*/ "_l", /*output_suffix_for_right*/ "_r"}; - ARROW_ASSIGN_OR_RAISE( - auto hashjoin, - cp::MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); + arrow::compute::Declaration hashjoin{ + "hashjoin", {std::move(left), std::move(right)}, join_opts}; - ARROW_ASSIGN_OR_RAISE(std::ignore, cp::MakeExecNode("sink", plan.get(), {hashjoin}, - cp::SinkNodeOptions{&sink_gen})); // expected columns l_a, l_b - std::shared_ptr sink_reader = cp::MakeGeneratorReader( - hashjoin->output_schema(), std::move(sink_gen), exec_context.memory_pool()); - - // validate the ExecPlan - ARROW_RETURN_NOT_OK(plan->Validate()); - // start the ExecPlan - ARROW_RETURN_NOT_OK(plan->StartProducing()); - - // collect sink_reader into a Table - std::shared_ptr response_table; - - ARROW_ASSIGN_OR_RAISE(response_table, - arrow::Table::FromRecordBatchReader(sink_reader.get())); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr response_table, + arrow::compute::DeclarationToTable(std::move(hashjoin))); std::cout << "Results : " << response_table->ToString() << std::endl; diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index 8d3dcf0f2cd..ee02b26845b 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -48,6 +48,7 @@ #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" +#include "arrow/util/thread_pool.h" #include "arrow/util/vector.h" namespace arrow { @@ -56,6 +57,7 @@ using internal::BitmapAnd; using internal::checked_cast; using internal::CopyBitmap; using internal::CpuInfo; +using internal::GetCpuThreadPool; namespace compute { @@ -64,6 +66,11 @@ ExecContext* default_exec_context() { return &default_ctx; } +ExecContext* threaded_exec_context() { + static ExecContext threaded_ctx(default_memory_pool(), GetCpuThreadPool()); + return &threaded_ctx; +} + ExecBatch::ExecBatch(const RecordBatch& batch) : values(batch.num_columns()), length(batch.num_rows()) { auto columns = batch.column_data(); diff --git a/cpp/src/arrow/compute/exec/asof_join_benchmark.cc b/cpp/src/arrow/compute/exec/asof_join_benchmark.cc index a0362eb1ba8..5890e10c206 100644 --- a/cpp/src/arrow/compute/exec/asof_join_benchmark.cc +++ b/cpp/src/arrow/compute/exec/asof_join_benchmark.cc @@ -54,7 +54,6 @@ static void TableJoinOverhead(benchmark::State& state, TableGenerationProperties right_table_properties, int batch_size, int num_right_tables, std::string factory_name, ExecNodeOptions& options) { - ExecContext ctx(default_memory_pool(), nullptr); left_table_properties.column_prefix = "lt"; left_table_properties.seed = 0; ASSERT_OK_AND_ASSIGN(TableStats left_table_stats, MakeTable(left_table_properties)); @@ -75,23 +74,18 @@ static void TableJoinOverhead(benchmark::State& state, for (auto _ : state) { state.PauseTiming(); - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - ExecPlan::Make(&ctx)); - std::vector input_nodes = {*arrow::compute::MakeExecNode( - "table_source", plan.get(), {}, + std::vector input_nodes = {Declaration( + "table_source", arrow::compute::TableSourceNodeOptions(left_table_stats.table, batch_size))}; input_nodes.reserve(right_input_tables.size() + 1); for (TableStats table_stats : right_input_tables) { - input_nodes.push_back(*arrow::compute::MakeExecNode( - "table_source", plan.get(), {}, + input_nodes.push_back(Declaration( + "table_source", arrow::compute::TableSourceNodeOptions(table_stats.table, batch_size))); } - ASSERT_OK_AND_ASSIGN(arrow::compute::ExecNode * join_node, - MakeExecNode(factory_name, plan.get(), input_nodes, options)); - AsyncGenerator> sink_gen; - ASSERT_OK(MakeExecNode("sink", plan.get(), {join_node}, SinkNodeOptions{&sink_gen})); + Declaration join_node{factory_name, {input_nodes}, options}; state.ResumeTiming(); - ASSERT_FINISHES_OK(StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK(DeclarationToStatus(std::move(join_node))); } state.counters["input_rows_per_second"] = benchmark::Counter( @@ -104,7 +98,7 @@ static void TableJoinOverhead(benchmark::State& state, benchmark::Counter::kIsRate); state.counters["maximum_peak_memory"] = - benchmark::Counter(static_cast(ctx.memory_pool()->max_memory())); + benchmark::Counter(static_cast(default_memory_pool()->max_memory())); } static void AsOfJoinOverhead(benchmark::State& state) { diff --git a/cpp/src/arrow/compute/exec/asof_join_node.cc b/cpp/src/arrow/compute/exec/asof_join_node.cc index 979012e63e4..83bbf5df4ca 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node.cc +++ b/cpp/src/arrow/compute/exec/asof_join_node.cc @@ -807,10 +807,36 @@ class AsofJoinNode : public ExecNode { } } - void Process() { + template + struct Defer { + Callable callable; + explicit Defer(Callable callable) : callable(std::move(callable)) {} + ~Defer() noexcept { callable(); } + }; + + void EndFromProcessThread() { + // We must spawn a new task to transfer off the process thread when + // marking this finished. Otherwise there is a chance that doing so could + // mark the plan finished which may destroy the plan which will destroy this + // node which will cause us to join on ourselves. + ErrorIfNotOk(plan_->query_context()->executor()->Spawn([this] { + Defer cleanup([this]() { finished_.MarkFinished(); }); + outputs_[0]->InputFinished(this, batches_produced_); + })); + } + + bool CheckEnded() { + if (state_.at(0)->Finished()) { + EndFromProcessThread(); + return false; + } + return true; + } + + bool Process() { std::lock_guard guard(gate_); - if (finished_.is_finished()) { - return; + if (!CheckEnded()) { + return false; } // Process batches while we have data @@ -825,7 +851,8 @@ class AsofJoinNode : public ExecNode { outputs_[0]->InputReceived(this, std::move(out_b)); } else { ErrorIfNotOk(result.status()); - return; + EndFromProcessThread(); + return false; } } @@ -834,18 +861,24 @@ class AsofJoinNode : public ExecNode { // // It may happen here in cases where InputFinished was called before we were finished // producing results (so we didn't know the output size at that time) - if (state_.at(0)->Finished()) { - outputs_[0]->InputFinished(this, batches_produced_); - finished_.MarkFinished(); + if (!CheckEnded()) { + return false; } + + // There is no more we can do now but there is still work remaining for later when + // more data arrives. + return true; } void ProcessThread() { for (;;) { if (!process_.Pop()) { + EndFromProcessThread(); + return; + } + if (!Process()) { return; } - Process(); } } @@ -1120,10 +1153,7 @@ class AsofJoinNode : public ExecNode { // finished. process_.Push(true); } - Status StartProducing() override { - finished_ = arrow::Future<>::Make(); - return Status::OK(); - } + Status StartProducing() override { return Status::OK(); } void PauseProducing(ExecNode* output, int32_t counter) override {} void ResumeProducing(ExecNode* output, int32_t counter) override {} void StopProducing(ExecNode* output) override { @@ -1137,7 +1167,6 @@ class AsofJoinNode : public ExecNode { arrow::Future<> finished() override { return finished_; } private: - arrow::Future<> finished_; std::vector indices_of_on_key_; std::vector> indices_of_by_key_; std::vector> key_hashers_; @@ -1176,9 +1205,7 @@ AsofJoinNode::AsofJoinNode(ExecPlan* plan, NodeVector inputs, may_rehash_(may_rehash), tolerance_(tolerance), process_(), - process_thread_(&AsofJoinNode::ProcessThreadWrapper, this) { - finished_ = arrow::Future<>::MakeFinished(); -} + process_thread_(&AsofJoinNode::ProcessThreadWrapper, this) {} namespace internal { void RegisterAsofJoinNode(ExecFactoryRegistry* registry) { diff --git a/cpp/src/arrow/compute/exec/asof_join_node_test.cc b/cpp/src/arrow/compute/exec/asof_join_node_test.cc index 31bc094c52e..c865f9f38f8 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/asof_join_node_test.cc @@ -223,9 +223,6 @@ void CheckRunOutput(const BatchesWithSchema& l_batches, const BatchesWithSchema& r1_batches, const BatchesWithSchema& exp_batches, const AsofJoinNodeOptions join_options) { - auto exec_ctx = std::make_unique(default_memory_pool(), nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - Declaration join{"asofjoin", join_options}; join.inputs.emplace_back(Declaration{ @@ -235,21 +232,12 @@ void CheckRunOutput(const BatchesWithSchema& l_batches, join.inputs.emplace_back(Declaration{ "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}}); - AsyncGenerator> sink_gen; - - ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}}) - .AddToPlan(plan.get())); - - ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); - for (auto batch : res) { - ASSERT_EQ(exp_batches.schema->num_fields(), batch.values.size()); - } + ASSERT_OK_AND_ASSIGN(auto res_table, + DeclarationToTable(std::move(join), /*use_threads=*/false)); ASSERT_OK_AND_ASSIGN(auto exp_table, TableFromExecBatches(exp_batches.schema, exp_batches.batches)); - ASSERT_OK_AND_ASSIGN(auto res_table, TableFromExecBatches(exp_batches.schema, res)); - AssertTablesEqual(*exp_table, *res_table, /*same_chunk_layout=*/true, /*flatten=*/true); } @@ -270,8 +258,7 @@ void DoInvalidPlanTest(const BatchesWithSchema& l_batches, const AsofJoinNodeOptions& join_options, const std::string& expected_error_str, bool fail_on_plan_creation = false) { - ExecContext exec_ctx; - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_ctx)); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(*threaded_exec_context())); Declaration join{"asofjoin", join_options}; join.inputs.emplace_back(Declaration{ diff --git a/cpp/src/arrow/compute/exec/benchmark_util.cc b/cpp/src/arrow/compute/exec/benchmark_util.cc index dcc7ca6e165..3c4dda2992a 100644 --- a/cpp/src/arrow/compute/exec/benchmark_util.cc +++ b/cpp/src/arrow/compute/exec/benchmark_util.cc @@ -35,7 +35,6 @@ namespace compute { // calling InputFinished and InputReceived. Status BenchmarkIsolatedNodeOverhead(benchmark::State& state, - arrow::compute::ExecContext ctx, arrow::compute::Expression expr, int32_t num_batches, int32_t batch_size, arrow::compute::BatchesWithSchema data, @@ -46,7 +45,7 @@ Status BenchmarkIsolatedNodeOverhead(benchmark::State& state, AsyncGenerator> sink_gen; ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - arrow::compute::ExecPlan::Make(&ctx)); + arrow::compute::ExecPlan::Make()); // Source and sink nodes have no effect on the benchmark. // Used for dummy purposes as they are referenced in InputReceived and InputFinished. ARROW_ASSIGN_OR_RAISE(arrow::compute::ExecNode * source_node, @@ -113,13 +112,13 @@ Status BenchmarkIsolatedNodeOverhead(benchmark::State& state, // a source -> node_declarations -> sink sequence. Status BenchmarkNodeOverhead( - benchmark::State& state, arrow::compute::ExecContext ctx, int32_t num_batches, - int32_t batch_size, arrow::compute::BatchesWithSchema data, + benchmark::State& state, int32_t num_batches, int32_t batch_size, + arrow::compute::BatchesWithSchema data, std::vector& node_declarations) { for (auto _ : state) { state.PauseTiming(); ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - arrow::compute::ExecPlan::Make(&ctx)); + arrow::compute::ExecPlan::Make()); AsyncGenerator> sink_gen; arrow::compute::Declaration source = arrow::compute::Declaration( {"source", diff --git a/cpp/src/arrow/compute/exec/benchmark_util.h b/cpp/src/arrow/compute/exec/benchmark_util.h index 7897288cb8f..c66c2e91dbf 100644 --- a/cpp/src/arrow/compute/exec/benchmark_util.h +++ b/cpp/src/arrow/compute/exec/benchmark_util.h @@ -29,13 +29,11 @@ namespace arrow { namespace compute { -Status BenchmarkNodeOverhead(benchmark::State& state, arrow::compute::ExecContext ctx, - int32_t num_batches, int32_t batch_size, - arrow::compute::BatchesWithSchema data, +Status BenchmarkNodeOverhead(benchmark::State& state, int32_t num_batches, + int32_t batch_size, arrow::compute::BatchesWithSchema data, std::vector& node_declarations); Status BenchmarkIsolatedNodeOverhead(benchmark::State& state, - arrow::compute::ExecContext ctx, arrow::compute::Expression expr, int32_t num_batches, int32_t batch_size, arrow::compute::BatchesWithSchema data, diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/compute/exec/exec_plan.cc index 666ab1d8c02..757c0db8822 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/compute/exec/exec_plan.cc @@ -17,6 +17,7 @@ #include "arrow/compute/exec/exec_plan.h" +#include #include #include #include @@ -43,6 +44,7 @@ namespace arrow { using internal::checked_cast; +using internal::ThreadPool; using internal::ToChars; namespace compute { @@ -50,9 +52,12 @@ namespace compute { namespace { struct ExecPlanImpl : public ExecPlan { - explicit ExecPlanImpl(QueryOptions options, ExecContext* exec_context, - std::shared_ptr metadata = NULLPTR) - : metadata_(std::move(metadata)), query_context_(options, *exec_context) {} + explicit ExecPlanImpl(QueryOptions options, ExecContext exec_context, + std::shared_ptr metadata = nullptr, + std::shared_ptr owned_thread_pool = nullptr) + : metadata_(std::move(metadata)), + query_context_(options, exec_context), + owned_thread_pool_(std::move(owned_thread_pool)) {} ~ExecPlanImpl() override { if (started_ && !finished_.is_finished()) { @@ -90,6 +95,15 @@ struct ExecPlanImpl : public ExecPlan { if (started_) { return Status::Invalid("restarted ExecPlan"); } + if (query_context_.exec_context()->executor() == nullptr) { + return Status::Invalid( + "An exec plan must have an executor for CPU tasks. To run without threads use " + "a SerialExeuctor (the arrow::compute::DeclarationTo... methods should take " + "care of this for you and are an easier way to execute an ExecPlan.)"); + } + if (query_context_.io_context()->executor() == nullptr) { + return Status::Invalid("An exec plan must have an I/O executor for I/O tasks."); + } started_ = true; @@ -99,8 +113,8 @@ struct ExecPlanImpl : public ExecPlan { // If no source node schedules any tasks (e.g. they do all their word synchronously as // part of StartProducing) then the plan may be finished before we return from this // call. - Future<> scheduler_finished = - util::AsyncTaskScheduler::Make([this](util::AsyncTaskScheduler* async_scheduler) { + Future<> scheduler_finished = util::AsyncTaskScheduler::Make( + [this](util::AsyncTaskScheduler* async_scheduler) { QueryContext* ctx = query_context(); RETURN_NOT_OK(ctx->Init(ctx->max_concurrency(), async_scheduler)); @@ -120,6 +134,8 @@ struct ExecPlanImpl : public ExecPlan { // away soon (or at least be replaced by a sub-scheduler to facilitate OT) for (auto& n : nodes_) { RETURN_NOT_OK(n->Init()); + } + for (auto& n : nodes_) { async_scheduler->AddSimpleTask([&] { return n->finished(); }); } @@ -153,18 +169,21 @@ struct ExecPlanImpl : public ExecPlan { EVENT(span_, "StartProducing:" + node->label(), {{"status", st.ToString()}}); if (!st.ok()) { // Stop nodes that successfully started, in reverse order - stopped_ = true; - StopProducingImpl(it.base(), sorted_nodes_.end()); - for (NodeVector::iterator fw_it = sorted_nodes_.begin(); fw_it != it.base(); - ++fw_it) { - Future<> fut = (*fw_it)->finished(); - if (!fut.is_finished()) fut.MarkFinished(); + bool expected = false; + if (stopped_.compare_exchange_strong(expected, true)) { + StopProducingImpl(it.base(), sorted_nodes_.end()); + for (NodeVector::iterator fw_it = sorted_nodes_.begin(); + fw_it != it.base(); ++fw_it) { + Future<> fut = (*fw_it)->finished(); + if (!fut.is_finished()) fut.MarkFinished(); + } } return st; } } return st; - }); + }, + [this](const Status& st) { StopProducing(); }); scheduler_finished.AddCallback( [this](const Status& st) { finished_.MarkFinished(st); }); // TODO(weston) Do we really need to return status here? Could we change this return @@ -179,9 +198,11 @@ struct ExecPlanImpl : public ExecPlan { void StopProducing() { DCHECK(started_) << "stopped an ExecPlan which never started"; EVENT(span_, "StopProducing"); - stopped_ = true; - query_context()->scheduler()->Abort( - [this]() { StopProducingImpl(sorted_nodes_.begin(), sorted_nodes_.end()); }); + bool expected = false; + if (stopped_.compare_exchange_strong(expected, true)) { + query_context()->scheduler()->Abort( + [this]() { StopProducingImpl(sorted_nodes_.begin(), sorted_nodes_.end()); }); + } } template @@ -289,7 +310,8 @@ struct ExecPlanImpl : public ExecPlan { Status error_st_; Future<> finished_ = Future<>::Make(); - bool started_ = false, stopped_ = false; + bool started_ = false; + std::atomic stopped_{false}; std::vector> nodes_; NodeVector sources_, sinks_; NodeVector sorted_nodes_; @@ -297,6 +319,9 @@ struct ExecPlanImpl : public ExecPlan { util::tracing::Span span_; std::shared_ptr metadata_; QueryContext query_context_; + // This field only exists for backwards compatibility. Remove once the deprecated + // ExecPlan::Make overloads have been removed. + std::shared_ptr owned_thread_pool_; }; ExecPlanImpl* ToDerived(ExecPlan* ptr) { return checked_cast(ptr); } @@ -318,14 +343,36 @@ std::optional GetNodeIndex(const std::vector& nodes, const uint32_t ExecPlan::kMaxBatchSize; Result> ExecPlan::Make( - QueryOptions opts, ExecContext* ctx, + QueryOptions opts, ExecContext ctx, std::shared_ptr metadata) { return std::shared_ptr(new ExecPlanImpl{opts, ctx, std::move(metadata)}); } +Result> ExecPlan::Make( + ExecContext ctx, std::shared_ptr metadata) { + return Make(/*opts=*/{}, ctx, std::move(metadata)); +} + +// Deprecated and left for backwards compatibility. If the user does not supply a CPU +// executor then we will create a 1 thread pool and tie its lifetime to the plan +Result> ExecPlan::Make( + QueryOptions opts, ExecContext* ctx, + std::shared_ptr metadata) { + if (ctx->executor() == nullptr) { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr tpool, ThreadPool::Make(1)); + ExecContext actual_ctx(ctx->memory_pool(), tpool.get(), ctx->func_registry()); + return std::shared_ptr( + new ExecPlanImpl{opts, actual_ctx, std::move(metadata), std::move(tpool)}); + } + return ExecPlan::Make(opts, *ctx, std::move(metadata)); +} + +// Deprecated Result> ExecPlan::Make( ExecContext* ctx, std::shared_ptr metadata) { + ARROW_SUPPRESS_DEPRECATION_WARNING return Make(/*opts=*/{}, ctx, std::move(metadata)); + ARROW_UNSUPPRESS_DEPRECATION_WARNING } ExecNode* ExecPlan::AddNode(std::unique_ptr node) { @@ -443,7 +490,7 @@ std::shared_ptr MakeGeneratorReader( // reading from generator until end is reached. std::shared_ptr batch; RETURN_NOT_OK(ReadNext(&batch)); - while (batch != NULLPTR) { + while (batch != nullptr) { RETURN_NOT_OK(ReadNext(&batch)); } return Status::OK(); @@ -503,7 +550,7 @@ bool Declaration::IsValid(ExecFactoryRegistry* registry) const { } Future> DeclarationToTableAsync(Declaration declaration, - ExecContext* exec_context) { + ExecContext exec_context) { std::shared_ptr> output_table = std::make_shared>(); ARROW_ASSIGN_OR_RAISE(std::shared_ptr exec_plan, @@ -511,57 +558,143 @@ Future> DeclarationToTableAsync(Declaration declaration, Declaration with_sink = Declaration::Sequence( {declaration, {"table_sink", TableSinkNodeOptions(output_table.get())}}); ARROW_RETURN_NOT_OK(with_sink.AddToPlan(exec_plan.get())); + ARROW_RETURN_NOT_OK(exec_plan->Validate()); ARROW_RETURN_NOT_OK(exec_plan->StartProducing()); return exec_plan->finished().Then([exec_plan, output_table] { return *output_table; }); } +Future> DeclarationToTableAsync(Declaration declaration, + bool use_threads) { + if (use_threads) { + return DeclarationToTableAsync(std::move(declaration), *threaded_exec_context()); + } else { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr tpool, ThreadPool::Make(1)); + ExecContext ctx(default_memory_pool(), tpool.get()); + return DeclarationToTableAsync(std::move(declaration), ctx) + .Then([tpool](const std::shared_ptr& table) { return table; }); + } +} + Result> DeclarationToTable(Declaration declaration, - ExecContext* exec_context) { - return DeclarationToTableAsync(std::move(declaration), exec_context).result(); + bool use_threads) { + return ::arrow::internal::RunSynchronously>>( + [declaration = std::move(declaration)](::arrow::internal::Executor* executor) { + ExecContext ctx(default_memory_pool(), executor); + return DeclarationToTableAsync(std::move(declaration), ctx); + }, + use_threads); } Future>> DeclarationToBatchesAsync( - Declaration declaration, ExecContext* exec_context) { + Declaration declaration, ExecContext exec_context) { return DeclarationToTableAsync(std::move(declaration), exec_context) .Then([](const std::shared_ptr
& table) { return TableBatchReader(table).ToRecordBatches(); }); } +Future>> DeclarationToBatchesAsync( + Declaration declaration, bool use_threads) { + if (use_threads) { + return DeclarationToBatchesAsync(std::move(declaration), *threaded_exec_context()); + } else { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr tpool, ThreadPool::Make(1)); + ExecContext ctx(default_memory_pool(), tpool.get()); + return DeclarationToBatchesAsync(std::move(declaration), ctx) + .Then([tpool](const std::vector>& batches) { + return batches; + }); + } +} + Result>> DeclarationToBatches( - Declaration declaration, ExecContext* exec_context) { - return DeclarationToBatchesAsync(std::move(declaration), exec_context).result(); + Declaration declaration, bool use_threads) { + return ::arrow::internal::RunSynchronously< + Future>>>( + [declaration = std::move(declaration)](::arrow::internal::Executor* executor) { + ExecContext ctx(default_memory_pool(), executor); + return DeclarationToBatchesAsync(std::move(declaration), ctx); + }, + use_threads); } -Future> DeclarationToExecBatchesAsync(Declaration declaration, - ExecContext* exec_context) { +Future DeclarationToExecBatchesAsync(Declaration declaration, + ExecContext exec_context) { + std::shared_ptr out_schema; AsyncGenerator> sink_gen; ARROW_ASSIGN_OR_RAISE(std::shared_ptr exec_plan, ExecPlan::Make(exec_context)); - Declaration with_sink = - Declaration::Sequence({declaration, {"sink", SinkNodeOptions(&sink_gen)}}); + Declaration with_sink = Declaration::Sequence( + {declaration, {"sink", SinkNodeOptions(&sink_gen, &out_schema)}}); ARROW_RETURN_NOT_OK(with_sink.AddToPlan(exec_plan.get())); + ARROW_RETURN_NOT_OK(exec_plan->Validate()); ARROW_RETURN_NOT_OK(exec_plan->StartProducing()); auto collected_fut = CollectAsyncGenerator(sink_gen); return AllFinished({exec_plan->finished(), Future<>(collected_fut)}) - .Then([collected_fut, exec_plan]() -> Result> { + .Then([collected_fut, exec_plan, + schema = std::move(out_schema)]() -> Result { ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); - return ::arrow::internal::MapVector( + std::vector exec_batches = ::arrow::internal::MapVector( [](std::optional batch) { return batch.value_or(ExecBatch()); }, std::move(collected)); + return BatchesWithCommonSchema{std::move(exec_batches), schema}; }); } -Result> DeclarationToExecBatches(Declaration declaration, - ExecContext* exec_context) { - return DeclarationToExecBatchesAsync(std::move(declaration), exec_context).result(); +Future DeclarationToExecBatchesAsync(Declaration declaration, + bool use_threads) { + if (use_threads) { + return DeclarationToExecBatchesAsync(std::move(declaration), + *threaded_exec_context()); + } else { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr tpool, ThreadPool::Make(1)); + ExecContext ctx(default_memory_pool(), tpool.get()); + return DeclarationToExecBatchesAsync(std::move(declaration), ctx) + .Then([tpool](const BatchesWithCommonSchema& batches) { return batches; }); + } +} + +Result DeclarationToExecBatches(Declaration declaration, + bool use_threads) { + return ::arrow::internal::RunSynchronously>( + [declaration = std::move(declaration)](::arrow::internal::Executor* executor) { + ExecContext ctx(default_memory_pool(), executor); + return DeclarationToExecBatchesAsync(std::move(declaration), ctx); + }, + use_threads); +} + +Future<> DeclarationToStatusAsync(Declaration declaration, ExecContext exec_context) { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr exec_plan, + ExecPlan::Make(exec_context)); + ARROW_RETURN_NOT_OK(declaration.AddToPlan(exec_plan.get())); + ARROW_RETURN_NOT_OK(exec_plan->Validate()); + ARROW_RETURN_NOT_OK(exec_plan->StartProducing()); + // Keep the exec_plan alive until it finishes + return exec_plan->finished().Then([exec_plan]() {}); +} + +Future<> DeclarationToStatusAsync(Declaration declaration, bool use_threads) { + if (use_threads) { + return DeclarationToStatusAsync(std::move(declaration), *threaded_exec_context()); + } else { + ARROW_ASSIGN_OR_RAISE(std::shared_ptr tpool, ThreadPool::Make(1)); + ExecContext ctx(default_memory_pool(), tpool.get()); + return DeclarationToStatusAsync(std::move(declaration), ctx).Then([tpool]() {}); + } +} + +Status DeclarationToStatus(Declaration declaration, bool use_threads) { + return ::arrow::internal::RunSynchronously>( + [declaration = std::move(declaration)](::arrow::internal::Executor* executor) { + ExecContext ctx(default_memory_pool(), executor); + return DeclarationToStatusAsync(std::move(declaration), ctx); + }, + use_threads); } namespace { struct BatchConverter { - explicit BatchConverter(::arrow::internal::Executor* executor) - : exec_context(std::make_shared(default_memory_pool(), executor)) {} - ~BatchConverter() { if (!exec_plan) { return; @@ -593,7 +726,6 @@ struct BatchConverter { }); } - std::shared_ptr exec_context; AsyncGenerator> exec_batch_gen; std::shared_ptr schema; std::shared_ptr exec_plan; @@ -602,9 +734,9 @@ struct BatchConverter { Result>> DeclarationToRecordBatchGenerator( Declaration declaration, ::arrow::internal::Executor* executor, std::shared_ptr* out_schema) { - auto converter = std::make_shared(executor); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - ExecPlan::Make(converter->exec_context.get())); + auto converter = std::make_shared(); + ExecContext exec_context(default_memory_pool(), executor); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, ExecPlan::Make(exec_context)); Declaration with_sink = Declaration::Sequence( {declaration, {"sink", SinkNodeOptions(&converter->exec_batch_gen, &converter->schema)}}); diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index 52265b4f281..09fab007278 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -54,11 +54,21 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { /// Make an empty exec plan static Result> Make( - QueryOptions options, ExecContext* exec_context = default_exec_context(), + QueryOptions options, ExecContext exec_context = *threaded_exec_context(), std::shared_ptr metadata = NULLPTR); static Result> Make( - ExecContext* exec_context = default_exec_context(), + ExecContext exec_context = *threaded_exec_context(), + std::shared_ptr metadata = NULLPTR); + + ARROW_DEPRECATED("Deprecated in 11.0.0. Use version that takes ExecContext by value.") + static Result> Make( + QueryOptions options, ExecContext* exec_context, + std::shared_ptr metadata = NULLPTR); + + ARROW_DEPRECATED("Deprecated in 11.0.0. Use version that takes ExecContext by value.") + static Result> Make( + ExecContext* exec_context, std::shared_ptr metadata = NULLPTR); ExecNode* AddNode(std::unique_ptr node); @@ -419,37 +429,112 @@ struct ARROW_EXPORT Declaration { /// This method will add a sink node to the declaration to collect results into a /// table. It will then create an ExecPlan from the declaration, start the exec plan, /// block until the plan has finished, and return the created table. -ARROW_EXPORT Result> DeclarationToTable( - Declaration declaration, ExecContext* exec_context = default_exec_context()); +/// +/// If `use_threads` is false then all CPU work will be done on the calling thread. I/O +/// tasks will still happen on the I/O executor and may be multi-threaded (but should +/// not use significant CPU resources) +ARROW_EXPORT Result> DeclarationToTable(Declaration declaration, + bool use_threads = true); /// \brief Asynchronous version of \see DeclarationToTable +/// +/// The behavior of use_threads is slightly different than the synchronous version since +/// we cannot run synchronously on the calling thread. Instead, if use_threads=false then +/// a new thread pool will be created with a single thread and this will be used for all +/// compute work. ARROW_EXPORT Future> DeclarationToTableAsync( - Declaration declaration, ExecContext* exec_context = default_exec_context()); + Declaration declaration, bool use_threads = true); + +/// \brief Overload of \see DeclarationToTableAsync accepting a custom exec context +/// +/// The executor must be specified (cannot be null) and must be kept alive until the +/// returned future finishes. +ARROW_EXPORT Future> DeclarationToTableAsync( + Declaration declaration, ExecContext custom_exec_context); + +/// \brief a collection of exec batches with a common schema +struct BatchesWithCommonSchema { + std::vector batches; + std::shared_ptr schema; +}; /// \brief Utility method to run a declaration and collect the results into ExecBatch /// vector /// -/// \see DeclarationToTable for details -ARROW_EXPORT Result> DeclarationToExecBatches( - Declaration declaration, ExecContext* exec_context = default_exec_context()); +/// \see DeclarationToTable for details on threading & execution +ARROW_EXPORT Result DeclarationToExecBatches( + Declaration declaration, bool use_threads = true); /// \brief Asynchronous version of \see DeclarationToExecBatches -ARROW_EXPORT Future> DeclarationToExecBatchesAsync( - Declaration declaration, ExecContext* exec_context = default_exec_context()); +/// +/// \see DeclarationToTableAsync for details on threading & execution +ARROW_EXPORT Future DeclarationToExecBatchesAsync( + Declaration declaration, bool use_threads = true); + +/// \brief Overload of \see DeclarationToExecBatchesAsync accepting a custom exec context +/// +/// \see DeclarationToTableAsync for details on threading & execution +ARROW_EXPORT Future DeclarationToExecBatchesAsync( + Declaration declaration, ExecContext custom_exec_context); /// \brief Utility method to run a declaration and collect the results into a vector /// -/// \see DeclarationToTable for details +/// \see DeclarationToTable for details on threading & execution ARROW_EXPORT Result>> DeclarationToBatches( - Declaration declaration, ExecContext* exec_context = default_exec_context()); + Declaration declaration, bool use_threads = true); /// \brief Asynchronous version of \see DeclarationToBatches +/// +/// \see DeclarationToTableAsync for details on threading & execution +ARROW_EXPORT Future>> DeclarationToBatchesAsync( + Declaration declaration, bool use_threads = true); + +/// \brief Overload of \see DeclarationToBatchesAsync accepting a custom exec context +/// +/// \see DeclarationToTableAsync for details on threading & execution ARROW_EXPORT Future>> DeclarationToBatchesAsync( - Declaration declaration, ExecContext* exec_context = default_exec_context()); + Declaration declaration, ExecContext exec_context); /// \brief Utility method to run a declaration and return results as a RecordBatchReader +/// +/// If an exec context is not provided then a default exec context will be used based +/// on the value of `use_threads`. If `use_threads` is false then the CPU exeuctor will +/// be a serial executor and all CPU work will be done on the calling thread. I/O tasks +/// will still happen on the I/O executor and may be multi-threaded. +/// +/// If `use_threads` is false then all CPU work will happen during the calls to +/// RecordBatchReader::Next and no CPU work will happen in the background. If +/// `use_threads` is true then CPU work will happen on the CPU thread pool and tasks may +/// run in between calls to RecordBatchReader::Next. If the returned reader is not +/// consumed quickly enough then the plan will eventually pause as the backpressure queue +/// fills up. +/// +/// If a custom exec context is provided then the value of `use_threads` will be ignored. ARROW_EXPORT Result> DeclarationToReader( - Declaration declaration, bool use_threads); + Declaration declaration, bool use_threads = true); + +/// \brief Utility method to run a declaration and ignore results +/// +/// This can be useful when the data are consumed as part of the plan itself, for +/// example, when the plan ends with a write node. +/// +/// \see DeclarationToTable for details on threading & execution +ARROW_EXPORT Status DeclarationToStatus(Declaration declaration, bool use_threads = true); + +/// \brief Asynchronous version of \see DeclarationToStatus +/// +/// This can be useful when the data are consumed as part of the plan itself, for +/// example, when the plan ends with a write node. +/// +/// \see DeclarationToTableAsync for details on threading & execution +ARROW_EXPORT Future<> DeclarationToStatusAsync(Declaration declaration, + bool use_threads = true); + +/// \brief Overload of \see DeclarationToStatusAsync accepting a custom exec context +/// +/// \see DeclarationToTableAsync for details on threading & execution +ARROW_EXPORT Future<> DeclarationToStatusAsync(Declaration declaration, + ExecContext exec_context); /// \brief Wrap an ExecBatch generator in a RecordBatchReader. /// diff --git a/cpp/src/arrow/compute/exec/filter_benchmark.cc b/cpp/src/arrow/compute/exec/filter_benchmark.cc index 64cf307580b..aa8e3e8b77d 100644 --- a/cpp/src/arrow/compute/exec/filter_benchmark.cc +++ b/cpp/src/arrow/compute/exec/filter_benchmark.cc @@ -76,23 +76,20 @@ static void FilterOverhead(benchmark::State& state, std::vector expr arrow::compute::BatchesWithSchema data = MakeRandomBatchesWithNullProbability( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size, null_prob, bool_true_probability); - ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); std::vector filter_node_dec; for (Expression expr : expr_vector) { filter_node_dec.push_back({"filter", FilterNodeOptions(expr)}); } - ASSERT_OK( - BenchmarkNodeOverhead(state, ctx, num_batches, batch_size, data, filter_node_dec)); + ASSERT_OK(BenchmarkNodeOverhead(state, num_batches, batch_size, data, filter_node_dec)); } static void FilterOverheadIsolated(benchmark::State& state, Expression expr) { - ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); const int32_t batch_size = static_cast(state.range(0)); const int32_t num_batches = kTotalBatchSize / batch_size; arrow::compute::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); FilterNodeOptions options = FilterNodeOptions{expr}; - ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, ctx, expr, num_batches, batch_size, data, + ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, expr, num_batches, batch_size, data, "filter", options)); } diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/compute/exec/hash_join_node.cc index be3b01eb7f6..37bdb82517a 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node.cc @@ -941,7 +941,7 @@ class HashJoinNode : public ExecNode { "which is incompatible with legacy batching"); } - bool use_sync_execution = !(ctx->executor()); + bool use_sync_execution = ctx->executor()->GetCapacity() == 1; // TODO(ARROW-15732) // Each side of join might have an IO thread being called from. Once this is fixed // we will change it back to just the CPU's thread pool capacity. @@ -1007,16 +1007,15 @@ class HashJoinNode : public ExecNode { void StopProducing(ExecNode* output) override { DCHECK_EQ(output, outputs_[0]); - StopProducing(); + for (auto&& input : inputs_) { + input->StopProducing(this); + } } void StopProducing() override { EVENT(span_, "StopProducing"); bool expected = false; if (complete_.compare_exchange_strong(expected, true)) { - for (auto&& input : inputs_) { - input->StopProducing(this); - } impl_->Abort([this]() { finished_.MarkFinished(); }); } } diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/compute/exec/hash_join_node_test.cc index adc5ec70ebd..cd8f392ad70 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/compute/exec/hash_join_node_test.cc @@ -69,34 +69,20 @@ void CheckRunOutput(JoinType type, const BatchesWithSchema& l_batches, const std::vector& left_keys, const std::vector& right_keys, const BatchesWithSchema& exp_batches, bool parallel = false) { - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - + Declaration left{"source", + SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, + /*slow=*/false)}}; + Declaration right{"source", + SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, + /*slow=*/false)}}; HashJoinNodeOptions join_options{type, left_keys, right_keys}; - Declaration join{"hashjoin", join_options}; - - // add left source - join.inputs.emplace_back(Declaration{ - "source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, - /*slow=*/false)}}); - // add right source - join.inputs.emplace_back(Declaration{ - "source", SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, - /*slow=*/false)}}); - AsyncGenerator> sink_gen; - - ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}}) - .AddToPlan(plan.get())); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_options}; - ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto out_table, DeclarationToTable(std::move(join), parallel)); ASSERT_OK_AND_ASSIGN(auto exp_table, TableFromExecBatches(exp_batches.schema, exp_batches.batches)); - ASSERT_OK_AND_ASSIGN(auto out_table, TableFromExecBatches(exp_batches.schema, res)); - if (exp_table->num_rows() == 0) { ASSERT_EQ(exp_table->num_rows(), out_table->num_rows()); } else { @@ -890,44 +876,21 @@ Result> HashJoinWithExecPlan( const std::shared_ptr& output_schema, const std::vector>& l, const std::vector>& r, int num_batches_l, int num_batches_r) { - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - - ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(exec_ctx.get())); - // add left source BatchesWithSchema l_batches = TableToBatches(rng, num_batches_l, l, "l_"); - ARROW_ASSIGN_OR_RAISE( - ExecNode * l_source, - MakeExecNode("source", plan.get(), {}, + Declaration left{"source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, - /*slow=*/false)})); - + /*slow=*/false)}}; // add right source BatchesWithSchema r_batches = TableToBatches(rng, num_batches_r, r, "r_"); - ARROW_ASSIGN_OR_RAISE( - ExecNode * r_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, - /*slow=*/false)})); - - ARROW_ASSIGN_OR_RAISE( - ExecNode * join, - MakeExecNode("hashjoin", plan.get(), {l_source, r_source}, join_options)); - - AsyncGenerator> sink_gen; - ARROW_ASSIGN_OR_RAISE( - std::ignore, MakeExecNode("sink", plan.get(), {join}, SinkNodeOptions{&sink_gen})); - - auto batches_fut = StartAndCollect(plan.get(), sink_gen); - if (!batches_fut.Wait(::arrow::kDefaultAssertFinishesWaitSeconds)) { - plan->StopProducing(); - // If this second wait fails then there isn't much we can do. We will abort - // and probably get a segmentation fault. - plan->finished().Wait(::arrow::kDefaultAssertFinishesWaitSeconds); - return Status::Invalid("Plan did not finish in a reasonable amount of time"); - } - return batches_fut.result(); + Declaration right{"source", + SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, + /*slow=*/false)}}; + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_options}; + + ARROW_ASSIGN_OR_RAISE(BatchesWithCommonSchema batches_and_schema, + DeclarationToExecBatches(std::move(join), parallel)); + return batches_and_schema.batches; } TEST(HashJoin, Suffix) { @@ -961,40 +924,24 @@ TEST(HashJoin, Suffix) { field("ldistinct", int32()), field("rkey", int32()), field("shared_r", int32()), field("rdistinct", int32())}); - ExecContext exec_ctx; - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_ctx)); AsyncGenerator> sink_gen; - ExecNode* left_source; - ExecNode* right_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, + Declaration left{"source", SourceNodeOptions{input_left.schema, input_left.gen(/*parallel=*/false, - /*slow=*/false)})); - - ASSERT_OK_AND_ASSIGN(right_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right.schema, - input_right.gen(/*parallel=*/false, - /*slow=*/false)})); - + /*slow=*/false)}}; + Declaration right{ + "source", SourceNodeOptions{input_right.schema, input_right.gen(/*parallel=*/false, + /*slow=*/false)}}; HashJoinNodeOptions join_opts{JoinType::INNER, /*left_keys=*/{"lkey"}, /*right_keys=*/{"rkey"}, literal(true), "_l", "_r"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_opts}; - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto actual, DeclarationToExecBatches(std::move(join))); - AssertExecBatchesEqual(expected.schema, expected.batches, result); - AssertSchemaEqual(expected.schema, hashjoin->output_schema()); + AssertExecBatchesEqualIgnoringOrder(expected.schema, expected.batches, actual.batches); + AssertSchemaEqual(expected.schema, actual.schema); } TEST(HashJoin, Random) { @@ -1186,7 +1133,7 @@ TEST(HashJoin, Random) { TableFromExecBatches(output_schema, batches)); // Compare results - AssertTablesEqual(output_rows_ref, output_rows_test); + AssertTablesEqualIgnoringOrder(output_rows_ref, output_rows_test); } } @@ -1310,19 +1257,13 @@ void TestHashJoinDictionaryHelper( r_batches.batches.resize(0); } - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - ASSERT_OK_AND_ASSIGN( - ExecNode * l_source, - MakeExecNode("source", plan.get(), {}, + Declaration left{"source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, - /*slow=*/false)})); - ASSERT_OK_AND_ASSIGN( - ExecNode * r_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, - /*slow=*/false)})); + /*slow=*/false)}}; + + Declaration right{"source", + SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, + /*slow=*/false)}}; HashJoinNodeOptions join_options{join_type, {FieldRef(swap_sides ? "r_key" : "l_key")}, {FieldRef(swap_sides ? "l_key" : "r_key")}, @@ -1331,23 +1272,18 @@ void TestHashJoinDictionaryHelper( {FieldRef(swap_sides ? "l_key" : "r_key"), FieldRef(swap_sides ? "l_payload" : "r_payload")}, {cmp}}; - ASSERT_OK_AND_ASSIGN(ExecNode * join, MakeExecNode("hashjoin", plan.get(), - {(swap_sides ? r_source : l_source), - (swap_sides ? l_source : r_source)}, - join_options)); - AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN( - std::ignore, MakeExecNode("sink", plan.get(), {join}, SinkNodeOptions{&sink_gen})); - ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); + Declaration join{ + "hashjoin", {swap_sides ? right : left, swap_sides ? left : right}, join_options}; + ASSERT_OK_AND_ASSIGN(auto res, DeclarationToExecBatches(std::move(join), parallel)); - for (auto& batch : res) { - DecodeScalarsAndDictionariesInBatch(&batch, exec_ctx->memory_pool()); + for (auto& batch : res.batches) { + DecodeScalarsAndDictionariesInBatch(&batch, default_memory_pool()); } std::shared_ptr output_schema = - UpdateSchemaAfterDecodingDictionaries(join->output_schema()); + UpdateSchemaAfterDecodingDictionaries(res.schema); ASSERT_OK_AND_ASSIGN(std::shared_ptr
output, - TableFromExecBatches(output_schema, res)); + TableFromExecBatches(output_schema, res.batches)); ExecBatch expected_batch; if (swap_sides) { @@ -1358,7 +1294,7 @@ void TestHashJoinDictionaryHelper( r_out_key, r_out_payload})); } - DecodeScalarsAndDictionariesInBatch(&expected_batch, exec_ctx->memory_pool()); + DecodeScalarsAndDictionariesInBatch(&expected_batch, default_memory_pool()); // Slice expected batch into two to separate rows on right side with no matches from // everything else. @@ -1399,7 +1335,7 @@ void TestHashJoinDictionaryHelper( TableFromExecBatches(output_schema, expected_batches)); // Compare results - AssertTablesEqual(expected, output); + AssertTablesEqualIgnoringOrder(expected, output); } TEST(HashJoin, Dictionary) { @@ -1734,37 +1670,21 @@ TEST(HashJoin, DictNegative) { ExecBatch::Make({i == 2 ? datumSecondB : datumSecondA, i == 3 ? datumSecondB : datumSecondA})); - auto exec_ctx = std::make_unique(default_memory_pool(), nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - ASSERT_OK_AND_ASSIGN( - ExecNode * l_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{l.schema, l.gen(/*parallel=*/false, - /*slow=*/false)})); - ASSERT_OK_AND_ASSIGN( - ExecNode * r_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{r.schema, r.gen(/*parallel=*/false, - /*slow=*/false)})); + Declaration left{"source", SourceNodeOptions{l.schema, l.gen(/*parallel=*/false, + /*slow=*/false)}}; + Declaration right{"source", SourceNodeOptions{r.schema, r.gen(/*parallel=*/false, + /*slow=*/false)}}; HashJoinNodeOptions join_options{JoinType::INNER, {FieldRef("l_key")}, {FieldRef("r_key")}, {FieldRef("l_key"), FieldRef("l_payload")}, {FieldRef("r_key"), FieldRef("r_payload")}, {JoinKeyCmp::EQ}}; - ASSERT_OK_AND_ASSIGN( - ExecNode * join, - MakeExecNode("hashjoin", plan.get(), {l_source, r_source}, join_options)); - AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {join}, - SinkNodeOptions{&sink_gen})); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_options}; - EXPECT_FINISHES_AND_RAISES_WITH_MESSAGE_THAT( + EXPECT_RAISES_WITH_MESSAGE_THAT( NotImplemented, ::testing::HasSubstr("Unifying differing dictionaries"), - StartAndCollect(plan.get(), sink_gen)); - // Since we returned an error, the StartAndCollect future may return before - // the plan is done finishing. - plan->finished().Wait(); + DeclarationToTable(std::move(join), /*use_threads=*/false)); } } @@ -1787,63 +1707,44 @@ TEST(HashJoin, UnsupportedTypes) { BatchesWithSchema l_batches = GenerateBatchesFromString(schemas.first, {R"([])"}); BatchesWithSchema r_batches = GenerateBatchesFromString(schemas.second, {R"([])"}); - ExecContext exec_ctx; - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_ctx)); - HashJoinNodeOptions join_options{JoinType::LEFT_SEMI, l_keys, r_keys}; - Declaration join{"hashjoin", join_options}; - join.inputs.emplace_back(Declaration{ - "source", SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, slow)}}); - join.inputs.emplace_back(Declaration{ - "source", SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, slow)}}); + Declaration left{"source", + SourceNodeOptions{l_batches.schema, l_batches.gen(parallel, slow)}}; + Declaration right{"source", + SourceNodeOptions{r_batches.schema, r_batches.gen(parallel, slow)}}; + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_options}; - ASSERT_RAISES(Invalid, join.AddToPlan(plan.get())); + ASSERT_RAISES(Invalid, DeclarationToStatus(std::move(join))); } } void TestSimpleJoinHelper(BatchesWithSchema input_left, BatchesWithSchema input_right, BatchesWithSchema expected) { - ExecContext exec_ctx; - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_ctx)); AsyncGenerator> sink_gen; - ExecNode* left_source; - ExecNode* right_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, + Declaration left{"source", SourceNodeOptions{input_left.schema, input_left.gen(/*parallel=*/false, - /*slow=*/false)})); - - ASSERT_OK_AND_ASSIGN(right_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right.schema, - input_right.gen(/*parallel=*/false, - /*slow=*/false)})); + /*slow=*/false)}}; + Declaration right{ + "source", SourceNodeOptions{input_right.schema, input_right.gen(/*parallel=*/false, + /*slow=*/false)}}; HashJoinNodeOptions join_opts{JoinType::INNER, /*left_keys=*/{"lkey"}, /*right_keys=*/{"rkey"}, literal(true), "_l", "_r"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_opts}; - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto result, DeclarationToExecBatches(std::move(join))); - ASSERT_OK_AND_ASSIGN( - auto output_rows_test, - TableFromExecBatches(std::move(hashjoin->output_schema()), result)); - ASSERT_OK_AND_ASSIGN( - auto expected_rows_test, - TableFromExecBatches(std::move(expected.schema), expected.batches)); + ASSERT_OK_AND_ASSIGN(auto output_rows_test, + TableFromExecBatches(result.schema, result.batches)); + ASSERT_OK_AND_ASSIGN(auto expected_rows_test, + TableFromExecBatches(expected.schema, expected.batches)); AssertTablesEqual(*output_rows_test, *expected_rows_test, /*same_chunk_layout=*/false, /*flatten=*/true); - AssertSchemaEqual(expected.schema, hashjoin->output_schema()); + AssertSchemaEqual(expected.schema, result.schema); } TEST(HashJoin, ExtensionTypesSwissJoin) { @@ -1910,9 +1811,6 @@ TEST(HashJoin, ExtensionTypesHashJoin) { } TEST(HashJoin, CheckHashJoinNodeOptionsValidation) { - auto exec_ctx = std::make_unique(default_memory_pool(), nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - BatchesWithSchema input_left; input_left.batches = {ExecBatchFromJSON({int32(), int32(), int32()}, R"([ [1, 4, 7], @@ -1931,19 +1829,12 @@ TEST(HashJoin, CheckHashJoinNodeOptionsValidation) { input_right.schema = schema( {field("rkey", int32()), field("shared", int32()), field("rdistinct", int32())}); - ExecNode* l_source; - ExecNode* r_source; - ASSERT_OK_AND_ASSIGN( - l_source, - MakeExecNode("source", plan.get(), {}, + Declaration left{"source", SourceNodeOptions{input_left.schema, input_left.gen(/*parallel=*/false, - /*slow=*/false)})); - - ASSERT_OK_AND_ASSIGN(r_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right.schema, - input_right.gen(/*parallel=*/false, - /*slow=*/false)})) + /*slow=*/false)}}; + Declaration right{ + "source", SourceNodeOptions{input_right.schema, input_right.gen(/*parallel=*/false, + /*slow=*/false)}}; std::vector> l_keys = { {}, @@ -1965,9 +1856,9 @@ TEST(HashJoin, CheckHashJoinNodeOptionsValidation) { HashJoinNodeOptions options{JoinType::INNER, l_keys[j], r_keys[k], {}, {}, key_cmps[i]}; - EXPECT_RAISES_WITH_MESSAGE_THAT( - Invalid, ::testing::HasSubstr("key_cmp and keys"), - MakeExecNode("hashjoin", plan.get(), {l_source, r_source}, options)); + Declaration join{"hashjoin", {left, right}, options}; + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, ::testing::HasSubstr("key_cmp and keys"), + DeclarationToStatus(std::move(join))); } } } @@ -1995,25 +1886,12 @@ TEST(HashJoin, ResidualFilter) { input_right.schema = schema({field("r1", int32()), field("r2", int32()), field("r_str", utf8())}); - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - AsyncGenerator> sink_gen; - - ExecNode* left_source; - ExecNode* right_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_left.schema, - input_left.gen(parallel, /*slow=*/false)})); - - ASSERT_OK_AND_ASSIGN( - right_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right.schema, - input_right.gen(parallel, /*slow=*/false)})) + Declaration left{ + "source", + SourceNodeOptions{input_left.schema, input_left.gen(parallel, /*slow=*/false)}}; + Declaration right{ + "source", + SourceNodeOptions{input_right.schema, input_right.gen(parallel, /*slow=*/false)}}; Expression mul = call("multiply", {field_ref("l1"), field_ref("l2")}); Expression combination = call("add", {mul, field_ref("r1")}); @@ -2024,14 +1902,10 @@ TEST(HashJoin, ResidualFilter) { /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, std::move(residual_filter), "l_", "r_"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_opts}; - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto result, + DeclarationToExecBatches(std::move(join), parallel)); std::vector expected = { ExecBatchFromJSON({int32(), int32(), utf8(), int32(), int32(), utf8()}, R"([ @@ -2040,7 +1914,7 @@ TEST(HashJoin, ResidualFilter) { [2, 5, "beta", 2, 12, "beta"], [3, 4, "alpha", 4, 16, "alpha"]])")}; - AssertExecBatchesEqual(hashjoin->output_schema(), result, expected); + AssertExecBatchesEqualIgnoringOrder(result.schema, result.batches, expected); } } @@ -2076,41 +1950,27 @@ TEST(HashJoin, TrivialResidualFilter) { default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - AsyncGenerator> sink_gen; - - ExecNode* left_source; - ExecNode* right_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_left.schema, - input_left.gen(parallel, /*slow=*/false)})); - - ASSERT_OK_AND_ASSIGN( - right_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right.schema, - input_right.gen(parallel, /*slow=*/false)})) + Declaration left{ + "source", + SourceNodeOptions{input_left.schema, input_left.gen(parallel, /*slow=*/false)}}; + Declaration right{"source", + SourceNodeOptions{input_right.schema, + input_right.gen(parallel, /*slow=*/false)}}; HashJoinNodeOptions join_opts{ JoinType::INNER, /*left_keys=*/{"l_str"}, /*right_keys=*/{"r_str"}, filters[test_id], "l_", "r_"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_source, right_source}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + Declaration join{"hashjoin", {std::move(left), std::move(right)}, join_opts}; - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto result, + DeclarationToExecBatches(std::move(join), parallel)); std::vector expected = {ExecBatchFromJSON( {int32(), utf8(), int32(), utf8()}, expected_strings[test_id])}; - AssertExecBatchesEqual(hashjoin->output_schema(), result, expected); + AssertExecBatchesEqualIgnoringOrder(result.schema, result.batches, expected); } } } @@ -2212,42 +2072,32 @@ void TestSingleChainOfHashJoins(Random64Bit& rng) { for (bool bloom_filters : {false, true}) { bool kParallel = true; ARROW_SCOPED_TRACE(bloom_filters ? "bloom filtered" : "unfiltered"); - auto exec_ctx = std::make_unique( - default_memory_pool(), kParallel ? arrow::internal::GetCpuThreadPool() : nullptr); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - ExecNode* left_source; - ASSERT_OK_AND_ASSIGN( - left_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_left.schema, - input_left.gen(kParallel, /*slow=*/false)})); - std::vector joins(num_joins); + Declaration left{ + "source", + SourceNodeOptions{input_left.schema, input_left.gen(kParallel, /*slow=*/false)}}; + + Declaration last_join; for (int i = 0; i < num_joins; i++) { opts[i].disable_bloom_filter = !bloom_filters; - ExecNode* right_source; - ASSERT_OK_AND_ASSIGN( - right_source, - MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input_right[i].schema, - input_right[i].gen(kParallel, /*slow=*/false)})); + Declaration right{"source", + SourceNodeOptions{input_right[i].schema, + input_right[i].gen(kParallel, /*slow=*/false)}}; - std::vector inputs; + std::vector inputs; if (i == 0) - inputs = {left_source, right_source}; + inputs = {std::move(left), std::move(right)}; else - inputs = {joins[i - 1], right_source}; - ASSERT_OK_AND_ASSIGN(joins[i], - MakeExecNode("hashjoin", plan.get(), inputs, opts[i])); + inputs = {std::move(last_join), std::move(right)}; + last_join = Declaration{"hashjoin", std::move(inputs), opts[i]}; } - AsyncGenerator> sink_gen; - ASSERT_OK( - MakeExecNode("sink", plan.get(), {joins.back()}, SinkNodeOptions{&sink_gen})); - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + + ASSERT_OK_AND_ASSIGN(auto result, + DeclarationToExecBatches(std::move(last_join), kParallel)); if (!bloom_filters) - reference = std::move(result); + reference = std::move(result.batches); else - AssertExecBatchesEqual(joins.back()->output_schema(), reference, result); + AssertExecBatchesEqualIgnoringOrder(result.schema, reference, result.batches); } } diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index 7894e120617..5daaf0584ae 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -199,8 +199,8 @@ constexpr int32_t kDefaultBackpressureLowBytes = 1 << 28; // 256MiB class ARROW_EXPORT BackpressureMonitor { public: virtual ~BackpressureMonitor() = default; - virtual uint64_t bytes_in_use() const = 0; - virtual bool is_paused() const = 0; + virtual uint64_t bytes_in_use() = 0; + virtual bool is_paused() = 0; }; /// \brief Options to control backpressure behavior diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 25ef3d73a8c..6db069cd9ff 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -274,7 +274,7 @@ TEST(ExecPlanExecution, TableSourceSink) { ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen)); ASSERT_OK_AND_ASSIGN(auto out_table, TableFromExecBatches(exp_batches.schema, res)); - AssertTablesEqual(table, out_table); + AssertTablesEqualIgnoringOrder(table, out_table); } } @@ -322,9 +322,9 @@ void TestSourceSink( std::string source_factory_name, std::function>(const BatchesWithSchema&)> to_elements) { - ASSERT_OK_AND_ASSIGN(auto io_executor, arrow::internal::ThreadPool::Make(1)); - ExecContext exec_context(default_memory_pool(), io_executor.get()); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_context)); + ASSERT_OK_AND_ASSIGN(auto executor, arrow::internal::ThreadPool::Make(1)); + ExecContext exec_context(default_memory_pool(), executor.get()); + ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_context)); AsyncGenerator> sink_gen; auto exp_batches = MakeBasicBatches(); @@ -627,12 +627,12 @@ TEST(ExecPlanExecution, SourceConsumingSink) { // Source should finish fairly quickly ASSERT_FINISHES_OK(source->finished()); SleepABit(); - ASSERT_EQ(2, batches_seen); // Consumer isn't finished and so plan shouldn't have finished AssertNotFinished(plan->finished()); // Mark consumption complete, plan should finish finish.MarkFinished(); ASSERT_FINISHES_OK(plan->finished()); + ASSERT_EQ(2, batches_seen); } } } @@ -645,7 +645,7 @@ TEST(ExecPlanExecution, SourceTableConsumingSink) { SCOPED_TRACE(parallel ? "parallel" : "single threaded"); ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - std::shared_ptr
out; + std::shared_ptr
out = nullptr; auto basic_data = MakeBasicBatches(); @@ -660,11 +660,11 @@ TEST(ExecPlanExecution, SourceTableConsumingSink) { // Source should finish fairly quickly ASSERT_FINISHES_OK(source->finished()); SleepABit(); - ASSERT_OK_AND_ASSIGN(auto actual, + ASSERT_OK_AND_ASSIGN(auto expected, TableFromExecBatches(basic_data.schema, basic_data.batches)); - ASSERT_EQ(5, out->num_rows()); - AssertTablesEqual(*actual, *out); ASSERT_FINISHES_OK(plan->finished()); + ASSERT_EQ(5, out->num_rows()); + AssertTablesEqualIgnoringOrder(expected, out); } } } @@ -738,16 +738,12 @@ TEST(ExecPlanExecution, ConsumingSinkError) { std::make_shared()}; for (auto& consumer : consumers) { - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); auto basic_data = MakeBasicBatches(); - ASSERT_OK(Declaration::Sequence( - {{"source", - SourceNodeOptions(basic_data.schema, basic_data.gen(false, false))}, - {"consuming_sink", ConsumingSinkNodeOptions(consumer)}}) - .AddToPlan(plan.get())); + Declaration plan = Declaration::Sequence( + {{"source", SourceNodeOptions(basic_data.schema, basic_data.gen(false, false))}, + {"consuming_sink", ConsumingSinkNodeOptions(consumer)}}); // Since the source node is not parallel the entire plan is run during StartProducing - ASSERT_RAISES(Invalid, plan->StartProducing()); - ASSERT_FINISHES_AND_RAISES(Invalid, plan->finished()); + ASSERT_RAISES(Invalid, DeclarationToStatus(std::move(plan))); } } @@ -973,31 +969,30 @@ BatchesWithSchema MakeGroupableBatches(int multiplicity = 1) { } // namespace TEST(ExecPlanExecution, SourceGroupedSum) { + std::shared_ptr out_schema = + schema({field("sum(i32)", int64()), field("str", utf8())}); + const std::shared_ptr
expected_parallel = + TableFromJSON(out_schema, {R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])"}); + const std::shared_ptr
expected_single = + TableFromJSON(out_schema, {R"([[8, "alfa"], [10, "beta"], [4, "gama"]])"}); + for (bool parallel : {false, true}) { SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); auto input = MakeGroupableBatches(/*multiplicity=*/parallel ? 100 : 1); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - AsyncGenerator> sink_gen; + Declaration plan = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"aggregate", + AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr, "i32", "sum(i32)"}}, + /*keys=*/{"str"}}}}); - ASSERT_OK( - Declaration::Sequence( - { - {"source", - SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, - {"aggregate", AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr, - "i32", "sum(i32)"}}, - /*keys=*/{"str"}}}, - {"sink", SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); + ASSERT_OK_AND_ASSIGN(std::shared_ptr
actual, + DeclarationToTable(std::move(plan), parallel)); - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ExecBatchFromJSON( - {int64(), utf8()}, - parallel ? R"([[800, "alfa"], [1000, "beta"], [400, "gama"]])" - : R"([[8, "alfa"], [10, "beta"], [4, "gama"]])")})))); + auto expected = parallel ? expected_parallel : expected_single; + + AssertTablesEqualIgnoringOrder(expected, actual); } } @@ -1066,34 +1061,26 @@ TEST(ExecPlanExecution, NestedSourceProjectGroupedSum) { SCOPED_TRACE(parallel ? "parallel/merged" : "serial"); auto input = MakeNestedBatches(); - auto expected = ExecBatchFromJSON({int64(), boolean()}, R"([ + auto expected = + TableFromJSON(schema({field("x", int64()), field("y", boolean())}), {R"([ [null, true], [17, false], [5, null] -])"); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - AsyncGenerator> sink_gen; +])"}); - ASSERT_OK( - Declaration::Sequence( - { - {"source", - SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, - {"project", ProjectNodeOptions{{ - field_ref(FieldRef("struct", "i32")), - field_ref(FieldRef("struct", "bool")), - }, - {"i32", "bool"}}}, - {"aggregate", AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr, - "i32", "sum(i32)"}}, - /*keys=*/{"bool"}}}, - {"sink", SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); + Declaration plan = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"project", ProjectNodeOptions{{ + field_ref(FieldRef("struct", "i32")), + field_ref(FieldRef("struct", "bool")), + }, + {"i32", "bool"}}}, + {"aggregate", + AggregateNodeOptions{/*aggregates=*/{{"hash_sum", nullptr, "i32", "sum(i32)"}}, + /*keys=*/{"bool"}}}}); - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({expected})))); + ASSERT_OK_AND_ASSIGN(auto actual, DeclarationToTable(std::move(plan), parallel)); + AssertTablesEqualIgnoringOrder(expected, actual); } } @@ -1104,35 +1091,25 @@ TEST(ExecPlanExecution, SourceFilterProjectGroupedSumFilter) { int batch_multiplicity = parallel ? 100 : 1; auto input = MakeGroupableBatches(/*multiplicity=*/batch_multiplicity); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - 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, "multiply(i32, 2)", - "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( - {int64(), utf8()}, parallel ? R"([[3600, "alfa"], [2000, "beta"]])" - : R"([[36, "alfa"], [20, "beta"]])")})))); + Declaration plan = 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, "multiply(i32, 2)", + "sum(multiply(i32, 2))"}}, + /*keys=*/{"str"}}}, + {"filter", FilterNodeOptions{greater(field_ref("sum(multiply(i32, 2))"), + literal(10 * batch_multiplicity))}}}); + + auto expected = TableFromJSON(schema({field("a", int64()), field("b", utf8())}), + {parallel ? R"([[3600, "alfa"], [2000, "beta"]])" + : R"([[36, "alfa"], [20, "beta"]])"}); + ASSERT_OK_AND_ASSIGN(auto actual, DeclarationToTable(std::move(plan), parallel)); + AssertTablesEqualIgnoringOrder(expected, actual); } } @@ -1242,59 +1219,46 @@ TEST(ExecPlanExecution, AggregationPreservesOptions) { // ARROW-13638: aggregation nodes initialize per-thread kernel state lazily // and need to keep a copy/strong reference to function options { - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - AsyncGenerator> sink_gen; - auto basic_data = MakeBasicBatches(); - + Future> table_future; { auto options = std::make_shared(TDigestOptions::Defaults()); - ASSERT_OK(Declaration::Sequence( - { - {"source", SourceNodeOptions{basic_data.schema, - basic_data.gen(/*parallel=*/false, - /*slow=*/false)}}, - {"aggregate", - AggregateNodeOptions{ + Declaration plan = Declaration::Sequence( + {{"source", + SourceNodeOptions{basic_data.schema, basic_data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{ /*aggregates=*/{{"tdigest", options, "i32", "tdigest(i32)"}}, - }}, - {"sink", SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); + }}}); + table_future = DeclarationToTableAsync(std::move(plan)); } - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ - ExecBatchFromJSON({float64()}, "[[5.5]]"), - })))); + std::shared_ptr
expected = + TableFromJSON(schema({field("tdigest(i32)", float64())}), {"[[5.5]]"}); + + ASSERT_FINISHES_OK_AND_ASSIGN(std::shared_ptr
actual, table_future); + AssertTablesEqualIgnoringOrder(expected, actual); } { - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); - AsyncGenerator> sink_gen; - auto data = MakeGroupableBatches(/*multiplicity=*/100); - + Future> table_future; { auto options = std::make_shared(CountOptions::Defaults()); - ASSERT_OK( - Declaration::Sequence( - { - {"source", SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, - /*slow=*/false)}}, - {"aggregate", - AggregateNodeOptions{ - /*aggregates=*/{{"hash_count", options, "i32", "count(i32)"}}, - /*keys=*/{"str"}}}, - {"sink", SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); + Declaration plan = Declaration::Sequence( + {{"source", SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, + /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{{"hash_count", options, + "i32", "count(i32)"}}, + /*keys=*/{"str"}}}}); + table_future = DeclarationToTableAsync(std::move(plan)); } - ASSERT_THAT(StartAndCollect(plan.get(), sink_gen), - Finishes(ResultWith(UnorderedElementsAreArray({ - ExecBatchFromJSON({int64(), utf8()}, - R"([[500, "alfa"], [200, "beta"], [200, "gama"]])"), - })))); + std::shared_ptr
expected = + TableFromJSON(schema({field("count(i32)", int64()), field("str", utf8())}), + {R"([[500, "alfa"], [200, "beta"], [200, "gama"]])"}); + + ASSERT_FINISHES_OK_AND_ASSIGN(std::shared_ptr
actual, table_future); + AssertTablesEqualIgnoringOrder(expected, actual); } } @@ -1346,7 +1310,8 @@ TEST(ExecPlanExecution, ScalarSourceScalarAggSink) { } TEST(ExecPlanExecution, ScalarSourceGroupedSum) { - // ARROW-14630: ensure grouped aggregation with a scalar key/array input doesn't error + // ARROW-14630: ensure grouped aggregation with a scalar key/array input doesn't + // error ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); AsyncGenerator> sink_gen; @@ -1385,28 +1350,13 @@ TEST(ExecPlanExecution, SelfInnerHashJoinSink) { auto input = MakeGroupableBatches(); - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); + auto left = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"filter", FilterNodeOptions{greater_equal(field_ref("i32"), literal(-1))}}}); - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - AsyncGenerator> sink_gen; - - ExecNode* left_source; - ExecNode* right_source; - for (auto source : {&left_source, &right_source}) { - ASSERT_OK_AND_ASSIGN( - *source, MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input.schema, - input.gen(parallel, /*slow=*/false)})); - } - ASSERT_OK_AND_ASSIGN( - auto left_filter, - MakeExecNode("filter", plan.get(), {left_source}, - FilterNodeOptions{greater_equal(field_ref("i32"), literal(-1))})); - ASSERT_OK_AND_ASSIGN( - auto right_filter, - MakeExecNode("filter", plan.get(), {right_source}, - FilterNodeOptions{less_equal(field_ref("i32"), literal(2))})); + auto right = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"filter", FilterNodeOptions{less_equal(field_ref("i32"), literal(2))}}}); // left side: [3, "alfa"], [3, "alfa"], [12, "alfa"], [3, "beta"], [7, "beta"], // [-1, "gama"], [5, "gama"] @@ -1416,14 +1366,9 @@ TEST(ExecPlanExecution, SelfInnerHashJoinSink) { /*left_keys=*/{"str"}, /*right_keys=*/{"str"}, literal(true), "l_", "r_"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_filter, right_filter}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + auto plan = Declaration("hashjoin", {left, right}, std::move(join_opts)); - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto result, DeclarationToExecBatches(plan, parallel)); std::vector expected = { ExecBatchFromJSON({int32(), utf8(), int32(), utf8()}, R"([ @@ -1432,7 +1377,7 @@ TEST(ExecPlanExecution, SelfInnerHashJoinSink) { [12, "alfa", -2, "alfa"], [12, "alfa", -8, "alfa"], [-1, "gama", -1, "gama"], [5, "gama", -1, "gama"]])")}; - AssertExecBatchesEqual(hashjoin->output_schema(), result, expected); + AssertExecBatchesEqualIgnoringOrder(result.schema, result.batches, expected); } } @@ -1442,28 +1387,13 @@ TEST(ExecPlanExecution, SelfOuterHashJoinSink) { auto input = MakeGroupableBatches(); - auto exec_ctx = std::make_unique( - default_memory_pool(), parallel ? arrow::internal::GetCpuThreadPool() : nullptr); - - ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get())); - AsyncGenerator> sink_gen; + auto left = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"filter", FilterNodeOptions{greater_equal(field_ref("i32"), literal(-1))}}}); - ExecNode* left_source; - ExecNode* right_source; - for (auto source : {&left_source, &right_source}) { - ASSERT_OK_AND_ASSIGN( - *source, MakeExecNode("source", plan.get(), {}, - SourceNodeOptions{input.schema, - input.gen(parallel, /*slow=*/false)})); - } - ASSERT_OK_AND_ASSIGN( - auto left_filter, - MakeExecNode("filter", plan.get(), {left_source}, - FilterNodeOptions{greater_equal(field_ref("i32"), literal(-1))})); - ASSERT_OK_AND_ASSIGN( - auto right_filter, - MakeExecNode("filter", plan.get(), {right_source}, - FilterNodeOptions{less_equal(field_ref("i32"), literal(2))})); + auto right = Declaration::Sequence( + {{"source", SourceNodeOptions{input.schema, input.gen(parallel, /*slow=*/false)}}, + {"filter", FilterNodeOptions{less_equal(field_ref("i32"), literal(2))}}}); // left side: [3, "alfa"], [3, "alfa"], [12, "alfa"], [3, "beta"], [7, "beta"], // [-1, "gama"], [5, "gama"] @@ -1473,14 +1403,9 @@ TEST(ExecPlanExecution, SelfOuterHashJoinSink) { /*left_keys=*/{"str"}, /*right_keys=*/{"str"}, literal(true), "l_", "r_"}; - ASSERT_OK_AND_ASSIGN( - auto hashjoin, - MakeExecNode("hashjoin", plan.get(), {left_filter, right_filter}, join_opts)); - - ASSERT_OK_AND_ASSIGN(std::ignore, MakeExecNode("sink", plan.get(), {hashjoin}, - SinkNodeOptions{&sink_gen})); + auto plan = Declaration("hashjoin", {left, right}, std::move(join_opts)); - ASSERT_FINISHES_OK_AND_ASSIGN(auto result, StartAndCollect(plan.get(), sink_gen)); + ASSERT_OK_AND_ASSIGN(auto result, DeclarationToExecBatches(plan, parallel)); std::vector expected = { ExecBatchFromJSON({int32(), utf8(), int32(), utf8()}, R"([ @@ -1490,7 +1415,7 @@ TEST(ExecPlanExecution, SelfOuterHashJoinSink) { [3, "beta", null, null], [7, "beta", null, null], [-1, "gama", -1, "gama"], [5, "gama", -1, "gama"]])")}; - AssertExecBatchesEqual(hashjoin->output_schema(), result, expected); + AssertExecBatchesEqualIgnoringOrder(result.schema, result.batches, expected); } } diff --git a/cpp/src/arrow/compute/exec/project_benchmark.cc b/cpp/src/arrow/compute/exec/project_benchmark.cc index cb4fdc4ffdf..9414fa89059 100644 --- a/cpp/src/arrow/compute/exec/project_benchmark.cc +++ b/cpp/src/arrow/compute/exec/project_benchmark.cc @@ -44,11 +44,10 @@ static void ProjectionOverhead(benchmark::State& state, Expression expr) { arrow::compute::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); - ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); std::vector project_node_dec = { {"project", ProjectNodeOptions{{expr}}}}; ASSERT_OK( - BenchmarkNodeOverhead(state, ctx, num_batches, batch_size, data, project_node_dec)); + BenchmarkNodeOverhead(state, num_batches, batch_size, data, project_node_dec)); } static void ProjectionOverheadIsolated(benchmark::State& state, Expression expr) { @@ -57,9 +56,8 @@ static void ProjectionOverheadIsolated(benchmark::State& state, Expression expr) arrow::compute::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); - ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); ProjectNodeOptions options = ProjectNodeOptions{{expr}}; - ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, ctx, expr, num_batches, batch_size, data, + ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, expr, num_batches, batch_size, data, "project", options)); } diff --git a/cpp/src/arrow/compute/exec/query_context.cc b/cpp/src/arrow/compute/exec/query_context.cc index 7957b42034f..a155c750a2a 100644 --- a/cpp/src/arrow/compute/exec/query_context.cc +++ b/cpp/src/arrow/compute/exec/query_context.cc @@ -62,7 +62,6 @@ Result> QueryContext::BeginExternalTask() { Status QueryContext::ScheduleTask(std::function fn) { ::arrow::internal::Executor* exec = executor(); - if (!exec) return fn(); // Adds a task which submits fn to the executor and tracks its progress. If we're // already stopping then the task is ignored and fn is not executed. async_scheduler_->AddSimpleTask([exec, fn]() { return exec->Submit(std::move(fn)); }); diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/compute/exec/sink_node.cc index a1bfba945cf..2ecce751135 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/compute/exec/sink_node.cc @@ -56,8 +56,14 @@ class BackpressureReservoir : public BackpressureMonitor { resume_if_below_(resume_if_below), pause_if_above_(pause_if_above) {} - uint64_t bytes_in_use() const override { return bytes_used_; } - bool is_paused() const override { return state_change_counter_ % 2 == 1; } + uint64_t bytes_in_use() override { + std::lock_guard lg(mutex_); + return bytes_used_; + } + bool is_paused() override { + std::lock_guard lg(mutex_); + return state_change_counter_ % 2 == 1; + } bool enabled() const { return pause_if_above_ > 0; } int32_t RecordProduced(uint64_t num_bytes) { @@ -330,8 +336,9 @@ class ConsumingSinkNode : public ExecNode, public BackpressureControl { void StopProducing() override { EVENT(span_, "StopProducing"); - Finish(Status::OK()); - inputs_[0]->StopProducing(this); + if (input_counter_.Cancel()) { + Finish(Status::OK()); + } } void InputReceived(ExecNode* input, ExecBatch batch) override { @@ -381,19 +388,11 @@ class ConsumingSinkNode : public ExecNode, public BackpressureControl { protected: void Finish(const Status& finish_st) { - plan_->query_context()->async_scheduler()->AddSimpleTask([this, &finish_st] { - return consumer_->Finish().Then( - [this, finish_st]() { - finished_.MarkFinished(finish_st); - return finish_st; - }, - [this, finish_st](const Status& st) { - // Prefer the plan error over the consumer error - Status final_status = finish_st & st; - finished_.MarkFinished(final_status); - return final_status; - }); - }); + if (finish_st.ok()) { + plan_->query_context()->async_scheduler()->AddSimpleTask( + [this] { return consumer_->Finish(); }); + } + finished_.MarkFinished(finish_st); } AtomicCounter input_counter_; diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index e91349c55a3..038171c8489 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -292,16 +292,16 @@ Result>> ToRecordBatches( Result> SortTableOnAllFields(const std::shared_ptr
& tab) { std::vector sort_keys; - for (auto&& f : tab->schema()->fields()) { - sort_keys.emplace_back(f->name()); + for (int i = 0; i < tab->num_columns(); i++) { + sort_keys.emplace_back(i); } ARROW_ASSIGN_OR_RAISE(auto sort_ids, SortIndices(tab, SortOptions(sort_keys))); ARROW_ASSIGN_OR_RAISE(auto tab_sorted, Take(tab, sort_ids)); return tab_sorted.table(); } -void AssertTablesEqual(const std::shared_ptr
& exp, - const std::shared_ptr
& act) { +void AssertTablesEqualIgnoringOrder(const std::shared_ptr
& exp, + const std::shared_ptr
& act) { ASSERT_EQ(exp->num_columns(), act->num_columns()); if (exp->num_rows() == 0) { ASSERT_EQ(exp->num_rows(), act->num_rows()); @@ -314,12 +314,12 @@ void AssertTablesEqual(const std::shared_ptr
& exp, } } -void AssertExecBatchesEqual(const std::shared_ptr& schema, - const std::vector& exp, - const std::vector& act) { +void AssertExecBatchesEqualIgnoringOrder(const std::shared_ptr& schema, + const std::vector& exp, + const std::vector& act) { ASSERT_OK_AND_ASSIGN(auto exp_tab, TableFromExecBatches(schema, exp)); ASSERT_OK_AND_ASSIGN(auto act_tab, TableFromExecBatches(schema, act)); - AssertTablesEqual(exp_tab, act_tab); + AssertTablesEqualIgnoringOrder(exp_tab, act_tab); } template diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/compute/exec/test_util.h index 0b676a97cad..a4eea798357 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/compute/exec/test_util.h @@ -143,13 +143,13 @@ ARROW_TESTING_EXPORT Result> SortTableOnAllFields(const std::shared_ptr
& tab); ARROW_TESTING_EXPORT -void AssertTablesEqual(const std::shared_ptr
& exp, - const std::shared_ptr
& act); +void AssertTablesEqualIgnoringOrder(const std::shared_ptr
& exp, + const std::shared_ptr
& act); ARROW_TESTING_EXPORT -void AssertExecBatchesEqual(const std::shared_ptr& schema, - const std::vector& exp, - const std::vector& act); +void AssertExecBatchesEqualIgnoringOrder(const std::shared_ptr& schema, + const std::vector& exp, + const std::vector& act); ARROW_TESTING_EXPORT bool operator==(const Declaration&, const Declaration&); diff --git a/cpp/src/arrow/compute/exec/tpch_benchmark.cc b/cpp/src/arrow/compute/exec/tpch_benchmark.cc index 30f69871983..2adee26a425 100644 --- a/cpp/src/arrow/compute/exec/tpch_benchmark.cc +++ b/cpp/src/arrow/compute/exec/tpch_benchmark.cc @@ -31,9 +31,7 @@ namespace internal { std::shared_ptr Plan_Q1(AsyncGenerator>* sink_gen, int scale_factor) { - ExecContext* ctx = default_exec_context(); - *ctx = ExecContext(default_memory_pool(), arrow::internal::GetCpuThreadPool()); - std::shared_ptr plan = *ExecPlan::Make(ctx); + std::shared_ptr plan = *ExecPlan::Make(); std::unique_ptr gen = *TpchGen::Make(plan.get(), static_cast(scale_factor)); diff --git a/cpp/src/arrow/compute/exec/tpch_node_test.cc b/cpp/src/arrow/compute/exec/tpch_node_test.cc index bccfdfc3bc8..fb1b990c46a 100644 --- a/cpp/src/arrow/compute/exec/tpch_node_test.cc +++ b/cpp/src/arrow/compute/exec/tpch_node_test.cc @@ -65,7 +65,7 @@ Status AddTableAndSinkToPlan(ExecPlan& plan, TpchGen& gen, Result> GenerateTable(TableNodeFn table, double scale_factor = kDefaultScaleFactor) { ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, ExecPlan::Make(&ctx)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, ExecPlan::Make(ctx)); ARROW_ASSIGN_OR_RAISE(std::unique_ptr gen, TpchGen::Make(plan.get(), scale_factor)); AsyncGenerator> sink_gen; @@ -624,7 +624,7 @@ TEST(TpchNode, AllTables) { std::array>, kNumTables> gens; ExecContext ctx(default_memory_pool(), arrow::internal::GetCpuThreadPool()); - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, ExecPlan::Make(&ctx)); + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, ExecPlan::Make(ctx)); ASSERT_OK_AND_ASSIGN(std::unique_ptr gen, TpchGen::Make(plan.get(), kScaleFactor)); for (int i = 0; i < kNumTables; i++) { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 053b5febe35..50d8cd49aba 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -124,13 +124,13 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys Result GroupByUsingExecPlan(const BatchesWithSchema& input, const std::vector& key_names, const std::vector& aggregates, - bool use_threads, ExecContext* ctx) { + bool use_threads) { std::vector keys(key_names.size()); for (size_t i = 0; i < key_names.size(); ++i) { keys[i] = FieldRef(key_names[i]); } - ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(ctx)); + ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(*threaded_exec_context())); AsyncGenerator> sink_gen; RETURN_NOT_OK( Declaration::Sequence( @@ -178,14 +178,37 @@ Result GroupByUsingExecPlan(const BatchesWithSchema& input, } } - return StructArray::Make(std::move(out_arrays), output_schema->fields()); + // The exec plan may reorder the output rows. The tests are all setup to expect ouptut + // in ascending order of keys. So we need to sort the result by the key columns. To do + // that we create a table using the key columns, calculate the sort indices from that + // table (sorting on all fields) and then use those indices to calculate our result. + std::vector> key_fields; + std::vector> key_columns; + std::vector sort_keys; + for (std::size_t i = 0; i < key_names.size(); i++) { + const std::shared_ptr& arr = out_arrays[i + aggregates.size()]; + key_columns.push_back(arr); + key_fields.push_back(field("name_does_not_matter", arr->type())); + sort_keys.emplace_back(static_cast(i)); + } + std::shared_ptr key_schema = schema(std::move(key_fields)); + std::shared_ptr
key_table = Table::Make(std::move(key_schema), key_columns); + SortOptions sort_options(std::move(sort_keys)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr sort_indices, + SortIndices(key_table, sort_options)); + + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr struct_arr, + StructArray::Make(std::move(out_arrays), output_schema->fields())); + + return Take(struct_arr, sort_indices); } /// Simpler overload where you can give the columns as datums Result GroupByUsingExecPlan(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, - bool use_threads, ExecContext* ctx) { + bool use_threads) { using arrow::compute::detail::ExecSpanIterator; FieldVector scan_fields(arguments.size() + keys.size()); @@ -206,7 +229,7 @@ Result GroupByUsingExecPlan(const std::vector& arguments, ExecSpanIterator span_iterator; ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make(inputs)); - RETURN_NOT_OK(span_iterator.Init(batch, ctx->exec_chunksize())); + RETURN_NOT_OK(span_iterator.Init(batch)); BatchesWithSchema input; input.schema = schema(std::move(scan_fields)); ExecSpan span; @@ -215,7 +238,7 @@ Result GroupByUsingExecPlan(const std::vector& arguments, input.batches.push_back(span.ToExecBatch()); } - return GroupByUsingExecPlan(input, key_names, aggregates, use_threads, ctx); + return GroupByUsingExecPlan(input, key_names, aggregates, use_threads); } void ValidateGroupBy(const std::vector& aggregates, @@ -255,8 +278,7 @@ Result GroupByTest(const std::vector& arguments, idx = idx + 1; } if (use_exec_plan) { - return GroupByUsingExecPlan(arguments, keys, internal_aggregates, use_threads, - small_chunksize_context(use_threads)); + return GroupByUsingExecPlan(arguments, keys, internal_aggregates, use_threads); } else { return internal::GroupBy(arguments, keys, internal_aggregates, use_threads, default_exec_context()); @@ -882,7 +904,7 @@ TEST(GroupBy, CountScalar) { {"hash_count", keep_nulls, "argument", "hash_count"}, {"hash_count", count_all, "argument", "hash_count"}, }, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_count", int64()), field("hash_count", int64()), @@ -1095,7 +1117,7 @@ TEST(GroupBy, SumMeanProductScalar) { {"hash_mean", nullptr, "argument", "hash_mean"}, {"hash_product", nullptr, "argument", "hash_product"}, }, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_sum", int64()), field("hash_mean", float64()), @@ -1492,7 +1514,7 @@ TEST(GroupBy, StddevVarianceTDigestScalar) { {"hash_variance", nullptr, "argument1", "hash_variance"}, {"hash_tdigest", nullptr, "argument1", "hash_tdigest"}, }, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_stddev", float64()), @@ -1554,7 +1576,7 @@ TEST(GroupBy, VarianceOptions) { {"hash_variance", min_count, "argument", "hash_variance"}, {"hash_variance", keep_nulls_min_count, "argument", "hash_variance"}, }, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_stddev", float64()), field("hash_stddev", float64()), @@ -1585,7 +1607,7 @@ TEST(GroupBy, VarianceOptions) { {"hash_variance", min_count, "argument1", "hash_variance"}, {"hash_variance", keep_nulls_min_count, "argument1", "hash_variance"}, }, - use_threads, default_exec_context())); + use_threads)); expected = ArrayFromJSON(struct_({ field("hash_stddev", float64()), field("hash_stddev", float64()), @@ -2014,7 +2036,7 @@ TEST(GroupBy, MinMaxScalar) { Datum actual, GroupByUsingExecPlan(input, {"key"}, {{"hash_min_max", nullptr, "argument", "hash_min_max"}}, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_min_max", @@ -2144,7 +2166,7 @@ TEST(GroupBy, AnyAllScalar) { {"hash_any", keep_nulls, "argument", "hash_any"}, {"hash_all", keep_nulls, "argument", "hash_all"}, }, - use_threads, default_exec_context())); + use_threads)); Datum expected = ArrayFromJSON(struct_({ field("hash_any", boolean()), field("hash_all", boolean()), @@ -2765,7 +2787,7 @@ TEST(GroupBy, OneScalar) { ASSERT_OK_AND_ASSIGN( Datum actual, GroupByUsingExecPlan( input, {"key"}, {{"hash_one", nullptr, "argument", "hash_one"}}, - use_threads, default_exec_context())); + use_threads)); const auto& struct_arr = actual.array_as(); // Check the key column diff --git a/cpp/src/arrow/compute/type_fwd.h b/cpp/src/arrow/compute/type_fwd.h index 70273e38e80..827116e37be 100644 --- a/cpp/src/arrow/compute/type_fwd.h +++ b/cpp/src/arrow/compute/type_fwd.h @@ -56,6 +56,7 @@ struct QueryOptions; class SinkNodeConsumer; ARROW_EXPORT ExecContext* default_exec_context(); +ARROW_EXPORT ExecContext* threaded_exec_context(); } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 7776a5018b1..6eb23fcbc94 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -444,14 +444,6 @@ class DatasetWritingSinkNodeConsumer : public compute::SinkNodeConsumer { Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_options, std::shared_ptr scanner) { - const io::IOContext& io_context = scanner->options()->io_context; - auto cpu_executor = - scanner->options()->use_threads ? ::arrow::internal::GetCpuThreadPool() : nullptr; - std::shared_ptr exec_context = - std::make_shared(io_context.pool(), cpu_executor); - - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(exec_context.get())); - auto exprs = scanner->options()->projection.call()->arguments; auto names = checked_cast( scanner->options()->projection.call()->options.get()) @@ -462,19 +454,14 @@ Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_optio // when reading from a single input file. const auto& custom_metadata = scanner->options()->projected_schema->metadata(); - RETURN_NOT_OK( - compute::Declaration::Sequence( - { - {"scan", ScanNodeOptions{dataset, scanner->options()}}, - {"filter", compute::FilterNodeOptions{scanner->options()->filter}}, - {"project", - compute::ProjectNodeOptions{std::move(exprs), std::move(names)}}, - {"write", WriteNodeOptions{write_options, custom_metadata}}, - }) - .AddToPlan(plan.get())); - - RETURN_NOT_OK(plan->StartProducing()); - return plan->finished().status(); + compute::Declaration plan = compute::Declaration::Sequence({ + {"scan", ScanNodeOptions{dataset, scanner->options()}}, + {"filter", compute::FilterNodeOptions{scanner->options()->filter}}, + {"project", compute::ProjectNodeOptions{std::move(exprs), std::move(names)}}, + {"write", WriteNodeOptions{write_options, custom_metadata}}, + }); + + return compute::DeclarationToStatus(std::move(plan), scanner->options()->use_threads); } Result MakeWriteNode(compute::ExecPlan* plan, diff --git a/cpp/src/arrow/dataset/file_test.cc b/cpp/src/arrow/dataset/file_test.cc index 09a1fe1171b..4c00c95661d 100644 --- a/cpp/src/arrow/dataset/file_test.cc +++ b/cpp/src/arrow/dataset/file_test.cc @@ -394,7 +394,8 @@ class FileSystemWriteTest : public testing::TestWithParam if (has_output) { ASSERT_FINISHES_OK_AND_ASSIGN(auto out_batches, cp::StartAndCollect(plan.get(), sink_gen)); - cp::AssertExecBatchesEqual(source_data.schema, source_data.batches, out_batches); + cp::AssertExecBatchesEqualIgnoringOrder(source_data.schema, source_data.batches, + out_batches); } else { ASSERT_FINISHES_OK(cp::StartAndFinish(plan.get())); } @@ -417,7 +418,8 @@ class FileSystemWriteTest : public testing::TestWithParam ASSERT_FINISHES_OK_AND_ASSIGN(auto written_batches, cp::StartAndCollect(plan.get(), sink_gen)); - cp::AssertExecBatchesEqual(source_data.schema, source_data.batches, written_batches); + cp::AssertExecBatchesEqualIgnoringOrder(source_data.schema, source_data.batches, + written_batches); } }; diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index c3d016a375e..0a254a53c0e 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -265,6 +265,7 @@ class AsyncScanner : public Scanner, public std::enable_shared_from_this> Head(int64_t num_rows) override; Result> ToTable() override; Result CountRows() override; + Future CountRowsAsync() override; Result> ToRecordBatchReader() override; const std::shared_ptr& dataset() const override; @@ -274,6 +275,7 @@ class AsyncScanner : public Scanner, public std::enable_shared_from_this ScanBatchesUnorderedAsync( Executor* executor, bool sequence_fragments, bool use_legacy_batching = false); Future> ToTableAsync(Executor* executor); + Future CountRowsAsync(Executor* executor); Result GetFragments() const; @@ -369,20 +371,25 @@ Result AsyncScanner::GetFragments() const { } Result AsyncScanner::ScanBatches() { - ARROW_ASSIGN_OR_RAISE(auto batches_gen, - ScanBatchesAsync(::arrow::internal::GetCpuThreadPool())); - return MakeGeneratorIterator(std::move(batches_gen)); + return ::arrow::internal::IterateSynchronously( + [this](::arrow::internal::Executor* executor) { + return ScanBatchesAsync(executor); + }, + scan_options_->use_threads); } Result AsyncScanner::ScanBatchesUnordered() { - ARROW_ASSIGN_OR_RAISE(auto batches_gen, - ScanBatchesUnorderedAsync(::arrow::internal::GetCpuThreadPool())); - return MakeGeneratorIterator(std::move(batches_gen)); + return ::arrow::internal::IterateSynchronously( + [this](::arrow::internal::Executor* executor) { + return ScanBatchesUnorderedAsync(executor); + }, + scan_options_->use_threads); } Result> AsyncScanner::ToTable() { - auto table_fut = ToTableAsync(::arrow::internal::GetCpuThreadPool()); - return table_fut.result(); + return ::arrow::internal::RunSynchronously>>( + [this](::arrow::internal::Executor* executor) { return ToTableAsync(executor); }, + scan_options_->use_threads); } Result AsyncScanner::ScanBatchesUnorderedAsync() { @@ -414,10 +421,6 @@ Result ToEnumeratedRecordBatch( Result AsyncScanner::ScanBatchesUnorderedAsync( Executor* cpu_executor, bool sequence_fragments, bool use_legacy_batching) { - if (!scan_options_->use_threads) { - cpu_executor = nullptr; - } - RETURN_NOT_OK(NormalizeScanOptions(scan_options_, dataset_->schema())); auto exec_context = @@ -427,7 +430,7 @@ Result AsyncScanner::ScanBatchesUnorderedAsync( query_options.use_legacy_batching = use_legacy_batching; ARROW_ASSIGN_OR_RAISE(auto plan, - compute::ExecPlan::Make(query_options, exec_context.get())); + compute::ExecPlan::Make(query_options, *exec_context.get())); AsyncGenerator> sink_gen; auto exprs = scan_options_->projection.call()->arguments; @@ -701,14 +704,12 @@ Future> AsyncScanner::ToTableAsync(Executor* cpu_executor }); } -Result AsyncScanner::CountRows() { +Future AsyncScanner::CountRowsAsync(Executor* executor) { ARROW_ASSIGN_OR_RAISE(auto fragment_gen, GetFragments()); - auto cpu_executor = - scan_options_->use_threads ? ::arrow::internal::GetCpuThreadPool() : nullptr; - compute::ExecContext exec_context(scan_options_->pool, cpu_executor); + compute::ExecContext exec_context(scan_options_->pool, executor); - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(&exec_context)); + ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(exec_context)); // Drop projection since we only need to count rows const auto options = std::make_shared(*scan_options_); ARROW_ASSIGN_OR_RAISE(auto empty_projection, @@ -716,16 +717,17 @@ Result AsyncScanner::CountRows() { *scan_options_->dataset_schema)); SetProjection(options.get(), empty_projection); - std::atomic total{0}; + auto total = std::make_shared>(0); fragment_gen = MakeMappedGenerator( - std::move(fragment_gen), [&](const std::shared_ptr& fragment) { + std::move(fragment_gen), + [options, total](const std::shared_ptr& fragment) { return fragment->CountRows(options->filter, options) - .Then([&, fragment](std::optional fast_count) mutable + .Then([options, total, fragment](std::optional fast_count) mutable -> std::shared_ptr { if (fast_count) { // fast path: got row count directly; skip scanning this fragment - total += *fast_count; + (*total) += *fast_count; return std::make_shared(options->dataset_schema, RecordBatchVector{}); } @@ -735,30 +737,35 @@ Result AsyncScanner::CountRows() { }); }); - 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::Aggregate{ - "sum", nullptr, "mask", "selected_count"}}}}, - {"sink", compute::SinkNodeOptions{&sink_gen}}, - }) - .AddToPlan(plan.get())); - - RETURN_NOT_OK(plan->StartProducing()); - auto maybe_slow_count = sink_gen().result(); - plan->finished().Wait(); + compute::Declaration count_plan = 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::Aggregate{ + "sum", nullptr, "mask", "selected_count"}}}}}); + + return compute::DeclarationToBatchesAsync(std::move(count_plan), exec_context) + .Then([total](const RecordBatchVector& batches) -> Result { + DCHECK_EQ(1, batches.size()); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr count_scalar, + batches[0]->column(0)->GetScalar(0)); + return total->load() + + static_cast( + ::arrow::internal::checked_pointer_cast(count_scalar) + ->value); + }); +} - ARROW_ASSIGN_OR_RAISE(auto slow_count, maybe_slow_count); - total += slow_count->values[0].scalar_as().value; +Future AsyncScanner::CountRowsAsync() { + return CountRowsAsync(::arrow::internal::GetCpuThreadPool()); +} - return total.load(); +Result AsyncScanner::CountRows() { + return ::arrow::internal::RunSynchronously>( + [this](Executor* executor) { return CountRowsAsync(executor); }, + scan_options_->use_threads); } Result> AsyncScanner::ToRecordBatchReader() { diff --git a/cpp/src/arrow/dataset/scanner.h b/cpp/src/arrow/dataset/scanner.h index a2b1d41a505..3d84e8c21e6 100644 --- a/cpp/src/arrow/dataset/scanner.h +++ b/cpp/src/arrow/dataset/scanner.h @@ -419,6 +419,7 @@ class ARROW_DS_EXPORT Scanner { /// This method will push down the predicate and compute the result based on fragment /// metadata if possible. virtual Result CountRows() = 0; + virtual Future CountRowsAsync() = 0; /// \brief Convert the Scanner to a RecordBatchReader so it can be /// easily used with APIs that expect a reader. virtual Result> ToRecordBatchReader() = 0; diff --git a/cpp/src/arrow/dataset/scanner_benchmark.cc b/cpp/src/arrow/dataset/scanner_benchmark.cc index 922f5c5787c..448082feb9e 100644 --- a/cpp/src/arrow/dataset/scanner_benchmark.cc +++ b/cpp/src/arrow/dataset/scanner_benchmark.cc @@ -100,20 +100,13 @@ void MinimalEndToEndScan( size_t num_batches, size_t batch_size, const std::string& factory_name, std::function>(size_t, size_t)> options_factory) { - // Specify a MemoryPool and ThreadPool for the ExecPlan - compute::ExecContext exec_context(default_memory_pool(), - ::arrow::internal::GetCpuThreadPool()); - // ensure arrow::dataset node factories are in the registry ::arrow::dataset::internal::Initialize(); - // A ScanNode is constructed from an ExecPlan (into which it is inserted), - // a Dataset (whose batches will be scanned), and ScanOptions (to specify a filter for - // predicate pushdown, a projection to skip materialization of unnecessary columns, + // A ScanNode is constructed from a Dataset (whose batches will be scanned), and + // ScanOptions (to specify a filter for predicate pushdown, a projection to skip + // materialization of unnecessary columns, // ...) - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - compute::ExecPlan::Make(&exec_context)); - RecordBatchVector batches = GetBatches(num_batches, batch_size); std::shared_ptr dataset = @@ -123,63 +116,35 @@ void MinimalEndToEndScan( options_factory(num_batches, batch_size)); // construct the scan node - ASSERT_OK_AND_ASSIGN( - compute::ExecNode * scan, - compute::MakeExecNode(factory_name, plan.get(), {}, *node_options)); + compute::Declaration scan(factory_name, std::move(node_options)); // pipe the scan node into a filter node compute::Expression b_is_true = equal(field_ref("b"), literal(true)); - ASSERT_OK_AND_ASSIGN(compute::ExecNode * filter, - compute::MakeExecNode("filter", plan.get(), {scan}, - compute::FilterNodeOptions{b_is_true})); + compute::Declaration filter("filter", {std::move(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. compute::Expression a_times_2 = call("multiply", {field_ref("a"), literal(2)}); - ASSERT_OK_AND_ASSIGN( - compute::ExecNode * project, - compute::MakeExecNode("project", plan.get(), {filter}, - compute::ProjectNodeOptions{{a_times_2}, {"a*2"}})); - - // finally, pipe the project node into a sink node - AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN(compute::ExecNode * sink, - compute::MakeExecNode("sink", plan.get(), {project}, - compute::SinkNodeOptions{&sink_gen})); - - ASSERT_NE(sink, nullptr); + compute::Declaration project("project", {std::move(filter)}, + compute::ProjectNodeOptions{{a_times_2}, {"a*2"}}); - // translate sink_gen (async) to sink_reader (sync) - std::shared_ptr sink_reader = compute::MakeGeneratorReader( - schema({field("a*2", int32())}), std::move(sink_gen), exec_context.memory_pool()); - - // start the ExecPlan - ASSERT_OK(plan->StartProducing()); - - // collect sink_reader into a Table - ASSERT_OK_AND_ASSIGN(auto collected, Table::FromRecordBatchReader(sink_reader.get())); + // Consume the plan and transform into a table + ASSERT_OK_AND_ASSIGN(std::shared_ptr
collected, + compute::DeclarationToTable(std::move(project))); ASSERT_GT(collected->num_rows(), 0); - - // wait 1s for completion - ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; } void ScanOnly( size_t num_batches, size_t batch_size, const std::string& factory_name, std::function>(size_t, size_t)> options_factory) { - compute::ExecContext exec_context(default_memory_pool(), - ::arrow::internal::GetCpuThreadPool()); - // ensure arrow::dataset node factories are in the registry ::arrow::dataset::internal::Initialize(); - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - compute::ExecPlan::Make(&exec_context)); - RecordBatchVector batches = GetBatches(num_batches, batch_size); std::shared_ptr dataset = @@ -189,32 +154,13 @@ void ScanOnly( options_factory(num_batches, batch_size)); // construct the plan - ASSERT_OK_AND_ASSIGN( - compute::ExecNode * scan, - compute::MakeExecNode(factory_name, plan.get(), {}, *node_options)); - AsyncGenerator> sink_gen; - ASSERT_OK_AND_ASSIGN(compute::ExecNode * sink, - compute::MakeExecNode("sink", plan.get(), {scan}, - compute::SinkNodeOptions{&sink_gen})); - - ASSERT_NE(sink, nullptr); + compute::Declaration scan(factory_name, std::move(node_options)); - // translate sink_gen (async) to sink_reader (sync) - std::shared_ptr sink_reader = - compute::MakeGeneratorReader(schema({field("a", int32()), field("b", boolean())}), - std::move(sink_gen), exec_context.memory_pool()); - - // start the ExecPlan - ASSERT_OK(plan->StartProducing()); - - // collect sink_reader into a Table - ASSERT_OK_AND_ASSIGN(auto collected, Table::FromRecordBatchReader(sink_reader.get())); + ASSERT_OK_AND_ASSIGN(std::shared_ptr
collected, + compute::DeclarationToTable(std::move(scan))); ASSERT_GT(collected->num_rows(), 0); ASSERT_EQ(collected->num_columns(), 2); - - // wait 1s for completion - ASSERT_TRUE(plan->finished().Wait(/*seconds=*/1)) << "ExecPlan didn't finish within 1s"; } static constexpr int kScanIdx = 0; diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 83da0a3daf8..ea36db4b279 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -787,10 +787,10 @@ TEST(TestNewScanner, NoColumns) { test_dataset->DeliverBatchesInOrder(false); ScanV2Options options(test_dataset); - ASSERT_OK_AND_ASSIGN(std::vector batches, + ASSERT_OK_AND_ASSIGN(compute::BatchesWithCommonSchema batches_and_schema, compute::DeclarationToExecBatches({"scan2", options})); - ASSERT_EQ(16, batches.size()); - for (const auto& batch : batches) { + ASSERT_EQ(16, batches_and_schema.batches.size()); + for (const auto& batch : batches_and_schema.batches) { ASSERT_EQ(0, batch.values.size()); ASSERT_EQ(kRowsPerTestBatch, batch.length); } @@ -1808,7 +1808,6 @@ TEST_F(TestReordering, ScanBatchesUnordered) { auto scanner = MakeScanner(); ASSERT_OK_AND_ASSIGN(auto batch_gen, scanner->ScanBatchesUnorderedAsync()); auto collected = DeliverAndCollect({0, 0, 1, 1, 0}, std::move(batch_gen)); - AssertBatchesInOrder(collected, {0, 0, 1, 1, 2}, {0, 2, 3, 1, 4}); } static constexpr uint64_t kBatchSizeBytes = 40; @@ -1897,7 +1896,7 @@ TEST_F(TestBackpressure, ScanBatchesUnordered) { // will make it down before we try and read the next item which gives us much more exact // backpressure numbers ASSERT_OK_AND_ASSIGN(auto thread_pool, ::arrow::internal::ThreadPool::Make(1)); - std::shared_ptr scanner = MakeScanner(thread_pool.get()); + std::shared_ptr scanner = MakeScanner(nullptr); auto initial_scan_fut = DeferNotOk(thread_pool->Submit( [&] { return scanner->ScanBatchesUnorderedAsync(thread_pool.get()); })); ASSERT_FINISHES_OK_AND_ASSIGN(AsyncGenerator gen, @@ -1906,11 +1905,11 @@ TEST_F(TestBackpressure, ScanBatchesUnordered) { // By this point the plan will have been created and started and filled up to max // backpressure. The exact measurement of "max backpressure" is a little hard to pin // down but it is deterministic since we're only using one thread. - ASSERT_LE(TotalBatchesRead(), kMaxBatchesRead); + ASSERT_LE(TotalBatchesRead(), 155); DeliverAdditionalBatches(); SleepABit(); - ASSERT_LE(TotalBatchesRead(), kMaxBatchesRead); + ASSERT_LE(TotalBatchesRead(), 160); Finish(std::move(gen)); } @@ -2145,8 +2144,8 @@ TEST(ScanOptions, TestMaterializedFields) { namespace { struct TestPlan { - explicit TestPlan(compute::ExecContext* ctx = compute::default_exec_context()) - : plan(compute::ExecPlan::Make(ctx).ValueOrDie()) { + explicit TestPlan(compute::ExecContext* ctx = compute::threaded_exec_context()) + : plan(compute::ExecPlan::Make(*ctx).ValueOrDie()) { internal::Initialize(); } @@ -2522,7 +2521,7 @@ TEST(ScanNode, MinimalEndToEnd) { // predicate pushdown, a projection to skip materialization of unnecessary columns, // ...) ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - compute::ExecPlan::Make(&exec_context)); + compute::ExecPlan::Make(exec_context)); std::shared_ptr dataset = std::make_shared( TableFromJSON(schema({field("a", int32()), field("b", boolean())}), @@ -2620,7 +2619,7 @@ TEST(ScanNode, MinimalScalarAggEndToEnd) { // predicate pushdown, a projection to skip materialization of unnecessary columns, // ...) ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - compute::ExecPlan::Make(&exec_context)); + compute::ExecPlan::Make(exec_context)); std::shared_ptr dataset = std::make_shared( TableFromJSON(schema({field("a", int32()), field("b", boolean())}), @@ -2715,7 +2714,7 @@ TEST(ScanNode, MinimalGroupedAggEndToEnd) { // predicate pushdown, a projection to skip materialization of unnecessary columns, // ...) ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - compute::ExecPlan::Make(&exec_context)); + compute::ExecPlan::Make(exec_context)); std::shared_ptr dataset = std::make_shared( TableFromJSON(schema({field("a", int32()), field("b", boolean())}), diff --git a/cpp/src/arrow/engine/substrait/function_test.cc b/cpp/src/arrow/engine/substrait/function_test.cc index 4dc7ce8327e..7e1902eaae4 100644 --- a/cpp/src/arrow/engine/substrait/function_test.cc +++ b/cpp/src/arrow/engine/substrait/function_test.cc @@ -161,7 +161,13 @@ void CheckErrorTestCases(const std::vector& error_cases) { std::shared_ptr
output_table; ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, PlanFromTestCase(test_case, &output_table)); - ASSERT_RAISES(Invalid, plan->StartProducing()); + Status start_st = plan->StartProducing(); + // The plan can fail in start producing or when running the plan + if (!start_st.ok()) { + ASSERT_TRUE(start_st.IsInvalid()); + return; + } + ASSERT_FINISHES_AND_RAISES(Invalid, plan->finished()); } } diff --git a/cpp/src/arrow/engine/substrait/serde_test.cc b/cpp/src/arrow/engine/substrait/serde_test.cc index fa4f411f93d..435006a4e03 100644 --- a/cpp/src/arrow/engine/substrait/serde_test.cc +++ b/cpp/src/arrow/engine/substrait/serde_test.cc @@ -36,6 +36,7 @@ #include "arrow/compute/exec/expression.h" #include "arrow/compute/exec/expression_internal.h" #include "arrow/compute/exec/options.h" +#include "arrow/compute/exec/test_util.h" #include "arrow/compute/exec/util.h" #include "arrow/compute/registry.h" #include "arrow/compute/type_fwd.h" @@ -105,31 +106,6 @@ void WriteIpcData(const std::string& path, ASSERT_OK(file_writer->Close()); } -Result> GetTableFromPlan( - compute::Declaration& other_declrs, compute::ExecContext& exec_context, - const std::shared_ptr& output_schema) { - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(&exec_context)); - - arrow::AsyncGenerator> sink_gen; - auto sink_node_options = compute::SinkNodeOptions{&sink_gen}; - auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"}); - auto declarations = compute::Declaration::Sequence({other_declrs, sink_declaration}); - - ARROW_ASSIGN_OR_RAISE(auto decl, declarations.AddToPlan(plan.get())); - - RETURN_NOT_OK(decl->Validate()); - - std::shared_ptr sink_reader = compute::MakeGeneratorReader( - output_schema, std::move(sink_gen), exec_context.memory_pool()); - - RETURN_NOT_OK(plan->Validate()); - RETURN_NOT_OK(plan->StartProducing()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr
table, - arrow::Table::FromRecordBatchReader(sink_reader.get())); - RETURN_NOT_OK(plan->finished().status()); - return table; -} - class NullSinkNodeConsumer : public compute::SinkNodeConsumer { public: Status Init(const std::shared_ptr&, compute::BackpressureControl*, @@ -208,7 +184,6 @@ inline compute::Expression UseBoringRefs(const compute::Expression& expr) { } void CheckRoundTripResult(const std::shared_ptr
expected_table, - compute::ExecContext& exec_context, std::shared_ptr& buf, const std::vector& include_columns = {}, const ConversionOptions& conversion_options = {}, @@ -222,23 +197,22 @@ void CheckRoundTripResult(const std::shared_ptr
expected_table, auto& other_declrs = std::get(sink_decls[0].inputs[0]); ASSERT_OK_AND_ASSIGN(auto output_table, - compute::DeclarationToTable(other_declrs, &exec_context)); + compute::DeclarationToTable(other_declrs, /*use_threads=*/false)); if (!include_columns.empty()) { ASSERT_OK_AND_ASSIGN(output_table, output_table->SelectColumns(include_columns)); } if (sort_options) { - ASSERT_OK_AND_ASSIGN( - auto sort_indices, - SortIndices(output_table, std::move(*sort_options), &exec_context)); - ASSERT_OK_AND_ASSIGN( - auto maybe_table, - compute::Take(output_table, std::move(sort_indices), - compute::TakeOptions::NoBoundsCheck(), &exec_context)); + ASSERT_OK_AND_ASSIGN(auto sort_indices, + SortIndices(output_table, std::move(*sort_options))); + ASSERT_OK_AND_ASSIGN(auto maybe_table, + compute::Take(output_table, std::move(sort_indices), + compute::TakeOptions::NoBoundsCheck())); output_table = maybe_table.table(); } ASSERT_OK_AND_ASSIGN(output_table, output_table->CombineChunks()); - AssertTablesEqual(*expected_table, *output_table); + ASSERT_OK_AND_ASSIGN(auto merged_expected, expected_table->CombineChunks()); + compute::AssertTablesEqualIgnoringOrder(merged_expected, output_table); } TEST(Substrait, SupportedTypes) { @@ -2242,8 +2216,7 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { {"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"}), compute::Declaration({"filter", compute::FilterNodeOptions{filter}, "f"})}); - ASSERT_OK_AND_ASSIGN(auto expected_table, - GetTableFromPlan(declarations, exec_context, dummy_schema)); + ASSERT_OK_AND_ASSIGN(auto expected_table, compute::DeclarationToTable(declarations)); std::shared_ptr sp_ext_id_reg = MakeExtensionIdRegistry(); ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get(); @@ -2290,12 +2263,11 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { EXPECT_TRUE(l_frag->Equals(*r_frag)); } ASSERT_OK_AND_ASSIGN(auto rnd_trp_table, - GetTableFromPlan(roundtripped_filter, exec_context, dummy_schema)); - EXPECT_TRUE(expected_table->Equals(*rnd_trp_table)); + compute::DeclarationToTable(roundtripped_filter)); + compute::AssertTablesEqualIgnoringOrder(expected_table, rnd_trp_table); } TEST(SubstraitRoundTrip, FilterNamedTable) { - compute::ExecContext exec_context; arrow::dataset::internal::Initialize(); const std::vector table_names{"table", "1"}; @@ -2341,7 +2313,7 @@ TEST(SubstraitRoundTrip, FilterNamedTable) { [2, 2, 60] ])"}); - CheckRoundTripResult(std::move(expected_table), exec_context, serialized_plan, + CheckRoundTripResult(std::move(expected_table), serialized_plan, /*include_columns=*/{}, conversion_options); } @@ -2456,12 +2428,11 @@ TEST(SubstraitRoundTrip, ProjectRel) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, ProjectRelOnFunctionWithEmit) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -2575,12 +2546,11 @@ TEST(SubstraitRoundTrip, ProjectRelOnFunctionWithEmit) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, ReadRelWithEmit) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -2634,12 +2604,11 @@ TEST(SubstraitRoundTrip, ReadRelWithEmit) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, FilterRelWithEmit) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32()), field("D", int32())}); @@ -2752,12 +2721,11 @@ TEST(SubstraitRoundTrip, FilterRelWithEmit) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, JoinRel) { - compute::ExecContext exec_context; auto left_schema = schema({field("A", int32()), field("B", int32())}); auto right_schema = schema({field("X", int32()), field("Y", int32())}); @@ -2902,12 +2870,11 @@ TEST(SubstraitRoundTrip, JoinRel) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, JoinRelWithEmit) { - compute::ExecContext exec_context; auto left_schema = schema({field("A", int32()), field("B", int32())}); auto right_schema = schema({field("X", int32()), field("Y", int32())}); @@ -3054,12 +3021,11 @@ TEST(SubstraitRoundTrip, JoinRelWithEmit) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, AggregateRel) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -3163,12 +3129,11 @@ TEST(SubstraitRoundTrip, AggregateRel) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(SubstraitRoundTrip, AggregateRelEmit) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -3278,7 +3243,7 @@ TEST(SubstraitRoundTrip, AggregateRelEmit) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } @@ -3382,12 +3347,11 @@ TEST(Substrait, IsthmusPlan) { /*ignore_unknown_fields=*/false)); auto expected_table = TableFromJSON(test_schema, {"[[2], [3], [6]]"}); - CheckRoundTripResult(std::move(expected_table), *compute::default_exec_context(), buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(Substrait, ProjectWithMultiFieldExpressions) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -3536,12 +3500,11 @@ TEST(Substrait, ProjectWithMultiFieldExpressions) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(Substrait, NestedProjectWithMultiFieldExpressions) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32())}); // creating a dummy dataset using a dummy table @@ -3623,12 +3586,11 @@ TEST(Substrait, NestedProjectWithMultiFieldExpressions) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(Substrait, NestedEmitProjectWithMultiFieldExpressions) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32())}); // creating a dummy dataset using a dummy table @@ -3711,7 +3673,7 @@ TEST(Substrait, NestedEmitProjectWithMultiFieldExpressions) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } @@ -3719,7 +3681,6 @@ TEST(Substrait, ReadRelWithGlobFiles) { #ifdef _WIN32 GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows"; #endif - compute::ExecContext exec_context; arrow::dataset::internal::Initialize(); auto dummy_schema = @@ -3808,12 +3769,11 @@ TEST(Substrait, ReadRelWithGlobFiles) { // To avoid unnecessar metadata columns being included in the final result std::vector include_columns = {0, 1, 2}; compute::SortOptions options({compute::SortKey("A", compute::SortOrder::Ascending)}); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, - std::move(include_columns), /*conversion_options=*/{}, &options); + CheckRoundTripResult(std::move(expected_table), buf, std::move(include_columns), + /*conversion_options=*/{}, &options); } TEST(Substrait, RootRelationOutputNames) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -3917,12 +3877,11 @@ TEST(Substrait, RootRelationOutputNames) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, + CheckRoundTripResult(std::move(expected_table), buf, /*include_columns=*/{}, conversion_options); } TEST(Substrait, SetRelationBasic) { - compute::ExecContext exec_context; auto dummy_schema = schema({field("A", int32()), field("B", int32()), field("C", int32())}); @@ -4037,8 +3996,8 @@ TEST(Substrait, SetRelationBasic) { compute::SortOptions sort_options( {compute::SortKey("A", compute::SortOrder::Ascending)}); - CheckRoundTripResult(std::move(expected_table), exec_context, buf, {}, - conversion_options, &sort_options); + CheckRoundTripResult(std::move(expected_table), buf, {}, conversion_options, + &sort_options); } } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/util.cc b/cpp/src/arrow/engine/substrait/util.cc index 87a7a8e364f..22091d96998 100644 --- a/cpp/src/arrow/engine/substrait/util.cc +++ b/cpp/src/arrow/engine/substrait/util.cc @@ -140,7 +140,7 @@ Result> ExecuteSerializedPlan( const ConversionOptions& conversion_options) { compute::ExecContext exec_context(arrow::default_memory_pool(), ::arrow::internal::GetCpuThreadPool(), func_registry); - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(&exec_context)); + ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make(exec_context)); SubstraitExecutor executor(std::move(plan), exec_context, conversion_options); RETURN_NOT_OK(executor.Init(substrait_buffer, registry)); ARROW_ASSIGN_OR_RAISE(auto sink_reader, executor.Execute()); diff --git a/cpp/src/arrow/util/async_util.cc b/cpp/src/arrow/util/async_util.cc index d868ec78596..ebf264de060 100644 --- a/cpp/src/arrow/util/async_util.cc +++ b/cpp/src/arrow/util/async_util.cc @@ -20,6 +20,7 @@ #include "arrow/util/future.h" #include "arrow/util/logging.h" +#include #include #include #include @@ -121,8 +122,11 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { public: using Task = AsyncTaskScheduler::Task; - explicit AsyncTaskSchedulerImpl(StopToken stop_token) - : AsyncTaskScheduler(), stop_token_(std::move(stop_token)) {} + explicit AsyncTaskSchedulerImpl(StopToken stop_token, + FnOnce abort_callback) + : AsyncTaskScheduler(), + stop_token_(std::move(stop_token)), + abort_callback_(std::move(abort_callback)) {} ~AsyncTaskSchedulerImpl() { DCHECK_EQ(running_tasks_, 0) << " scheduler destroyed while tasks still running"; @@ -188,8 +192,20 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { void AbortUnlocked(const Status& st, std::unique_lock&& lk) { DCHECK(!st.ok()); + bool aborted = false; if (!IsAborted()) { maybe_error_ = st; + // Add one more "task" to represent running the abort callback. This + // will prevent any other task finishing and marking the scheduler finished + // while we are running the abort callback. + running_tasks_++; + aborted = true; + } + if (aborted) { + lk.unlock(); + std::move(abort_callback_)(st); + lk.lock(); + running_tasks_--; } MaybeEndUnlocked(std::move(lk)); } @@ -212,6 +228,9 @@ class AsyncTaskSchedulerImpl : public AsyncTaskScheduler { Status maybe_error_; std::mutex mutex_; StopToken stop_token_; + FnOnce abort_callback_; + bool abort_callback_pending_ = false; + std::condition_variable abort_callback_cv_; // Allows AsyncTaskScheduler::Make to call OnTaskFinished friend AsyncTaskScheduler; @@ -371,8 +390,10 @@ class AsyncTaskGroupImpl : public AsyncTaskGroup { } // namespace Future<> AsyncTaskScheduler::Make(FnOnce initial_task, + FnOnce abort_callback, StopToken stop_token) { - auto scheduler = std::make_unique(std::move(stop_token)); + auto scheduler = std::make_unique(std::move(stop_token), + std::move(abort_callback)); Status initial_task_st = std::move(initial_task)(scheduler.get()); scheduler->OnTaskFinished(std::move(initial_task_st)); // Keep scheduler alive until finished diff --git a/cpp/src/arrow/util/async_util.h b/cpp/src/arrow/util/async_util.h index 1ccf3240f3c..de7aa4addd7 100644 --- a/cpp/src/arrow/util/async_util.h +++ b/cpp/src/arrow/util/async_util.h @@ -147,14 +147,22 @@ class ARROW_EXPORT AsyncTaskScheduler { /// /// \param initial_task The initial task which is responsible for adding /// the first subtasks to the scheduler. + /// \param abort_callback A callback that will be triggered immediately after a task + /// fails while other tasks may still be running. Nothing needs to be done here, + /// when a task fails the scheduler will stop accepting new tasks and eventually + /// return the error. However, this callback can be used to more quickly end + /// long running tasks that have already been submitted. Defaults to doing + /// nothing. /// \param stop_token An optional stop token that will allow cancellation of the /// scheduler. This will be checked before each task is submitted and, in the /// event of a cancellation, the scheduler will enter an aborted state. This is /// a graceful cancellation and submitted tasks will still complete. /// \return A future that will be completed when the initial task and all subtasks have /// finished. - static Future<> Make(FnOnce initial_task, - StopToken stop_token = StopToken::Unstoppable()); + static Future<> Make( + FnOnce initial_task, + FnOnce abort_callback = [](const Status&) {}, + StopToken stop_token = StopToken::Unstoppable()); }; class ARROW_EXPORT ThrottledAsyncTaskScheduler : public AsyncTaskScheduler { diff --git a/cpp/src/arrow/util/async_util_test.cc b/cpp/src/arrow/util/async_util_test.cc index 9b2e6ccc551..4fb17e4ea07 100644 --- a/cpp/src/arrow/util/async_util_test.cc +++ b/cpp/src/arrow/util/async_util_test.cc @@ -84,7 +84,7 @@ TEST(AsyncTaskScheduler, CancelWaitsForTasksToFinish) { scheduler->AddSimpleTask([&] { return task; }); return Status::OK(); }, - stop_source.token()); + /*abort_callback=*/[](const Status&) {}, stop_source.token()); stop_source.RequestStop(); AssertNotFinished(finished); task.MarkFinished(); @@ -108,7 +108,7 @@ TEST(AsyncTaskScheduler, CancelPurgesQueuedTasks) { }); return Status::OK(); }, - stop_source.token()); + /*abort_callback=*/[](const Status&) {}, stop_source.token()); stop_source.RequestStop(); task.MarkFinished(); ASSERT_FINISHES_AND_RAISES(Cancelled, finished); @@ -129,12 +129,29 @@ TEST(AsyncTaskScheduler, CancelPreventsAdditionalTasks) { }); return Status::OK(); }, - stop_source.token()); + /*abort_callback=*/[](const Status&) {}, stop_source.token()); task.MarkFinished(); ASSERT_FINISHES_AND_RAISES(Cancelled, finished); ASSERT_FALSE(second_task_submitted); } +TEST(AsyncTaskScheduler, AbortCallback) { + // `task` simulates a long running task that will not end for a while. The abort + // callback ends the task early. + Future<> task = Future<>::Make(); + Future<> finished = AsyncTaskScheduler::Make( + [&](AsyncTaskScheduler* scheduler) { + scheduler->AddSimpleTask([&] { return task; }); + scheduler->AddSimpleTask([] { return Status::Invalid("XYZ"); }); + return Status::OK(); + }, + [&](const Status& st) { + ASSERT_TRUE(st.IsInvalid()); + task.MarkFinished(); + }); + ASSERT_FINISHES_AND_RAISES(Invalid, finished); +} + TEST(AsyncTaskScheduler, TaskStaysAliveUntilFinished) { bool my_task_destroyed = false; Future<> task = Future<>::Make(); diff --git a/cpp/src/arrow/util/thread_pool.cc b/cpp/src/arrow/util/thread_pool.cc index 9f233a869b3..daffe8f077a 100644 --- a/cpp/src/arrow/util/thread_pool.cc +++ b/cpp/src/arrow/util/thread_pool.cc @@ -57,6 +57,7 @@ struct SerialExecutor::State { std::deque task_queue; std::mutex mutex; std::condition_variable wait_for_tasks; + std::thread::id current_thread; bool paused{false}; bool finished{false}; }; @@ -142,11 +143,16 @@ void SerialExecutor::Unpause() { } } +bool SerialExecutor::OwnsThisThread() { + std::lock_guard lk(state_->mutex); + return std::this_thread::get_id() == state_->current_thread; +} + void SerialExecutor::RunLoop() { // This is called from the SerialExecutor's main thread, so the // state is guaranteed to be kept alive. std::unique_lock lk(state_->mutex); - + state_->current_thread = std::this_thread::get_id(); // If paused we break out immediately. If finished we only break out // when all work is done. while (!state_->paused && !(state_->finished && state_->task_queue.empty())) { @@ -175,6 +181,7 @@ void SerialExecutor::RunLoop() { return state_->paused || state_->finished || !state_->task_queue.empty(); }); } + state_->current_thread = {}; } struct ThreadPool::State { diff --git a/cpp/src/arrow/util/thread_pool.h b/cpp/src/arrow/util/thread_pool.h index 9b3ec54d00d..4e0fd84068c 100644 --- a/cpp/src/arrow/util/thread_pool.h +++ b/cpp/src/arrow/util/thread_pool.h @@ -272,6 +272,7 @@ class ARROW_EXPORT SerialExecutor : public Executor { ~SerialExecutor() override; int GetCapacity() override { return 1; }; + bool OwnsThisThread() override; Status SpawnReal(TaskHints hints, FnOnce task, StopToken, StopCallback&&) override; diff --git a/python/pyarrow/_exec_plan.pyx b/python/pyarrow/_exec_plan.pyx index 85072b21f95..c01e6cba6a1 100644 --- a/python/pyarrow/_exec_plan.pyx +++ b/python/pyarrow/_exec_plan.pyx @@ -86,6 +86,9 @@ cdef execplan(inputs, output_type, vector[CDeclaration] plan, c_bool use_threads else: c_executor = NULL + # TODO(weston): This is deprecated. Once ordering is better supported + # in the exec plan we can remove all references to ExecPlan and use the + # DeclarationToXyz methods c_exec_context = make_shared[CExecContext]( c_default_memory_pool(), c_executor) c_exec_plan = GetResultValue(CExecPlan.Make(c_exec_context.get())) diff --git a/r/src/compute-exec.cpp b/r/src/compute-exec.cpp index 080805d75f0..cb8ebd588b6 100644 --- a/r/src/compute-exec.cpp +++ b/r/src/compute-exec.cpp @@ -41,8 +41,14 @@ std::shared_ptr strings_to_kvm(cpp11::strings metadata) std::shared_ptr ExecPlan_create(bool use_threads) { static compute::ExecContext threaded_context{gc_memory_pool(), arrow::internal::GetCpuThreadPool()}; + // TODO(weston) using gc_context() in this way is deprecated. Once ordering has + // been added we can probably entirely remove all reference to ExecPlan from R + // in favor of DeclarationToXyz +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" auto plan = ValueOrStop( compute::ExecPlan::Make(use_threads ? &threaded_context : gc_context())); +#pragma GCC diagnostic pop return plan; }