From 5cd7c210267db13fc8f6c7295e21bb8c4c49033f Mon Sep 17 00:00:00 2001 From: Davide Pasetto Date: Mon, 20 Mar 2023 10:59:04 -0400 Subject: [PATCH 1/8] moved compute/exec in new arrow-acero library --- c_glib/arrow-glib/compute.cpp | 62 ++--- c_glib/arrow-glib/compute.hpp | 12 +- c_glib/arrow-glib/meson.build | 1 + c_glib/meson.build | 5 + ci/appveyor-cpp-build.bat | 2 + ci/conan/all/conanfile.py | 19 +- ci/docker/alpine-linux-3.16-cpp.dockerfile | 3 +- ci/docker/conda-cpp.dockerfile | 3 +- ci/docker/conda-integration.dockerfile | 3 +- ci/docker/conda-python-hdfs.dockerfile | 1 + ci/docker/conda-python-spark.dockerfile | 1 + ci/docker/conda-python-substrait.dockerfile | 3 +- ci/docker/conda-python.dockerfile | 3 +- ci/docker/debian-10-cpp.dockerfile | 1 + ci/docker/debian-11-cpp.dockerfile | 1 + ci/docker/fedora-35-cpp.dockerfile | 1 + ci/docker/linux-apt-docs.dockerfile | 3 +- ci/docker/linux-apt-jni.dockerfile | 3 +- ci/docker/linux-apt-python-3.dockerfile | 3 +- ci/docker/linux-apt-r.dockerfile | 1 + ci/docker/linux-dnf-python-3.dockerfile | 3 +- ci/docker/ubuntu-20.04-cpp-minimal.dockerfile | 3 +- ci/docker/ubuntu-20.04-cpp.dockerfile | 1 + ci/docker/ubuntu-22.04-cpp-minimal.dockerfile | 3 +- ci/docker/ubuntu-22.04-cpp.dockerfile | 1 + ci/scripts/PKGBUILD | 1 + ci/scripts/cpp_build.sh | 1 + ci/scripts/java_jni_macos_build.sh | 3 + ci/scripts/java_jni_manylinux_build.sh | 3 + ci/scripts/java_jni_windows_build.sh | 3 + ci/scripts/python_build.sh | 1 + ci/scripts/python_sdist_test.sh | 1 + ci/scripts/python_test.sh | 2 + ci/scripts/python_wheel_macos_build.sh | 3 + ci/scripts/python_wheel_manylinux_build.sh | 3 + ci/scripts/python_wheel_unix_test.sh | 1 + ci/scripts/python_wheel_windows_build.bat | 3 + ci/scripts/python_wheel_windows_test.bat | 1 + cpp/CMakePresets.json | 1 + cpp/cmake_modules/DefineOptions.cmake | 11 +- cpp/examples/arrow/CMakeLists.txt | 10 +- ...r_example.cc => acero_register_example.cc} | 27 +- .../arrow/engine_substrait_consumption.cc | 19 +- .../execution_plan_documentation_examples.cc | 139 +++++----- cpp/examples/arrow/join_example.cc | 22 +- cpp/src/arrow/CMakeLists.txt | 35 +-- cpp/src/arrow/acero/ArrowAceroConfig.cmake.in | 39 +++ cpp/src/arrow/acero/CMakeLists.txt | 250 ++++++++++++++++++ .../exec => acero}/accumulation_queue.cc | 4 +- .../exec => acero}/accumulation_queue.h | 3 + .../kernels => acero}/aggregate_benchmark.cc | 16 +- .../{compute/exec => acero}/aggregate_node.cc | 30 ++- cpp/src/arrow/acero/arrow-acero.pc.in | 27 ++ .../exec => acero}/asof_join_benchmark.cc | 10 +- .../{compute/exec => acero}/asof_join_node.cc | 27 +- .../{compute/exec => acero}/asof_join_node.h | 10 +- .../exec => acero}/asof_join_node_test.cc | 36 +-- .../{compute/exec => acero}/benchmark_util.cc | 59 +++-- .../{compute/exec => acero}/benchmark_util.h | 16 +- .../{compute/exec => acero}/bloom_filter.cc | 12 +- .../{compute/exec => acero}/bloom_filter.h | 23 +- .../exec => acero}/bloom_filter_avx2.cc | 6 +- .../exec => acero}/bloom_filter_test.cc | 16 +- .../exec => acero}/doc/img/key_map_1.jpg | Bin .../exec => acero}/doc/img/key_map_10.jpg | Bin .../exec => acero}/doc/img/key_map_11.jpg | Bin .../exec => acero}/doc/img/key_map_2.jpg | Bin .../exec => acero}/doc/img/key_map_3.jpg | Bin .../exec => acero}/doc/img/key_map_4.jpg | Bin .../exec => acero}/doc/img/key_map_5.jpg | Bin .../exec => acero}/doc/img/key_map_6.jpg | Bin .../exec => acero}/doc/img/key_map_7.jpg | Bin .../exec => acero}/doc/img/key_map_8.jpg | Bin .../exec => acero}/doc/img/key_map_9.jpg | Bin .../{compute/exec => acero}/doc/key_map.md | 0 .../{compute/exec => acero}/exec_plan.cc | 40 +-- .../arrow/{compute/exec => acero}/exec_plan.h | 88 +++--- .../exec => acero}/expression_benchmark.cc | 13 +- .../{compute/exec => acero}/fetch_node.cc | 20 +- .../exec => acero}/fetch_node_test.cc | 10 +- .../exec => acero}/filter_benchmark.cc | 18 +- .../{compute/exec => acero}/filter_node.cc | 18 +- .../{compute/exec => acero}/forest_internal.h | 4 +- .../arrow/{compute/exec => acero}/groupby.cc | 6 +- .../arrow/{compute/exec => acero}/groupby.h | 14 +- .../{compute/exec => acero}/groupby_test.cc | 6 +- .../kernels => acero}/hash_aggregate_test.cc | 37 ++- .../{compute/exec => acero}/hash_join.cc | 15 +- .../arrow/{compute/exec => acero}/hash_join.h | 20 +- .../exec => acero}/hash_join_benchmark.cc | 14 +- .../{compute/exec => acero}/hash_join_dict.cc | 21 +- .../{compute/exec => acero}/hash_join_dict.h | 13 +- .../exec => acero}/hash_join_graphs.py | 0 .../{compute/exec => acero}/hash_join_node.cc | 43 +-- .../{compute/exec => acero}/hash_join_node.h | 13 +- .../exec => acero}/hash_join_node_test.cc | 22 +- .../arrow/{compute/exec => acero}/map_node.cc | 6 +- .../arrow/{compute/exec => acero}/map_node.h | 12 +- .../arrow/{compute/exec => acero}/options.cc | 8 +- .../arrow/{compute/exec => acero}/options.h | 70 ++--- .../{compute/exec => acero}/order_by_impl.cc | 11 +- .../{compute/exec => acero}/order_by_impl.h | 9 +- .../{compute/exec => acero}/order_by_node.cc | 18 +- .../exec => acero}/order_by_node_test.cc | 13 +- .../{compute/exec => acero}/partition_util.cc | 6 +- .../{compute/exec => acero}/partition_util.h | 6 +- cpp/src/arrow/acero/pch.h | 23 ++ .../exec => acero}/pivot_longer_node.cc | 14 +- .../exec => acero}/pivot_longer_node_test.cc | 11 +- .../{compute/exec => acero}/plan_test.cc | 31 ++- .../exec => acero}/project_benchmark.cc | 22 +- .../{compute/exec => acero}/project_node.cc | 20 +- .../{compute/exec => acero}/query_context.cc | 8 +- .../{compute/exec => acero}/query_context.h | 24 +- .../{compute/exec => acero}/schema_util.h | 4 +- .../{compute/exec => acero}/sink_node.cc | 30 +-- .../{compute/exec => acero}/source_node.cc | 25 +- .../exec => acero}/subtree_internal.h | 7 +- .../{compute/exec => acero}/subtree_test.cc | 11 +- .../{compute/exec => acero}/swiss_join.cc | 146 +++++----- .../exec => acero}/swiss_join_avx2.cc | 6 +- .../exec => acero}/swiss_join_internal.h | 40 +-- .../{compute/exec => acero}/task_util.cc | 6 +- .../arrow/{compute/exec => acero}/task_util.h | 7 +- .../{compute/exec => acero}/task_util_test.cc | 8 +- .../{compute/exec => acero}/test_nodes.cc | 14 +- .../{compute/exec => acero}/test_nodes.h | 8 +- .../exec => acero}/test_nodes_test.cc | 8 +- .../test_util_internal.cc} | 74 +++++- .../test_util_internal.h} | 41 +-- .../{compute/exec => acero}/tpch_benchmark.cc | 21 +- .../{compute/exec => acero}/tpch_node.cc | 18 +- .../arrow/{compute/exec => acero}/tpch_node.h | 13 +- .../{compute/exec => acero}/tpch_node_test.cc | 18 +- .../arrow/{compute/exec => acero}/type_fwd.h | 4 +- .../{compute/exec => acero}/union_node.cc | 14 +- .../exec => acero}/union_node_test.cc | 8 +- cpp/src/arrow/{compute/exec => acero}/util.cc | 8 +- cpp/src/arrow/{compute/exec => acero}/util.h | 27 +- .../{compute/exec => acero}/util_avx2.cc | 2 +- .../{compute/exec => acero}/util_test.cc | 8 +- cpp/src/arrow/acero/visibility.h | 50 ++++ cpp/src/arrow/compute/CMakeLists.txt | 1 - cpp/src/arrow/compute/exec/CMakeLists.txt | 70 ----- cpp/src/arrow/compute/kernel.h | 10 +- cpp/src/arrow/compute/kernels/CMakeLists.txt | 15 +- cpp/src/arrow/compute/key_hash.h | 14 +- cpp/src/arrow/compute/key_hash_test.cc | 4 +- cpp/src/arrow/compute/key_map.h | 45 ++-- cpp/src/arrow/compute/row/compare_internal.h | 2 +- cpp/src/arrow/compute/row/encode_internal.h | 2 +- cpp/src/arrow/compute/util.h | 4 +- .../arrow/dataset/ArrowDatasetConfig.cmake.in | 1 + cpp/src/arrow/dataset/CMakeLists.txt | 12 +- cpp/src/arrow/dataset/dataset.cc | 2 +- cpp/src/arrow/dataset/file_base.cc | 62 ++--- cpp/src/arrow/dataset/file_base.h | 5 +- cpp/src/arrow/dataset/file_test.cc | 1 - cpp/src/arrow/dataset/plan.cc | 4 +- cpp/src/arrow/dataset/scan_node.cc | 33 +-- cpp/src/arrow/dataset/scanner.cc | 68 ++--- cpp/src/arrow/dataset/scanner.h | 16 +- cpp/src/arrow/dataset/scanner_benchmark.cc | 40 +-- cpp/src/arrow/dataset/scanner_test.cc | 3 +- cpp/src/arrow/dataset/test_util_internal.cc | 218 +++++++++++++++ cpp/src/arrow/dataset/test_util_internal.h | 83 +++++- .../engine/ArrowSubstraitConfig.cmake.in | 1 + cpp/src/arrow/engine/CMakeLists.txt | 4 +- cpp/src/arrow/engine/arrow-substrait.pc.in | 2 +- .../arrow/engine/substrait/function_test.cc | 44 ++- cpp/src/arrow/engine/substrait/options.cc | 22 +- cpp/src/arrow/engine/substrait/options.h | 10 +- .../arrow/engine/substrait/plan_internal.cc | 2 +- .../arrow/engine/substrait/plan_internal.h | 2 +- cpp/src/arrow/engine/substrait/relation.h | 4 +- .../engine/substrait/relation_internal.cc | 93 ++++--- .../engine/substrait/relation_internal.h | 2 +- cpp/src/arrow/engine/substrait/serde.cc | 61 +++-- cpp/src/arrow/engine/substrait/serde.h | 18 +- cpp/src/arrow/engine/substrait/serde_test.cc | 182 +++++++------ cpp/src/arrow/engine/substrait/test_util.cc | 88 ++++++ cpp/src/arrow/engine/substrait/test_util.h | 45 ++++ cpp/src/arrow/engine/substrait/util.cc | 10 +- .../arrow/flight/sql/example/acero_server.cc | 10 +- python/CMakeLists.txt | 36 ++- python/pyarrow/_acero.pxd | 1 + python/pyarrow/_acero.pyx | 34 ++- python/pyarrow/_compute.pyx | 32 --- python/pyarrow/compute.py | 1 - python/pyarrow/includes/libarrow.pxd | 144 +--------- python/pyarrow/includes/libarrow_acero.pxd | 160 +++++++++++ python/pyarrow/includes/libarrow_dataset.pxd | 1 + .../pyarrow/includes/libarrow_substrait.pxd | 1 + python/pyarrow/table.pxi | 2 +- python/pyarrow/tests/test_acero.py | 29 +- python/setup.py | 14 + r/configure | 6 + r/src/arrowExports.cpp | 70 ++--- r/src/arrow_types.h | 13 +- r/src/compute-exec.cpp | 118 ++++----- 200 files changed, 2739 insertions(+), 1627 deletions(-) rename cpp/examples/arrow/{compute_register_example.cc => acero_register_example.cc} (90%) create mode 100644 cpp/src/arrow/acero/ArrowAceroConfig.cmake.in create mode 100644 cpp/src/arrow/acero/CMakeLists.txt rename cpp/src/arrow/{compute/exec => acero}/accumulation_queue.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/accumulation_queue.h (99%) rename cpp/src/arrow/{compute/kernels => acero}/aggregate_benchmark.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/aggregate_node.cc (98%) create mode 100644 cpp/src/arrow/acero/arrow-acero.pc.in rename cpp/src/arrow/{compute/exec => acero}/asof_join_benchmark.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/asof_join_node.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/asof_join_node.h (89%) rename cpp/src/arrow/{compute/exec => acero}/asof_join_node_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/benchmark_util.cc (72%) rename cpp/src/arrow/{compute/exec => acero}/benchmark_util.h (76%) rename cpp/src/arrow/{compute/exec => acero}/bloom_filter.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/bloom_filter.h (95%) rename cpp/src/arrow/{compute/exec => acero}/bloom_filter_avx2.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/bloom_filter_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_1.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_10.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_11.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_2.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_3.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_4.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_5.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_6.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_7.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_8.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/img/key_map_9.jpg (100%) rename cpp/src/arrow/{compute/exec => acero}/doc/key_map.md (100%) rename cpp/src/arrow/{compute/exec => acero}/exec_plan.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/exec_plan.h (91%) rename cpp/src/arrow/{compute/exec => acero}/expression_benchmark.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/fetch_node.cc (95%) rename cpp/src/arrow/{compute/exec => acero}/fetch_node_test.cc (94%) rename cpp/src/arrow/{compute/exec => acero}/filter_benchmark.cc (94%) rename cpp/src/arrow/{compute/exec => acero}/filter_node.cc (94%) rename cpp/src/arrow/{compute/exec => acero}/forest_internal.h (98%) rename cpp/src/arrow/{compute/exec => acero}/groupby.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/groupby.h (92%) rename cpp/src/arrow/{compute/exec => acero}/groupby_test.cc (98%) rename cpp/src/arrow/{compute/kernels => acero}/hash_aggregate_test.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/hash_join.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/hash_join.h (87%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_benchmark.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_dict.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_dict.h (98%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_graphs.py (100%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_node.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_node.h (95%) rename cpp/src/arrow/{compute/exec => acero}/hash_join_node_test.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/map_node.cc (97%) rename cpp/src/arrow/{compute/exec => acero}/map_node.h (91%) rename cpp/src/arrow/{compute/exec => acero}/options.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/options.h (93%) rename cpp/src/arrow/{compute/exec => acero}/order_by_impl.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/order_by_impl.h (93%) rename cpp/src/arrow/{compute/exec => acero}/order_by_node.cc (95%) rename cpp/src/arrow/{compute/exec => acero}/order_by_node_test.cc (95%) rename cpp/src/arrow/{compute/exec => acero}/partition_util.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/partition_util.h (98%) create mode 100644 cpp/src/arrow/acero/pch.h rename cpp/src/arrow/{compute/exec => acero}/pivot_longer_node.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/pivot_longer_node_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/plan_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/project_benchmark.cc (89%) rename cpp/src/arrow/{compute/exec => acero}/project_node.cc (92%) rename cpp/src/arrow/{compute/exec => acero}/query_context.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/query_context.h (91%) rename cpp/src/arrow/{compute/exec => acero}/schema_util.h (99%) rename cpp/src/arrow/{compute/exec => acero}/sink_node.cc (97%) rename cpp/src/arrow/{compute/exec => acero}/source_node.cc (97%) rename cpp/src/arrow/{compute/exec => acero}/subtree_internal.h (98%) rename cpp/src/arrow/{compute/exec => acero}/subtree_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/swiss_join.cc (95%) rename cpp/src/arrow/{compute/exec => acero}/swiss_join_avx2.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/swiss_join_internal.h (96%) rename cpp/src/arrow/{compute/exec => acero}/task_util.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/task_util.h (96%) rename cpp/src/arrow/{compute/exec => acero}/task_util_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/test_nodes.cc (96%) rename cpp/src/arrow/{compute/exec => acero}/test_nodes.h (94%) rename cpp/src/arrow/{compute/exec => acero}/test_nodes_test.cc (94%) rename cpp/src/arrow/{compute/exec/test_util.cc => acero/test_util_internal.cc} (92%) rename cpp/src/arrow/{compute/exec/test_util.h => acero/test_util_internal.h} (90%) rename cpp/src/arrow/{compute/exec => acero}/tpch_benchmark.cc (93%) rename cpp/src/arrow/{compute/exec => acero}/tpch_node.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/tpch_node.h (92%) rename cpp/src/arrow/{compute/exec => acero}/tpch_node_test.cc (98%) rename cpp/src/arrow/{compute/exec => acero}/type_fwd.h (96%) rename cpp/src/arrow/{compute/exec => acero}/union_node.cc (95%) rename cpp/src/arrow/{compute/exec => acero}/union_node_test.cc (97%) rename cpp/src/arrow/{compute/exec => acero}/util.cc (97%) rename cpp/src/arrow/{compute/exec => acero}/util.h (94%) rename cpp/src/arrow/{compute/exec => acero}/util_avx2.cc (99%) rename cpp/src/arrow/{compute/exec => acero}/util_test.cc (98%) create mode 100644 cpp/src/arrow/acero/visibility.h delete mode 100644 cpp/src/arrow/compute/exec/CMakeLists.txt create mode 100644 cpp/src/arrow/dataset/test_util_internal.cc create mode 100644 cpp/src/arrow/engine/substrait/test_util.cc create mode 100644 cpp/src/arrow/engine/substrait/test_util.h create mode 100644 python/pyarrow/includes/libarrow_acero.pxd diff --git a/c_glib/arrow-glib/compute.cpp b/c_glib/arrow-glib/compute.cpp index e5cefe2e01c..1a0c7284f6e 100644 --- a/c_glib/arrow-glib/compute.cpp +++ b/c_glib/arrow-glib/compute.cpp @@ -33,8 +33,8 @@ #include #include -#include -#include +#include +#include template typename ArrowType::c_type @@ -817,7 +817,7 @@ garrow_function_to_string(GArrowFunction *function) typedef struct GArrowExecuteNodeOptionsPrivate_ { - arrow::compute::ExecNodeOptions *options; + arrow::acero::ExecNodeOptions *options; } GArrowExecuteNodeOptionsPrivate; enum { @@ -852,7 +852,7 @@ garrow_execute_node_options_set_property(GObject *object, switch (prop_id) { case PROP_FUNCTION: priv->options = - static_cast(g_value_get_pointer(value)); + static_cast(g_value_get_pointer(value)); break; default: G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec); @@ -877,7 +877,7 @@ garrow_execute_node_options_class_init(GArrowExecuteNodeOptionsClass *klass) GParamSpec *spec; spec = g_param_spec_pointer("options", "Options", - "The raw arrow::compute::ExecNodeOptions *", + "The raw arrow::acero::ExecNodeOptions *", static_cast(G_PARAM_WRITABLE | G_PARAM_CONSTRUCT_ONLY)); g_object_class_install_property(gobject_class, @@ -988,7 +988,7 @@ garrow_source_node_options_new_record_batch_reader( GArrowRecordBatchReader *reader) { auto arrow_reader = garrow_record_batch_reader_get_raw(reader); - auto arrow_options = new arrow::compute::SourceNodeOptions( + auto arrow_options = new arrow::acero::SourceNodeOptions( arrow_reader->schema(), [arrow_reader]() { using ExecBatch = arrow::compute::ExecBatch; @@ -1029,7 +1029,7 @@ garrow_source_node_options_new_record_batch(GArrowRecordBatch *record_batch) auto state = std::make_shared(); state->record_batch = garrow_record_batch_get_raw(record_batch); state->generated = false; - auto arrow_options = new arrow::compute::SourceNodeOptions( + auto arrow_options = new arrow::acero::SourceNodeOptions( state->record_batch->schema(), [state]() { using ExecBatch = arrow::compute::ExecBatch; @@ -1095,7 +1095,7 @@ garrow_filter_node_options_new(GArrowExpression *expression) { auto arrow_expression = garrow_expression_get_raw(expression); auto arrow_options = - new arrow::compute::FilterNodeOptions(*arrow_expression); + new arrow::acero::FilterNodeOptions(*arrow_expression); auto options = g_object_new(GARROW_TYPE_FILTER_NODE_OPTIONS, "options", arrow_options, NULL); @@ -1150,7 +1150,7 @@ garrow_project_node_options_new(GList *expressions, } } auto arrow_options = - new arrow::compute::ProjectNodeOptions(arrow_expressions, arrow_names); + new arrow::acero::ProjectNodeOptions(arrow_expressions, arrow_names); auto options = g_object_new(GARROW_TYPE_PROJECT_NODE_OPTIONS, "options", arrow_options, NULL); @@ -1430,7 +1430,7 @@ garrow_aggregate_node_options_new(GList *aggregations, } } auto arrow_options = - new arrow::compute::AggregateNodeOptions(std::move(arrow_aggregates), + new arrow::acero::AggregateNodeOptions(std::move(arrow_aggregates), std::move(arrow_keys)); auto options = g_object_new(GARROW_TYPE_AGGREGATE_NODE_OPTIONS, "options", arrow_options, @@ -1500,7 +1500,7 @@ garrow_sink_node_options_new(void) { auto options = g_object_new(GARROW_TYPE_SINK_NODE_OPTIONS, NULL); auto priv = GARROW_SINK_NODE_OPTIONS_GET_PRIVATE(options); - auto arrow_options = new arrow::compute::SinkNodeOptions(&(priv->generator)); + auto arrow_options = new arrow::acero::SinkNodeOptions(&(priv->generator)); auto execute_node_options_priv = GARROW_EXECUTE_NODE_OPTIONS_GET_PRIVATE(options); execute_node_options_priv->options = arrow_options; return GARROW_SINK_NODE_OPTIONS(options); @@ -1523,7 +1523,7 @@ garrow_sink_node_options_get_reader(GArrowSinkNodeOptions *options, auto priv = GARROW_SINK_NODE_OPTIONS_GET_PRIVATE(options); if (!priv->reader) { auto arrow_reader = - arrow::compute::MakeGeneratorReader(arrow_schema, + arrow::acero::MakeGeneratorReader(arrow_schema, std::move(priv->generator), arrow::default_memory_pool()); priv->reader = garrow_record_batch_reader_new_raw(&arrow_reader); @@ -1570,7 +1570,7 @@ garrow_hash_join_node_options_new(GArrowJoinType type, gsize n_right_keys, GError **error) { - auto arrow_type = static_cast(type); + auto arrow_type = static_cast(type); std::vector arrow_left_keys; for (gsize i = 0; i < n_left_keys; ++i) { if (!garrow_field_refs_add(arrow_left_keys, @@ -1590,7 +1590,7 @@ garrow_hash_join_node_options_new(GArrowJoinType type, } } auto arrow_options = - new arrow::compute::HashJoinNodeOptions(arrow_type, + new arrow::acero::HashJoinNodeOptions(arrow_type, std::move(arrow_left_keys), std::move(arrow_right_keys)); auto options = g_object_new(GARROW_TYPE_HASH_JOIN_NODE_OPTIONS, @@ -1618,7 +1618,7 @@ garrow_hash_join_node_options_set_left_outputs( GError **error) { auto arrow_options = - static_cast( + static_cast( garrow_execute_node_options_get_raw( GARROW_EXECUTE_NODE_OPTIONS(options))); arrow_options->output_all = false; @@ -1653,7 +1653,7 @@ garrow_hash_join_node_options_set_right_outputs( GError **error) { auto arrow_options = - static_cast( + static_cast( garrow_execute_node_options_get_raw( GARROW_EXECUTE_NODE_OPTIONS(options))); arrow_options->output_all = false; @@ -1671,7 +1671,7 @@ garrow_hash_join_node_options_set_right_outputs( typedef struct GArrowExecuteNodePrivate_ { - arrow::compute::ExecNode *node; + arrow::acero::ExecNode *node; } GArrowExecuteNodePrivate; enum { @@ -1698,7 +1698,7 @@ garrow_execute_node_set_property(GObject *object, switch (prop_id) { case PROP_NODE: priv->node = - static_cast(g_value_get_pointer(value)); + static_cast(g_value_get_pointer(value)); break; default: G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec); @@ -1720,7 +1720,7 @@ garrow_execute_node_class_init(GArrowExecuteNodeClass *klass) GParamSpec *spec; spec = g_param_spec_pointer("node", "Node", - "The raw arrow::compute::ExecNode *", + "The raw arrow::acero::ExecNode *", static_cast(G_PARAM_WRITABLE | G_PARAM_CONSTRUCT_ONLY)); g_object_class_install_property(gobject_class, PROP_NODE, spec); @@ -1759,7 +1759,7 @@ garrow_execute_node_get_output_schema(GArrowExecuteNode *node) typedef struct GArrowExecutePlanPrivate_ { - std::shared_ptr plan; + std::shared_ptr plan; } GArrowExecutePlanPrivate; enum { @@ -1794,7 +1794,7 @@ garrow_execute_plan_set_property(GObject *object, switch (prop_id) { case PROP_PLAN: priv->plan = - *static_cast *>( + *static_cast *>( g_value_get_pointer(value)); break; default: @@ -1807,7 +1807,7 @@ static void garrow_execute_plan_init(GArrowExecutePlan *object) { auto priv = GARROW_EXECUTE_PLAN_GET_PRIVATE(object); - new(&(priv->plan)) std::shared_ptr; + new(&(priv->plan)) std::shared_ptr; } static void @@ -1820,7 +1820,7 @@ garrow_execute_plan_class_init(GArrowExecutePlanClass *klass) GParamSpec *spec; spec = g_param_spec_pointer("plan", "Plan", - "The raw std::shared_ptr", + "The raw std::shared_ptr", static_cast(G_PARAM_WRITABLE | G_PARAM_CONSTRUCT_ONLY)); g_object_class_install_property(gobject_class, PROP_PLAN, spec); @@ -1838,7 +1838,7 @@ garrow_execute_plan_class_init(GArrowExecutePlanClass *klass) GArrowExecutePlan * garrow_execute_plan_new(GError **error) { - auto arrow_plan_result = arrow::compute::ExecPlan::Make(); + auto arrow_plan_result = arrow::acero::ExecPlan::Make(); if (garrow::check(error, arrow_plan_result, "[execute-plan][new]")) { return GARROW_EXECUTE_PLAN(g_object_new(GARROW_TYPE_EXECUTE_PLAN, "plan", &(*arrow_plan_result), @@ -1869,14 +1869,14 @@ garrow_execute_plan_build_node(GArrowExecutePlan *plan, GError **error) { auto arrow_plan = garrow_execute_plan_get_raw(plan); - std::vector arrow_inputs; + std::vector arrow_inputs; for (auto node = inputs; node; node = node->next) { auto arrow_node = garrow_execute_node_get_raw(GARROW_EXECUTE_NODE(node->data)); arrow_inputs.push_back(arrow_node); } auto arrow_options = garrow_execute_node_options_get_raw(options); - auto arrow_node_result = arrow::compute::MakeExecNode(factory_name, + auto arrow_node_result = arrow::acero::MakeExecNode(factory_name, arrow_plan.get(), arrow_inputs, *arrow_options); @@ -5914,7 +5914,7 @@ garrow_function_get_raw(GArrowFunction *function) GArrowExecuteNodeOptions * garrow_execute_node_options_new_raw( - arrow::compute::ExecNodeOptions *arrow_options) + arrow::acero::ExecNodeOptions *arrow_options) { return GARROW_EXECUTE_NODE_OPTIONS( g_object_new(GARROW_TYPE_EXECUTE_NODE_OPTIONS, @@ -5922,7 +5922,7 @@ garrow_execute_node_options_new_raw( NULL)); } -arrow::compute::ExecNodeOptions * +arrow::acero::ExecNodeOptions * garrow_execute_node_options_get_raw(GArrowExecuteNodeOptions *options) { auto priv = GARROW_EXECUTE_NODE_OPTIONS_GET_PRIVATE(options); @@ -5931,14 +5931,14 @@ garrow_execute_node_options_get_raw(GArrowExecuteNodeOptions *options) GArrowExecuteNode * -garrow_execute_node_new_raw(arrow::compute::ExecNode *arrow_node) +garrow_execute_node_new_raw(arrow::acero::ExecNode *arrow_node) { return GARROW_EXECUTE_NODE(g_object_new(GARROW_TYPE_EXECUTE_NODE, "node", arrow_node, NULL)); } -arrow::compute::ExecNode * +arrow::acero::ExecNode * garrow_execute_node_get_raw(GArrowExecuteNode *node) { auto priv = GARROW_EXECUTE_NODE_GET_PRIVATE(node); @@ -5946,7 +5946,7 @@ garrow_execute_node_get_raw(GArrowExecuteNode *node) } -std::shared_ptr +std::shared_ptr garrow_execute_plan_get_raw(GArrowExecutePlan *plan) { auto priv = GARROW_EXECUTE_PLAN_GET_PRIVATE(plan); diff --git a/c_glib/arrow-glib/compute.hpp b/c_glib/arrow-glib/compute.hpp index 08e220a09d2..3297adf104b 100644 --- a/c_glib/arrow-glib/compute.hpp +++ b/c_glib/arrow-glib/compute.hpp @@ -20,7 +20,7 @@ #pragma once #include -#include +#include #include @@ -53,18 +53,18 @@ garrow_function_get_raw(GArrowFunction *function); GArrowExecuteNodeOptions * garrow_execute_node_options_new_raw( - arrow::compute::ExecNodeOptions *arrow_options); -arrow::compute::ExecNodeOptions * + arrow::acero::ExecNodeOptions *arrow_options); +arrow::acero::ExecNodeOptions * garrow_execute_node_options_get_raw(GArrowExecuteNodeOptions *options); GArrowExecuteNode * -garrow_execute_node_new_raw(arrow::compute::ExecNode *arrow_node); -arrow::compute::ExecNode * +garrow_execute_node_new_raw(arrow::acero::ExecNode *arrow_node); +arrow::acero::ExecNode * garrow_execute_node_get_raw(GArrowExecuteNode *node); -std::shared_ptr +std::shared_ptr garrow_execute_plan_get_raw(GArrowExecutePlan *plan); diff --git a/c_glib/arrow-glib/meson.build b/c_glib/arrow-glib/meson.build index c7233d8b180..2b9c3c2472c 100644 --- a/c_glib/arrow-glib/meson.build +++ b/c_glib/arrow-glib/meson.build @@ -241,6 +241,7 @@ if not gio.found() endif dependencies = [ arrow, + arrow_acero, gobject, gio, ] diff --git a/c_glib/meson.build b/c_glib/meson.build index a8b0690336c..e679ce73ee1 100644 --- a/c_glib/meson.build +++ b/c_glib/meson.build @@ -87,6 +87,8 @@ if arrow_cpp_build_lib_dir == '' have_arrow_orc = dependency('arrow-orc', required: false).found() arrow_cuda = dependency('arrow-cuda', required: false) + # we do not support compiling glib without acero engine + arrow_acero = dependency('arrow-acero', required: true) arrow_dataset = dependency('arrow-dataset', required: false) arrow_flight = dependency('arrow-flight', required: false) arrow_flight_sql = dependency('arrow-flight-sql', required: false) @@ -116,6 +118,9 @@ main(void) arrow_cuda = cpp_compiler.find_library('arrow_cuda', dirs: [arrow_cpp_build_lib_dir], required: false) + arrow_acero = cpp_compiler.find_library('arrow_acero', + dirs: [arrow_cpp_build_lib_dir], + required: true) arrow_dataset = cpp_compiler.find_library('arrow_dataset', dirs: [arrow_cpp_build_lib_dir], required: false) diff --git a/ci/appveyor-cpp-build.bat b/ci/appveyor-cpp-build.bat index b34d00e85ab..0da8a7daccd 100644 --- a/ci/appveyor-cpp-build.bat +++ b/ci/appveyor-cpp-build.bat @@ -56,6 +56,7 @@ pushd cpp\build @rem and enable runtime assertions. cmake -G "%GENERATOR%" %CMAKE_ARGS% ^ + -DARROW_ACERO=ON ^ -DARROW_BOOST_USE_SHARED=ON ^ -DARROW_BUILD_EXAMPLES=ON ^ -DARROW_BUILD_STATIC=OFF ^ @@ -109,6 +110,7 @@ pushd python set PYARROW_CMAKE_GENERATOR=%GENERATOR% set PYARROW_CXXFLAGS=%ARROW_CXXFLAGS% set PYARROW_PARALLEL=2 +set PYARROW_WITH_ACERO=ON set PYARROW_WITH_DATASET=ON set PYARROW_WITH_FLIGHT=%ARROW_BUILD_FLIGHT% set PYARROW_WITH_GANDIVA=%ARROW_BUILD_GANDIVA% diff --git a/ci/conan/all/conanfile.py b/ci/conan/all/conanfile.py index cbfbabddf43..89484064f9b 100644 --- a/ci/conan/all/conanfile.py +++ b/ci/conan/all/conanfile.py @@ -51,6 +51,7 @@ class ArrowConan(ConanFile): "plasma": [True, False], "cli": [True, False], "compute": ["auto", True, False], + "acero": ["auto", True, False], "dataset_modules": ["auto", True, False], "deprecated": [True, False], "encryption": [True, False], @@ -96,6 +97,7 @@ class ArrowConan(ConanFile): "plasma": False, "cli": False, "compute": "auto", + "acero": "auto", "dataset_modules": "auto", "deprecated": True, "encryption": False, @@ -191,6 +193,8 @@ def validate(self): del self.options.fPIC if self.options.compute == False and not self._compute(True): raise ConanInvalidConfiguration("compute options is required (or choose auto)") + if self.options.acero == False and not self._acero(True): + raise ConanInvalidConfiguration("acero options is required (or choose auto)") if self.options.parquet == False and self._parquet(True): raise ConanInvalidConfiguration("parquet options is required (or choose auto)") if self.options.dataset_modules == False and self._dataset_modules(True): @@ -234,10 +238,16 @@ def layout(self): def _compute(self, required=False): if required or self.options.compute == "auto": - return bool(self._parquet()) or bool(self._dataset_modules()) or bool(self.options.get_safe("substrait", False)) + return bool(self._parquet()) or bool(self._acero()) else: return bool(self.options.compute) + def _acero(self, required=False): + if required or self.options.acero == "auto": + return bool(self._dataset_modules()) + else: + return bool(self.options.acero) + def _parquet(self, required=False): if required or self.options.parquet == "auto": return bool(self.options.get_safe("substrait", False)) @@ -446,6 +456,7 @@ def generate(self): tc.variables["ARROW_SUBSTRAIT"] = bool(self.options.get_safe("substrait", False)) if Version(self.version) < "12.0.0": tc.variables["ARROW_PLASMA"] = bool(self._plasma()) + tc.variables["ARROW_ACERO"] = self._acero() tc.variables["ARROW_DATASET"] = self._dataset_modules() tc.variables["ARROW_FILESYSTEM"] = bool(self.options.filesystem_layer) tc.variables["PARQUET_REQUIRE_ENCRYPTION"] = bool(self.options.encryption) @@ -553,6 +564,7 @@ def _patch_sources(self): for filename in glob.glob(os.path.join(self.source_folder, "cpp", "cmake_modules", "Find*.cmake")): if os.path.basename(filename) not in [ "FindArrow.cmake", + "FindArrowAcero.cmake", "FindArrowCUDA.cmake", "FindArrowDataset.cmake", "FindArrowFlight.cmake", @@ -624,7 +636,7 @@ def package_info(self): self.cpp_info.components["libarrow_substrait"].names["cmake_find_package"] = "arrow_substrait" self.cpp_info.components["libarrow_substrait"].names["cmake_find_package_multi"] = "arrow_substrait" self.cpp_info.components["libarrow_substrait"].names["pkg_config"] = "arrow_substrait" - self.cpp_info.components["libarrow_substrait"].requires = ["libparquet", "dataset"] + self.cpp_info.components["libarrow_substrait"].requires = ["libparquet", "dataset", "acero"] if self._plasma(): self.cpp_info.components["libplasma"].libs = [self._lib_name("plasma")] @@ -654,6 +666,9 @@ def package_info(self): self.cpp_info.components["libarrow_flight_sql"].names["pkg_config"] = "flight_sql" self.cpp_info.components["libarrow_flight_sql"].requires = ["libarrow", "libarrow_flight"] + if self._acero(): + self.cpp_info.components["acero"].libs = ["arrow_acero"] + if self._dataset_modules(): self.cpp_info.components["dataset"].libs = ["arrow_dataset"] diff --git a/ci/docker/alpine-linux-3.16-cpp.dockerfile b/ci/docker/alpine-linux-3.16-cpp.dockerfile index 29ff5480e37..f269fa548c1 100644 --- a/ci/docker/alpine-linux-3.16-cpp.dockerfile +++ b/ci/docker/alpine-linux-3.16-cpp.dockerfile @@ -73,7 +73,8 @@ RUN /arrow/ci/scripts/install_minio.sh latest /usr/local COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_FLIGHT=ON \ diff --git a/ci/docker/conda-cpp.dockerfile b/ci/docker/conda-cpp.dockerfile index 82d7edd2d58..ee061838611 100644 --- a/ci/docker/conda-cpp.dockerfile +++ b/ci/docker/conda-cpp.dockerfile @@ -46,7 +46,8 @@ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=CONDA \ ARROW_FLIGHT=ON \ diff --git a/ci/docker/conda-integration.dockerfile b/ci/docker/conda-integration.dockerfile index 9a91c2cfcb3..da315776b58 100644 --- a/ci/docker/conda-integration.dockerfile +++ b/ci/docker/conda-integration.dockerfile @@ -56,7 +56,8 @@ ENV DOTNET_ROOT=/opt/dotnet \ PATH=/opt/dotnet:$PATH RUN curl -sSL https://dot.net/v1/dotnet-install.sh | bash /dev/stdin -Channel 7.0 -InstallDir /opt/dotnet -ENV ARROW_BUILD_INTEGRATION=ON \ +ENV ARROW_ACERO=OFF \ + ARROW_BUILD_INTEGRATION=ON \ ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_COMPUTE=OFF \ diff --git a/ci/docker/conda-python-hdfs.dockerfile b/ci/docker/conda-python-hdfs.dockerfile index 07980384a86..c7b01edf8c2 100644 --- a/ci/docker/conda-python-hdfs.dockerfile +++ b/ci/docker/conda-python-hdfs.dockerfile @@ -42,6 +42,7 @@ COPY ci/etc/hdfs-site.xml $HADOOP_HOME/etc/hadoop/ # build cpp with tests ENV CC=gcc \ CXX=g++ \ + ARROW_ACERO=ON \ ARROW_BUILD_TESTS=ON \ ARROW_COMPUTE=ON \ ARROW_CSV=ON \ diff --git a/ci/docker/conda-python-spark.dockerfile b/ci/docker/conda-python-spark.dockerfile index 58e3d5e5d56..221c8409924 100644 --- a/ci/docker/conda-python-spark.dockerfile +++ b/ci/docker/conda-python-spark.dockerfile @@ -42,6 +42,7 @@ RUN /arrow/ci/scripts/install_spark.sh ${spark} /spark # build cpp with tests ENV CC=gcc \ CXX=g++ \ + ARROW_ACERO=ON \ ARROW_BUILD_TESTS=OFF \ ARROW_COMPUTE=ON \ ARROW_CSV=ON \ diff --git a/ci/docker/conda-python-substrait.dockerfile b/ci/docker/conda-python-substrait.dockerfile index 05f57386f4b..33f6957a86f 100644 --- a/ci/docker/conda-python-substrait.dockerfile +++ b/ci/docker/conda-python-substrait.dockerfile @@ -36,7 +36,8 @@ RUN mamba install -q -y \ ARG substrait=latest COPY ci/scripts/install_substrait_consumer.sh /arrow/ci/scripts/ -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_COMPUTE=ON \ ARROW_CSV=ON \ ARROW_DATASET=ON \ diff --git a/ci/docker/conda-python.dockerfile b/ci/docker/conda-python.dockerfile index 85cf5f3a93b..6a6ae73585d 100644 --- a/ci/docker/conda-python.dockerfile +++ b/ci/docker/conda-python.dockerfile @@ -37,7 +37,8 @@ RUN mamba install -q -y \ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts RUN /arrow/ci/scripts/install_gcs_testbench.sh default -ENV ARROW_BUILD_STATIC=OFF \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_BUILD_UTILITIES=OFF \ ARROW_COMPUTE=ON \ diff --git a/ci/docker/debian-10-cpp.dockerfile b/ci/docker/debian-10-cpp.dockerfile index 542860e8468..ca1017ecc3d 100644 --- a/ci/docker/debian-10-cpp.dockerfile +++ b/ci/docker/debian-10-cpp.dockerfile @@ -93,6 +93,7 @@ COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin ENV absl_SOURCE=BUNDLED \ + ARROW_ACERO=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ diff --git a/ci/docker/debian-11-cpp.dockerfile b/ci/docker/debian-11-cpp.dockerfile index f3b24deeac8..e6ac9e6071d 100644 --- a/ci/docker/debian-11-cpp.dockerfile +++ b/ci/docker/debian-11-cpp.dockerfile @@ -95,6 +95,7 @@ COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin ENV absl_SOURCE=BUNDLED \ + ARROW_ACERO=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ diff --git a/ci/docker/fedora-35-cpp.dockerfile b/ci/docker/fedora-35-cpp.dockerfile index aeb7c5b7951..668e35b4435 100644 --- a/ci/docker/fedora-35-cpp.dockerfile +++ b/ci/docker/fedora-35-cpp.dockerfile @@ -76,6 +76,7 @@ COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin ENV absl_SOURCE=BUNDLED \ + ARROW_ACERO=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_DATASET=ON \ diff --git a/ci/docker/linux-apt-docs.dockerfile b/ci/docker/linux-apt-docs.dockerfile index 9b27358a69a..584c31d1f81 100644 --- a/ci/docker/linux-apt-docs.dockerfile +++ b/ci/docker/linux-apt-docs.dockerfile @@ -93,7 +93,8 @@ COPY r/DESCRIPTION /arrow/r/ RUN /arrow/ci/scripts/r_deps.sh /arrow && \ R -e "install.packages('pkgdown')" -ENV ARROW_BUILD_STATIC=OFF \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_BUILD_UTILITIES=OFF \ ARROW_COMPUTE=ON \ diff --git a/ci/docker/linux-apt-jni.dockerfile b/ci/docker/linux-apt-jni.dockerfile index 7a238a58722..c0f915206bf 100644 --- a/ci/docker/linux-apt-jni.dockerfile +++ b/ci/docker/linux-apt-jni.dockerfile @@ -70,7 +70,8 @@ ARG cmake=3.11.4 RUN wget -nv -O - https://github.com/Kitware/CMake/releases/download/v${cmake}/cmake-${cmake}-Linux-x86_64.tar.gz | tar -xzf - -C /opt ENV PATH=/opt/cmake-${cmake}-Linux-x86_64/bin:$PATH -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_FLIGHT=OFF \ ARROW_GANDIVA=ON \ diff --git a/ci/docker/linux-apt-python-3.dockerfile b/ci/docker/linux-apt-python-3.dockerfile index 19f3666ced4..d4e4dec36ab 100644 --- a/ci/docker/linux-apt-python-3.dockerfile +++ b/ci/docker/linux-apt-python-3.dockerfile @@ -45,7 +45,8 @@ RUN if [ "${numba}" != "" ]; then \ /arrow/ci/scripts/install_numba.sh ${numba} \ ; fi -ENV ARROW_BUILD_STATIC=OFF \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_BUILD_UTILITIES=OFF \ ARROW_COMPUTE=ON \ diff --git a/ci/docker/linux-apt-r.dockerfile b/ci/docker/linux-apt-r.dockerfile index 15107fe9c04..19f30717ca2 100644 --- a/ci/docker/linux-apt-r.dockerfile +++ b/ci/docker/linux-apt-r.dockerfile @@ -100,6 +100,7 @@ COPY python/requirements-build.txt /arrow/python/ RUN pip install -r arrow/python/requirements-build.txt ENV \ + ARROW_ACERO=ON \ ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_BUILD_UTILITIES=OFF \ diff --git a/ci/docker/linux-dnf-python-3.dockerfile b/ci/docker/linux-dnf-python-3.dockerfile index c37febb4e00..d1035255d3a 100644 --- a/ci/docker/linux-dnf-python-3.dockerfile +++ b/ci/docker/linux-dnf-python-3.dockerfile @@ -36,7 +36,8 @@ RUN pip install \ -r arrow/python/requirements-build.txt \ -r arrow/python/requirements-test.txt -ENV ARROW_BUILD_STATIC=OFF \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_STATIC=OFF \ ARROW_BUILD_TESTS=OFF \ ARROW_BUILD_UTILITIES=OFF \ ARROW_COMPUTE=ON \ diff --git a/ci/docker/ubuntu-20.04-cpp-minimal.dockerfile b/ci/docker/ubuntu-20.04-cpp-minimal.dockerfile index 0b95d835a15..3df895b427c 100644 --- a/ci/docker/ubuntu-20.04-cpp-minimal.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp-minimal.dockerfile @@ -75,7 +75,8 @@ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_FLIGHT=ON \ ARROW_GANDIVA=ON \ diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index a9d26d63c1f..12dfe2d67a8 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -139,6 +139,7 @@ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin # ARROW-17051: this build uses static Protobuf, so we must also use # static Arrow to run Flight/Flight SQL tests ENV absl_SOURCE=BUNDLED \ + ARROW_ACERO=ON \ ARROW_BUILD_STATIC=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ diff --git a/ci/docker/ubuntu-22.04-cpp-minimal.dockerfile b/ci/docker/ubuntu-22.04-cpp-minimal.dockerfile index a1f33268b6f..7eba541a63a 100644 --- a/ci/docker/ubuntu-22.04-cpp-minimal.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp-minimal.dockerfile @@ -75,7 +75,8 @@ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_sccache.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_ACERO=ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_FLIGHT=ON \ ARROW_GANDIVA=ON \ diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index 5b08ad77f81..03a6bb9c729 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -166,6 +166,7 @@ RUN /arrow/ci/scripts/install_sccache.sh unknown-linux-musl /usr/local/bin # ARROW-17051: this build uses static Protobuf, so we must also use # static Arrow to run Flight/Flight SQL tests ENV absl_SOURCE=BUNDLED \ + ARROW_ACERO=ON \ ARROW_BUILD_STATIC=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ diff --git a/ci/scripts/PKGBUILD b/ci/scripts/PKGBUILD index 2a0285b136a..c77b2001fd5 100644 --- a/ci/scripts/PKGBUILD +++ b/ci/scripts/PKGBUILD @@ -80,6 +80,7 @@ build() { ${MINGW_PREFIX}/bin/cmake.exe \ ${ARROW_CPP_DIR} \ -G "MSYS Makefiles" \ + -DARROW_ACERO=ON \ -DARROW_BUILD_SHARED=OFF \ -DARROW_BUILD_STATIC=ON \ -DARROW_BUILD_UTILITIES=OFF \ diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh index 029391cf0c4..abeb6540475 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -70,6 +70,7 @@ pushd ${build_dir} cmake \ -Dabsl_SOURCE=${absl_SOURCE:-} \ + -DARROW_ACERO=${ARROW_ACERO:-ON} \ -DARROW_BOOST_USE_SHARED=${ARROW_BOOST_USE_SHARED:-ON} \ -DARROW_BUILD_BENCHMARKS_REFERENCE=${ARROW_BUILD_BENCHMARKS:-OFF} \ -DARROW_BUILD_BENCHMARKS=${ARROW_BUILD_BENCHMARKS:-OFF} \ diff --git a/ci/scripts/java_jni_macos_build.sh b/ci/scripts/java_jni_macos_build.sh index 8cdfbb3afb4..a80c9be69a5 100755 --- a/ci/scripts/java_jni_macos_build.sh +++ b/ci/scripts/java_jni_macos_build.sh @@ -39,6 +39,8 @@ rm -rf ${build_dir} echo "=== Building Arrow C++ libraries ===" install_dir=${build_dir}/cpp-install +: ${ARROW_ACERO:=ON} +export ARROW_ACERO : ${ARROW_BUILD_TESTS:=ON} : ${ARROW_DATASET:=ON} export ARROW_DATASET @@ -65,6 +67,7 @@ mkdir -p "${build_dir}/cpp" pushd "${build_dir}/cpp" cmake \ + -DARROW_ACERO=${ARROW_ACERO} \ -DARROW_BUILD_SHARED=OFF \ -DARROW_BUILD_TESTS=${ARROW_BUILD_TESTS} \ -DARROW_CSV=${ARROW_DATASET} \ diff --git a/ci/scripts/java_jni_manylinux_build.sh b/ci/scripts/java_jni_manylinux_build.sh index 9be310317aa..7480aa75aeb 100755 --- a/ci/scripts/java_jni_manylinux_build.sh +++ b/ci/scripts/java_jni_manylinux_build.sh @@ -32,6 +32,8 @@ echo "=== Building Arrow C++ libraries ===" devtoolset_version=$(rpm -qa "devtoolset-*-gcc" --queryformat %{VERSION} | \ grep -o "^[0-9]*") devtoolset_include_cpp="/opt/rh/devtoolset-${devtoolset_version}/root/usr/include/c++/${devtoolset_version}" +: ${ARROW_ACERO:=ON} +export ARROW_ACERO : ${ARROW_BUILD_TESTS:=ON} : ${ARROW_DATASET:=ON} export ARROW_DATASET @@ -64,6 +66,7 @@ mkdir -p "${build_dir}/cpp" pushd "${build_dir}/cpp" cmake \ + -DARROW_ACERO=${ARROW_ACERO} \ -DARROW_BUILD_SHARED=OFF \ -DARROW_BUILD_TESTS=ON \ -DARROW_CSV=${ARROW_DATASET} \ diff --git a/ci/scripts/java_jni_windows_build.sh b/ci/scripts/java_jni_windows_build.sh index ce445db578f..7e079b17283 100755 --- a/ci/scripts/java_jni_windows_build.sh +++ b/ci/scripts/java_jni_windows_build.sh @@ -30,6 +30,8 @@ rm -rf ${build_dir} echo "=== Building Arrow C++ libraries ===" install_dir=${build_dir}/cpp-install +: ${ARROW_ACERO:=ON} +export ARROW_ACERO : ${ARROW_BUILD_TESTS:=ON} : ${ARROW_DATASET:=ON} export ARROW_DATASET @@ -54,6 +56,7 @@ mkdir -p "${build_dir}/cpp" pushd "${build_dir}/cpp" cmake \ + -DARROW_ACERO=${ARROW_ACERO} \ -DARROW_BUILD_SHARED=OFF \ -DARROW_BUILD_TESTS=ON \ -DARROW_CSV=${ARROW_DATASET} \ diff --git a/ci/scripts/python_build.sh b/ci/scripts/python_build.sh index 766c0f7a5f3..b5b5b75b967 100755 --- a/ci/scripts/python_build.sh +++ b/ci/scripts/python_build.sh @@ -54,6 +54,7 @@ fi export PYARROW_CMAKE_GENERATOR=${CMAKE_GENERATOR:-Ninja} export PYARROW_BUILD_TYPE=${CMAKE_BUILD_TYPE:-debug} +export PYARROW_WITH_ACERO=${ARROW_ACERO:-ON} export PYARROW_WITH_CUDA=${ARROW_CUDA:-OFF} export PYARROW_WITH_DATASET=${ARROW_DATASET:-ON} export PYARROW_WITH_FLIGHT=${ARROW_FLIGHT:-OFF} diff --git a/ci/scripts/python_sdist_test.sh b/ci/scripts/python_sdist_test.sh index 1c61ebd8310..d3c6f0e6ade 100755 --- a/ci/scripts/python_sdist_test.sh +++ b/ci/scripts/python_sdist_test.sh @@ -27,6 +27,7 @@ export PARQUET_TEST_DATA=${arrow_dir}/cpp/submodules/parquet-testing/data export PYARROW_CMAKE_GENERATOR=${CMAKE_GENERATOR:-Ninja} export PYARROW_BUILD_TYPE=${CMAKE_BUILD_TYPE:-debug} +export PYARROW_WITH_ACERO=${ARROW_ACERO:-ON} export PYARROW_WITH_S3=${ARROW_S3:-OFF} export PYARROW_WITH_ORC=${ARROW_ORC:-OFF} export PYARROW_WITH_CUDA=${ARROW_CUDA:-OFF} diff --git a/ci/scripts/python_test.sh b/ci/scripts/python_test.sh index 2d5bd5dd9ff..8d818346faa 100755 --- a/ci/scripts/python_test.sh +++ b/ci/scripts/python_test.sh @@ -36,6 +36,7 @@ export PYTHONDEVMODE=1 export ARROW_DEBUG_MEMORY_POOL=trap # By default, force-test all optional components +: ${PYARROW_TEST_ACERO:=${ARROW_ACERO:-ON}} : ${PYARROW_TEST_CUDA:=${ARROW_CUDA:-ON}} : ${PYARROW_TEST_DATASET:=${ARROW_DATASET:-ON}} : ${PYARROW_TEST_FLIGHT:=${ARROW_FLIGHT:-ON}} @@ -46,6 +47,7 @@ export ARROW_DEBUG_MEMORY_POOL=trap : ${PYARROW_TEST_PARQUET:=${ARROW_PARQUET:-ON}} : ${PYARROW_TEST_S3:=${ARROW_S3:-ON}} +export PYARROW_TEST_ACERO export PYARROW_TEST_CUDA export PYARROW_TEST_DATASET export PYARROW_TEST_FLIGHT diff --git a/ci/scripts/python_wheel_macos_build.sh b/ci/scripts/python_wheel_macos_build.sh index 6f1d216a372..5a3c6fb6d1f 100755 --- a/ci/scripts/python_wheel_macos_build.sh +++ b/ci/scripts/python_wheel_macos_build.sh @@ -59,6 +59,7 @@ pip install \ pip install "delocate>=0.10.3" echo "=== (${PYTHON_VERSION}) Building Arrow C++ libraries ===" +: ${ARROW_ACERO:=ON} : ${ARROW_DATASET:=ON} : ${ARROW_FLIGHT:=ON} : ${ARROW_GANDIVA:=OFF} @@ -90,6 +91,7 @@ mkdir -p ${build_dir}/build pushd ${build_dir}/build cmake \ + -DARROW_ACERO=${ARROW_ACERO} \ -DARROW_BUILD_SHARED=ON \ -DARROW_BUILD_STATIC=OFF \ -DARROW_BUILD_TESTS=OFF \ @@ -142,6 +144,7 @@ export PYARROW_BUILD_TYPE=${CMAKE_BUILD_TYPE} export PYARROW_BUNDLE_ARROW_CPP=1 export PYARROW_CMAKE_GENERATOR=${CMAKE_GENERATOR} export PYARROW_INSTALL_TESTS=1 +export PYARROW_WITH_ACERO=${ARROW_ACERO} export PYARROW_WITH_DATASET=${ARROW_DATASET} export PYARROW_WITH_FLIGHT=${ARROW_FLIGHT} export PYARROW_WITH_GANDIVA=${ARROW_GANDIVA} diff --git a/ci/scripts/python_wheel_manylinux_build.sh b/ci/scripts/python_wheel_manylinux_build.sh index 0bc4daa2a87..cb5c2fbb7cc 100755 --- a/ci/scripts/python_wheel_manylinux_build.sh +++ b/ci/scripts/python_wheel_manylinux_build.sh @@ -48,6 +48,7 @@ rm -rf /arrow/python/pyarrow/*.so rm -rf /arrow/python/pyarrow/*.so.* echo "=== (${PYTHON_VERSION}) Building Arrow C++ libraries ===" +: ${ARROW_ACERO:=ON} : ${ARROW_DATASET:=ON} : ${ARROW_FLIGHT:=ON} : ${ARROW_GANDIVA:=OFF} @@ -88,6 +89,7 @@ pushd /tmp/arrow-build # https://github.com/aws/aws-sdk-cpp/issues/1809 is fixed and vcpkg # ships the fix. cmake \ + -DARROW_ACERO=${ARROW_ACERO} \ -DARROW_BUILD_SHARED=ON \ -DARROW_BUILD_STATIC=OFF \ -DARROW_BUILD_TESTS=OFF \ @@ -142,6 +144,7 @@ export PYARROW_BUILD_TYPE=${CMAKE_BUILD_TYPE} export PYARROW_BUNDLE_ARROW_CPP=1 export PYARROW_CMAKE_GENERATOR=${CMAKE_GENERATOR} export PYARROW_INSTALL_TESTS=1 +export PYARROW_WITH_ACERO=${ARROW_ACERO} export PYARROW_WITH_DATASET=${ARROW_DATASET} export PYARROW_WITH_FLIGHT=${ARROW_FLIGHT} export PYARROW_WITH_GANDIVA=${ARROW_GANDIVA} diff --git a/ci/scripts/python_wheel_unix_test.sh b/ci/scripts/python_wheel_unix_test.sh index b7a7114cc5c..a6cc3bb7b29 100755 --- a/ci/scripts/python_wheel_unix_test.sh +++ b/ci/scripts/python_wheel_unix_test.sh @@ -36,6 +36,7 @@ source_dir=${1} : ${CHECK_UNITTESTS:=ON} : ${INSTALL_PYARROW:=ON} +export PYARROW_TEST_ACERO=ON export PYARROW_TEST_CYTHON=OFF export PYARROW_TEST_DATASET=ON export PYARROW_TEST_FLIGHT=${ARROW_FLIGHT} diff --git a/ci/scripts/python_wheel_windows_build.bat b/ci/scripts/python_wheel_windows_build.bat index 6a5c994d08b..80270a84c45 100644 --- a/ci/scripts/python_wheel_windows_build.bat +++ b/ci/scripts/python_wheel_windows_build.bat @@ -30,6 +30,7 @@ del /s /q C:\arrow\python\pyarrow\*.so del /s /q C:\arrow\python\pyarrow\*.so.* echo "=== (%PYTHON_VERSION%) Building Arrow C++ libraries ===" +set ARROW_ACERO=ON set ARROW_DATASET=ON set ARROW_FLIGHT=ON set ARROW_GANDIVA=OFF @@ -59,6 +60,7 @@ set VCGPK_TARGET_TRIPLET=amd64-windows-static-md-%CMAKE_BUILD_TYPE% mkdir C:\arrow-build pushd C:\arrow-build cmake ^ + -DARROW_ACERO=%ARROW_ACERO% ^ -DARROW_BUILD_SHARED=ON ^ -DARROW_BUILD_STATIC=OFF ^ -DARROW_BUILD_TESTS=OFF ^ @@ -105,6 +107,7 @@ set PYARROW_BUILD_TYPE=%CMAKE_BUILD_TYPE% set PYARROW_BUNDLE_ARROW_CPP=ON set PYARROW_CMAKE_GENERATOR=%CMAKE_GENERATOR% set PYARROW_INSTALL_TESTS=ON +set PYARROW_WITH_ACERO=%ARROW_ACERO% set PYARROW_WITH_DATASET=%ARROW_DATASET% set PYARROW_WITH_FLIGHT=%ARROW_FLIGHT% set PYARROW_WITH_GANDIVA=%ARROW_GANDIVA% diff --git a/ci/scripts/python_wheel_windows_test.bat b/ci/scripts/python_wheel_windows_test.bat index 0aa24e491da..cae1b7ef32a 100755 --- a/ci/scripts/python_wheel_windows_test.bat +++ b/ci/scripts/python_wheel_windows_test.bat @@ -17,6 +17,7 @@ @echo on +set PYARROW_TEST_ACERO=ON set PYARROW_TEST_CYTHON=ON set PYARROW_TEST_DATASET=ON set PYARROW_TEST_FLIGHT=ON diff --git a/cpp/CMakePresets.json b/cpp/CMakePresets.json index d015ed24788..d714af54ddf 100644 --- a/cpp/CMakePresets.json +++ b/cpp/CMakePresets.json @@ -71,6 +71,7 @@ "hidden": true, "cacheVariables": { "ARROW_SUBSTRAIT": "ON", + "ARROW_ACERO": "ON", "ARROW_MIMALLOC": "ON", "ARROW_PARQUET": "ON", "ARROW_WITH_BROTLI": "ON", diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index ff34f3f5697..6038bfff3c7 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -309,7 +309,7 @@ takes precedence over ccache if a storage backend is configured" ON) "Build the Arrow Dataset Modules" OFF DEPENDS - ARROW_COMPUTE + ARROW_ACERO ARROW_FILESYSTEM) define_option(ARROW_FILESYSTEM "Build the Arrow Filesystem Layer" OFF) @@ -376,7 +376,6 @@ takes precedence over ccache if a storage backend is configured" ON) (This is a deprecated option. Use CMake presets instead.)" OFF DEPENDS - ARROW_COMPUTE ARROW_CSV ARROW_DATASET ARROW_FILESYSTEM @@ -402,6 +401,14 @@ takes precedence over ccache if a storage backend is configured" ON) ARROW_IPC ARROW_PARQUET) + define_option(ARROW_ACERO + "Build the Arrow Acero Engine Module" + OFF + DEPENDS + ARROW_COMPUTE + ARROW_IPC + ARROW_PARQUET) + define_option(ARROW_TENSORFLOW "Build Arrow with TensorFlow support enabled" OFF) define_option(ARROW_TESTING diff --git a/cpp/examples/arrow/CMakeLists.txt b/cpp/examples/arrow/CMakeLists.txt index 76c176086ac..4625f130565 100644 --- a/cpp/examples/arrow/CMakeLists.txt +++ b/cpp/examples/arrow/CMakeLists.txt @@ -19,8 +19,14 @@ add_arrow_example(row_wise_conversion_example) add_arrow_example(rapidjson_row_converter) -if(ARROW_COMPUTE) - add_arrow_example(compute_register_example) +if(ARROW_ACERO) + if(ARROW_BUILD_SHARED) + set(ENGINE_ACERO_EXAMPLE_LINK_LIBS arrow_acero_shared) + else() + set(ENGINE_ACERO_EXAMPLE_LINK_LIBS arrow_acero_static) + endif() + add_arrow_example(acero_register_example EXTRA_LINK_LIBS + ${ENGINE_ACERO_EXAMPLE_LINK_LIBS}) endif() if(ARROW_SUBSTRAIT) diff --git a/cpp/examples/arrow/compute_register_example.cc b/cpp/examples/arrow/acero_register_example.cc similarity index 90% rename from cpp/examples/arrow/compute_register_example.cc rename to cpp/examples/arrow/acero_register_example.cc index a97c17ac704..f6c73b08a72 100644 --- a/cpp/examples/arrow/compute_register_example.cc +++ b/cpp/examples/arrow/acero_register_example.cc @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. +#include +#include #include #include -#include -#include +#include #include #include -#include "arrow/compute/expression.h" #include #include @@ -30,6 +30,7 @@ // Demonstrate registering an Arrow compute function outside of the Arrow source tree namespace cp = ::arrow::compute; +namespace ac = ::arrow::acero; class ExampleFunctionOptionsType : public cp::FunctionOptionsType { const char* type_name() const override { return "ExampleFunctionOptionsType"; } @@ -66,10 +67,10 @@ arrow::Status ExampleFunctionImpl(cp::KernelContext* ctx, const cp::ExecSpan& ba return arrow::Status::OK(); } -class ExampleNodeOptions : public cp::ExecNodeOptions {}; +class ExampleNodeOptions : public ac::ExecNodeOptions {}; // a basic ExecNode which ignores all input batches -class ExampleNode : public cp::ExecNode { +class ExampleNode : public ac::ExecNode { public: ExampleNode(ExecNode* input, const ExampleNodeOptions&) : ExecNode(/*plan=*/input->plan(), /*inputs=*/{input}, @@ -97,9 +98,9 @@ class ExampleNode : public cp::ExecNode { } }; -arrow::Result ExampleExecNodeFactory(cp::ExecPlan* plan, - std::vector inputs, - const cp::ExecNodeOptions& options) { +arrow::Result ExampleExecNodeFactory(ac::ExecPlan* plan, + std::vector inputs, + const ac::ExecNodeOptions& options) { const auto& example_options = arrow::internal::checked_cast(options); @@ -138,21 +139,21 @@ arrow::Status RunComputeRegister(int argc, char** argv) { auto maybe_serialized = cp::Serialize(expr); std::cerr << maybe_serialized.status().ToString() << std::endl; - auto exec_registry = cp::default_exec_factory_registry(); + auto exec_registry = ac::default_exec_factory_registry(); ARROW_RETURN_NOT_OK( exec_registry->AddFactory("compute_register_example", ExampleExecNodeFactory)); - auto maybe_plan = cp::ExecPlan::Make(); + auto maybe_plan = ac::ExecPlan::Make(); ARROW_RETURN_NOT_OK(maybe_plan.status()); ARROW_ASSIGN_OR_RAISE(auto plan, maybe_plan); arrow::AsyncGenerator> source_gen, sink_gen; ARROW_RETURN_NOT_OK( - cp::Declaration::Sequence( + ac::Declaration::Sequence( { - {"source", cp::SourceNodeOptions{arrow::schema({}), source_gen}}, + {"source", ac::SourceNodeOptions{arrow::schema({}), source_gen}}, {"compute_register_example", ExampleNodeOptions{}}, - {"sink", cp::SinkNodeOptions{&sink_gen}}, + {"sink", ac::SinkNodeOptions{&sink_gen}}, }) .AddToPlan(plan.get()) .status()); diff --git a/cpp/examples/arrow/engine_substrait_consumption.cc b/cpp/examples/arrow/engine_substrait_consumption.cc index 61bf8814fd3..f833a977d5e 100644 --- a/cpp/examples/arrow/engine_substrait_consumption.cc +++ b/cpp/examples/arrow/engine_substrait_consumption.cc @@ -26,14 +26,15 @@ namespace eng = arrow::engine; namespace cp = arrow::compute; +namespace ac = arrow::acero; -class IgnoringConsumer : public cp::SinkNodeConsumer { +class IgnoringConsumer : public ac::SinkNodeConsumer { public: explicit IgnoringConsumer(size_t tag) : tag_{tag} {} arrow::Status Init(const std::shared_ptr& schema, - cp::BackpressureControl* backpressure_control, - cp::ExecPlan* plan) override { + ac::BackpressureControl* backpressure_control, + ac::ExecPlan* plan) override { return arrow::Status::OK(); } @@ -130,8 +131,8 @@ arrow::Status RunSubstraitConsumer(int argc, char** argv) { // Arrow. Therefore, deserializing a plan requires a factory for consumers: each // time the root of a substrait relation tree is deserialized, an Arrow consumer is // constructed into which its batches will be piped. - std::vector> consumers; - std::function()> consumer_factory = [&] { + std::vector> consumers; + std::function()> consumer_factory = [&] { // All batches produced by the plan will be fed into IgnoringConsumers: auto tag = consumers.size(); consumers.emplace_back(new IgnoringConsumer{tag}); @@ -139,19 +140,19 @@ arrow::Status RunSubstraitConsumer(int argc, char** argv) { }; // Deserialize each relation tree in the substrait plan to an Arrow compute Declaration - arrow::Result> maybe_decls = + arrow::Result> maybe_decls = eng::DeserializePlans(*serialized_plan, consumer_factory); ARROW_RETURN_NOT_OK(maybe_decls.status()); - ARROW_ASSIGN_OR_RAISE(std::vector decls, std::move(maybe_decls)); + ARROW_ASSIGN_OR_RAISE(std::vector decls, std::move(maybe_decls)); // It's safe to drop the serialized plan; we don't leave references to its memory serialized_plan.reset(); // Construct an empty plan (note: configure Function registry and ThreadPool here) - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, cp::ExecPlan::Make()); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, ac::ExecPlan::Make()); // Add decls to plan (note: configure ExecNode registry before this point) - for (const cp::Declaration& decl : decls) { + for (const ac::Declaration& decl : decls) { ARROW_RETURN_NOT_OK(decl.AddToPlan(plan.get()).status()); } diff --git a/cpp/examples/arrow/execution_plan_documentation_examples.cc b/cpp/examples/arrow/execution_plan_documentation_examples.cc index 25baaa7d0a1..caccb62aaa6 100644 --- a/cpp/examples/arrow/execution_plan_documentation_examples.cc +++ b/cpp/examples/arrow/execution_plan_documentation_examples.cc @@ -20,10 +20,10 @@ #include #include +#include #include #include #include -#include #include @@ -54,6 +54,7 @@ // Demonstrate various operators in Arrow Streaming Execution Engine namespace cp = ::arrow::compute; +namespace ac = ::arrow::acero; constexpr char kSep[] = "******"; @@ -256,10 +257,10 @@ arrow::Result MakeGroupableBatches(int multiplicity = 1) { return out; } -arrow::Status ExecutePlanAndCollectAsTable(cp::Declaration plan) { +arrow::Status ExecutePlanAndCollectAsTable(ac::Declaration plan) { // collect sink_reader into a Table std::shared_ptr response_table; - ARROW_ASSIGN_OR_RAISE(response_table, cp::DeclarationToTable(std::move(plan))); + ARROW_ASSIGN_OR_RAISE(response_table, ac::DeclarationToTable(std::move(plan))); std::cout << "Results : " << response_table->ToString() << std::endl; @@ -283,7 +284,7 @@ arrow::Status ScanSinkExample() { // construct the scan node auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - cp::Declaration scan{"scan", std::move(scan_node_options)}; + ac::Declaration scan{"scan", std::move(scan_node_options)}; return ExecutePlanAndCollectAsTable(std::move(scan)); } @@ -304,9 +305,9 @@ arrow::Status ScanSinkExample() { arrow::Status SourceSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - cp::Declaration source{"source", std::move(source_node_options)}; + ac::Declaration source{"source", std::move(source_node_options)}; return ExecutePlanAndCollectAsTable(std::move(source)); } @@ -327,9 +328,9 @@ arrow::Status TableSourceSinkExample() { arrow::AsyncGenerator> sink_gen; int max_batch_size = 2; - auto table_source_options = cp::TableSourceNodeOptions{table, max_batch_size}; + auto table_source_options = ac::TableSourceNodeOptions{table, max_batch_size}; - cp::Declaration source{"table_source", std::move(table_source_options)}; + ac::Declaration source{"table_source", std::move(table_source_options)}; return ExecutePlanAndCollectAsTable(std::move(source)); } @@ -362,14 +363,14 @@ arrow::Status ScanFilterSinkExample() { auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; std::cout << "Scan node options created" << std::endl; - cp::Declaration scan{"scan", std::move(scan_node_options)}; + ac::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::Declaration filter{ - "filter", {std::move(scan)}, cp::FilterNodeOptions(std::move(filter_expr))}; + ac::Declaration filter{ + "filter", {std::move(scan)}, ac::FilterNodeOptions(std::move(filter_expr))}; return ExecutePlanAndCollectAsTable(std::move(filter)); } @@ -394,9 +395,9 @@ arrow::Status ScanProjectSinkExample() { auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - cp::Declaration scan{"scan", std::move(scan_node_options)}; - cp::Declaration project{ - "project", {std::move(scan)}, cp::ProjectNodeOptions({a_times_2})}; + ac::Declaration scan{"scan", std::move(scan_node_options)}; + ac::Declaration project{ + "project", {std::move(scan)}, ac::ProjectNodeOptions({a_times_2})}; return ExecutePlanAndCollectAsTable(std::move(project)); } @@ -416,12 +417,12 @@ arrow::Status ScanProjectSinkExample() { arrow::Status SourceScalarAggregateSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - cp::Declaration source{"source", std::move(source_node_options)}; + ac::Declaration source{"source", std::move(source_node_options)}; auto aggregate_options = - cp::AggregateNodeOptions{/*aggregates=*/{{"sum", nullptr, "a", "sum(a)"}}}; - cp::Declaration aggregate{ + ac::AggregateNodeOptions{/*aggregates=*/{{"sum", nullptr, "a", "sum(a)"}}}; + ac::Declaration aggregate{ "aggregate", {std::move(source)}, std::move(aggregate_options)}; return ExecutePlanAndCollectAsTable(std::move(aggregate)); @@ -443,14 +444,14 @@ arrow::Status SourceGroupAggregateSinkExample() { arrow::AsyncGenerator> sink_gen; - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - cp::Declaration source{"source", std::move(source_node_options)}; + ac::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)"}}, + ac::AggregateNodeOptions{/*aggregates=*/{{"hash_count", options, "a", "count(a)"}}, /*keys=*/{"b"}}; - cp::Declaration aggregate{ + ac::Declaration aggregate{ "aggregate", {std::move(source)}, std::move(aggregate_options)}; return ExecutePlanAndCollectAsTable(std::move(aggregate)); @@ -467,19 +468,19 @@ arrow::Status SourceGroupAggregateSinkExample() { arrow::Status SourceConsumingSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - cp::Declaration source{"source", std::move(source_node_options)}; + ac::Declaration source{"source", std::move(source_node_options)}; std::atomic batches_seen{0}; arrow::Future<> finish = arrow::Future<>::Make(); - struct CustomSinkNodeConsumer : public cp::SinkNodeConsumer { + struct CustomSinkNodeConsumer : public ac::SinkNodeConsumer { CustomSinkNodeConsumer(std::atomic* batches_seen, arrow::Future<> finish) : batches_seen(batches_seen), finish(std::move(finish)) {} arrow::Status Init(const std::shared_ptr& schema, - cp::BackpressureControl* backpressure_control, - cp::ExecPlan* plan) override { + ac::BackpressureControl* backpressure_control, + ac::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 @@ -504,13 +505,13 @@ arrow::Status SourceConsumingSinkExample() { std::shared_ptr consumer = std::make_shared(&batches_seen, finish); - cp::Declaration consuming_sink{"consuming_sink", + ac::Declaration consuming_sink{"consuming_sink", {std::move(source)}, - cp::ConsumingSinkNodeOptions(std::move(consumer))}; + ac::ConsumingSinkNodeOptions(std::move(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(ac::DeclarationToStatus(std::move(consuming_sink))); std::cout << "The consuming sink node saw " << batches_seen.load() << " batches" << std::endl; @@ -521,11 +522,11 @@ arrow::Status SourceConsumingSinkExample() { // (Doc section: OrderBySink Example) arrow::Status ExecutePlanAndCollectAsTableWithCustomSink( - std::shared_ptr plan, std::shared_ptr schema, + 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()); + ac::MakeGeneratorReader(schema, std::move(sink_gen), arrow::default_memory_pool()); // validate the ExecPlan ARROW_RETURN_NOT_OK(plan->Validate()); @@ -556,20 +557,20 @@ arrow::Status ExecutePlanAndCollectAsTableWithCustomSink( /// ASCENDING or DESCENDING and it is configurable. The output /// is obtained as a table from the sink node. arrow::Status SourceOrderBySinkExample() { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(*cp::threaded_exec_context())); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, + ac::ExecPlan::Make(*cp::threaded_exec_context())); ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeSortTestBasicBatches()); 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)); + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + ARROW_ASSIGN_OR_RAISE(ac::ExecNode * source, + ac::MakeExecNode("source", plan.get(), {}, source_node_options)); - ARROW_RETURN_NOT_OK(cp::MakeExecNode( + ARROW_RETURN_NOT_OK(ac::MakeExecNode( "order_by_sink", plan.get(), {source}, - cp::OrderBySinkNodeOptions{ + ac::OrderBySinkNodeOptions{ cp::SortOptions{{cp::SortKey{"a", cp::SortOrder::Descending}}}, &sink_gen})); return ExecutePlanAndCollectAsTableWithCustomSink(plan, basic_data.schema, sink_gen); @@ -588,15 +589,15 @@ arrow::Status SourceOrderBySinkExample() { arrow::Status SourceHashJoinSinkExample() { ARROW_ASSIGN_OR_RAISE(auto input, MakeGroupableBatches()); - cp::Declaration left{"source", cp::SourceNodeOptions{input.schema, input.gen()}}; - cp::Declaration right{"source", cp::SourceNodeOptions{input.schema, input.gen()}}; + ac::Declaration left{"source", ac::SourceNodeOptions{input.schema, input.gen()}}; + ac::Declaration right{"source", ac::SourceNodeOptions{input.schema, input.gen()}}; - cp::HashJoinNodeOptions join_opts{ - cp::JoinType::INNER, + ac::HashJoinNodeOptions join_opts{ + ac::JoinType::INNER, /*left_keys=*/{"str"}, /*right_keys=*/{"str"}, cp::literal(true), "l_", "r_"}; - cp::Declaration hashjoin{ + ac::Declaration hashjoin{ "hashjoin", {std::move(left), std::move(right)}, std::move(join_opts)}; return ExecutePlanAndCollectAsTable(std::move(hashjoin)); @@ -614,19 +615,19 @@ arrow::Status SourceHashJoinSinkExample() { /// sink node where output can be obtained as a table. arrow::Status SourceKSelectExample() { ARROW_ASSIGN_OR_RAISE(auto input, MakeGroupableBatches()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(*cp::threaded_exec_context())); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, + ac::ExecPlan::Make(*cp::threaded_exec_context())); arrow::AsyncGenerator> sink_gen; ARROW_ASSIGN_OR_RAISE( - cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, - cp::SourceNodeOptions{input.schema, input.gen()})); + ac::ExecNode * source, + ac::MakeExecNode("source", plan.get(), {}, + ac::SourceNodeOptions{input.schema, input.gen()})); cp::SelectKOptions options = cp::SelectKOptions::TopKDefault(/*k=*/2, {"i32"}); - ARROW_RETURN_NOT_OK(cp::MakeExecNode("select_k_sink", plan.get(), {source}, - cp::SelectKSinkNodeOptions{options, &sink_gen})); + ARROW_RETURN_NOT_OK(ac::MakeExecNode("select_k_sink", plan.get(), {source}, + ac::SelectKSinkNodeOptions{options, &sink_gen})); auto schema = arrow::schema( {arrow::field("i32", arrow::int32()), arrow::field("str", arrow::utf8())}); @@ -653,7 +654,7 @@ arrow::Status ScanFilterWriteExample(const std::string& file_path) { auto scan_node_options = arrow::dataset::ScanNodeOptions{dataset, options}; - cp::Declaration scan{"scan", std::move(scan_node_options)}; + ac::Declaration scan{"scan", std::move(scan_node_options)}; arrow::AsyncGenerator> sink_gen; @@ -686,11 +687,11 @@ arrow::Status ScanFilterWriteExample(const std::string& file_path) { arrow::dataset::WriteNodeOptions write_node_options{write_options}; - cp::Declaration write{"write", {std::move(scan)}, std::move(write_node_options)}; + ac::Declaration write{"write", {std::move(scan)}, std::move(write_node_options)}; // 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))); + ARROW_RETURN_NOT_OK(ac::DeclarationToStatus(std::move(write))); std::cout << "Dataset written to " << base_path << std::endl; return arrow::Status::OK(); @@ -708,14 +709,14 @@ arrow::Status ScanFilterWriteExample(const std::string& file_path) { arrow::Status SourceUnionSinkExample() { ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - cp::Declaration lhs{"source", - cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; + ac::Declaration lhs{"source", + ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; lhs.label = "lhs"; - cp::Declaration rhs{"source", - cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; + ac::Declaration rhs{"source", + ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}}; rhs.label = "rhs"; - cp::Declaration union_plan{ - "union", {std::move(lhs), std::move(rhs)}, cp::ExecNodeOptions{}}; + ac::Declaration union_plan{ + "union", {std::move(lhs), std::move(rhs)}, ac::ExecNodeOptions{}}; return ExecutePlanAndCollectAsTable(std::move(union_plan)); } @@ -732,21 +733,21 @@ arrow::Status SourceUnionSinkExample() { /// receiving data as batches and the table sink node /// which emits the output as a table. arrow::Status TableSinkExample() { - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - cp::ExecPlan::Make(*cp::threaded_exec_context())); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, + ac::ExecPlan::Make(*cp::threaded_exec_context())); ARROW_ASSIGN_OR_RAISE(auto basic_data, MakeBasicBatches()); - auto source_node_options = cp::SourceNodeOptions{basic_data.schema, basic_data.gen()}; + auto source_node_options = ac::SourceNodeOptions{basic_data.schema, basic_data.gen()}; - ARROW_ASSIGN_OR_RAISE(cp::ExecNode * source, - cp::MakeExecNode("source", plan.get(), {}, source_node_options)); + ARROW_ASSIGN_OR_RAISE(ac::ExecNode * source, + ac::MakeExecNode("source", plan.get(), {}, source_node_options)); std::shared_ptr output_table; - auto table_sink_options = cp::TableSinkNodeOptions{&output_table}; + auto table_sink_options = ac::TableSinkNodeOptions{&output_table}; ARROW_RETURN_NOT_OK( - cp::MakeExecNode("table_sink", plan.get(), {source}, table_sink_options)); + ac::MakeExecNode("table_sink", plan.get(), {source}, table_sink_options)); // validate the ExecPlan ARROW_RETURN_NOT_OK(plan->Validate()); std::cout << "ExecPlan created : " << plan->ToString() << std::endl; @@ -775,8 +776,8 @@ arrow::Status RecordBatchReaderSourceSinkExample() { ARROW_ASSIGN_OR_RAISE(auto table, GetTable()); std::shared_ptr reader = std::make_shared(table); - cp::Declaration reader_source{"record_batch_reader_source", - cp::RecordBatchReaderSourceNodeOptions{reader}}; + ac::Declaration reader_source{"record_batch_reader_source", + ac::RecordBatchReaderSourceNodeOptions{reader}}; return ExecutePlanAndCollectAsTable(std::move(reader_source)); } diff --git a/cpp/examples/arrow/join_example.cc b/cpp/examples/arrow/join_example.cc index d912399b3c8..17f709c720e 100644 --- a/cpp/examples/arrow/join_example.cc +++ b/cpp/examples/arrow/join_example.cc @@ -20,8 +20,8 @@ #include #include -#include #include +#include "arrow/acero/exec_plan.h" #include "arrow/compute/expression.h" #include @@ -101,22 +101,22 @@ 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::compute::Declaration left{"scan", std::move(l_scan_node_options)}; - arrow::compute::Declaration right{"scan", std::move(r_scan_node_options)}; + arrow::acero::Declaration left{"scan", std::move(l_scan_node_options)}; + arrow::acero::Declaration right{"scan", std::move(r_scan_node_options)}; - arrow::compute::HashJoinNodeOptions join_opts{arrow::compute::JoinType::INNER, - /*in_left_keys=*/{"lkey"}, - /*in_right_keys=*/{"rkey"}, - /*filter*/ arrow::compute::literal(true), - /*output_suffix_for_left*/ "_l", - /*output_suffix_for_right*/ "_r"}; + arrow::acero::HashJoinNodeOptions join_opts{arrow::acero::JoinType::INNER, + /*in_left_keys=*/{"lkey"}, + /*in_right_keys=*/{"rkey"}, + /*filter*/ arrow::compute::literal(true), + /*output_suffix_for_left*/ "_l", + /*output_suffix_for_right*/ "_r"}; - arrow::compute::Declaration hashjoin{ + arrow::acero::Declaration hashjoin{ "hashjoin", {std::move(left), std::move(right)}, join_opts}; // expected columns l_a, l_b ARROW_ASSIGN_OR_RAISE(std::shared_ptr response_table, - arrow::compute::DeclarationToTable(std::move(hashjoin))); + arrow::acero::DeclarationToTable(std::move(hashjoin))); std::cout << "Results : " << response_table->ToString() << std::endl; diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 3825e274ee9..07ea8930ff0 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -390,33 +390,7 @@ list(APPEND compute/api_vector.cc compute/cast.cc compute/exec.cc - compute/exec/groupby.cc - compute/exec/accumulation_queue.cc - compute/exec/aggregate_node.cc - compute/exec/asof_join_node.cc - compute/exec/bloom_filter.cc - compute/exec/exec_plan.cc compute/expression.cc - compute/exec/fetch_node.cc - compute/exec/filter_node.cc - compute/exec/hash_join.cc - compute/exec/hash_join_dict.cc - compute/exec/hash_join_node.cc - compute/exec/map_node.cc - compute/exec/options.cc - compute/exec/order_by_impl.cc - compute/exec/order_by_node.cc - compute/exec/partition_util.cc - compute/exec/pivot_longer_node.cc - compute/exec/project_node.cc - compute/exec/query_context.cc - compute/exec/sink_node.cc - compute/exec/source_node.cc - compute/exec/swiss_join.cc - compute/exec/task_util.cc - compute/exec/tpch_node.cc - compute/exec/union_node.cc - compute/exec/util.cc compute/function.cc compute/function_internal.cc compute/kernel.cc @@ -444,16 +418,11 @@ list(APPEND compute/row/row_internal.cc compute/util.cc) -append_avx2_src(compute/exec/bloom_filter_avx2.cc) append_avx2_src(compute/key_hash_avx2.cc) append_avx2_src(compute/key_map_avx2.cc) -append_avx2_src(compute/exec/swiss_join_avx2.cc) -append_avx2_src(compute/exec/util_avx2.cc) append_avx2_src(compute/row/compare_internal_avx2.cc) append_avx2_src(compute/row/encode_internal_avx2.cc) -list(APPEND ARROW_TESTING_SRCS compute/exec/test_util.cc) - if(ARROW_COMPUTE) # Include the remaining kernels list(APPEND @@ -861,6 +830,10 @@ if(ARROW_SUBSTRAIT) add_subdirectory(engine) endif() +if(ARROW_ACERO) + add_subdirectory(acero) +endif() + if(ARROW_CUDA) add_subdirectory(gpu) endif() diff --git a/cpp/src/arrow/acero/ArrowAceroConfig.cmake.in b/cpp/src/arrow/acero/ArrowAceroConfig.cmake.in new file mode 100644 index 00000000000..124cbcbf3d4 --- /dev/null +++ b/cpp/src/arrow/acero/ArrowAceroConfig.cmake.in @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# This config sets the following variables in your project:: +# +# ArrowAcero_FOUND - true if Arrow Acero found on the system +# +# This config sets the following targets in your project:: +# +# ArrowAcero::arrow_acero_shared - for linked as shared library if shared library is built +# ArrowAcero::arrow_acero_static - for linked as static library if static library is built + +@PACKAGE_INIT@ + +include(CMakeFindDependencyMacro) +find_dependency(Arrow) +find_dependency(Parquet) + +include("${CMAKE_CURRENT_LIST_DIR}/ArrowAceroTargets.cmake") + +arrow_keep_backward_compatibility(ArrowAcero arrow_acero) + +check_required_components(ArrowAcero) + +arrow_show_details(ArrowAcero ARROW_ACERO) diff --git a/cpp/src/arrow/acero/CMakeLists.txt b/cpp/src/arrow/acero/CMakeLists.txt new file mode 100644 index 00000000000..431cb53703f --- /dev/null +++ b/cpp/src/arrow/acero/CMakeLists.txt @@ -0,0 +1,250 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +add_custom_target(arrow_acero) + +arrow_install_all_headers("arrow/acero") + +set(ARROW_ACERO_SRCS + groupby.cc + accumulation_queue.cc + aggregate_node.cc + asof_join_node.cc + bloom_filter.cc + exec_plan.cc + fetch_node.cc + filter_node.cc + hash_join.cc + hash_join_dict.cc + hash_join_node.cc + map_node.cc + options.cc + order_by_node.cc + order_by_impl.cc + partition_util.cc + pivot_longer_node.cc + project_node.cc + query_context.cc + sink_node.cc + source_node.cc + swiss_join.cc + task_util.cc + tpch_node.cc + union_node.cc + util.cc) + +append_avx2_src(bloom_filter_avx2.cc) +append_avx2_src(swiss_join_avx2.cc) +append_avx2_src(util_avx2.cc) + +set(ARROW_ACERO_PKG_CONFIG_REQUIRES arrow) + +set(ARROW_ACERO_SHARED_LINK_LIBS) +set(ARROW_ACERO_STATIC_LINK_LIBS) +set(ARROW_ACERO_STATIC_INSTALL_INTERFACE_LIBS) +set(ARROW_ACERO_SHARED_INSTALL_INTERFACE_LIBS) + +if(ARROW_WITH_OPENTELEMETRY) + list(APPEND + ARROW_ACERO_SHARED_LINK_LIBS + opentelemetry-cpp::trace + opentelemetry-cpp::ostream_span_exporter + opentelemetry-cpp::otlp_http_exporter) + list(APPEND + ARROW_ACERO_STATIC_LINK_LIBS + opentelemetry-cpp::trace + opentelemetry-cpp::ostream_span_exporter + opentelemetry-cpp::otlp_http_exporter) +endif() + +list(APPEND ARROW_ACERO_STATIC_INSTALL_INTERFACE_LIBS Arrow::arrow_static) +list(APPEND ARROW_ACERO_SHARED_INSTALL_INTERFACE_LIBS Arrow::arrow_shared) +list(APPEND ARROW_ACERO_STATIC_LINK_LIBS arrow_static ${ARROW_STATIC_LINK_LIBS}) +list(APPEND ARROW_ACERO_SHARED_LINK_LIBS arrow_shared) + +add_arrow_lib(arrow_acero + CMAKE_PACKAGE_NAME + ArrowAcero + PKG_CONFIG_NAME + arrow-acero + OUTPUTS + ARROW_ACERO_LIBRARIES + SOURCES + ${ARROW_ACERO_SRCS} + PRECOMPILED_HEADERS + "$<$:arrow/acero/pch.h>" + DEPENDENCIES + toolchain + SHARED_LINK_LIBS + ${ARROW_ACERO_SHARED_LINK_LIBS} + SHARED_INSTALL_INTERFACE_LIBS + ${ARROW_ACERO_SHARED_INSTALL_INTERFACE_LIBS} + STATIC_LINK_LIBS + ${ARROW_ACERO_STATIC_LINK_LIBS} + STATIC_INSTALL_INTERFACE_LIBS + ${ARROW_ACERO_STATIC_INSTALL_INTERFACE_LIBS}) + +if(ARROW_BUILD_STATIC AND WIN32) + target_compile_definitions(arrow_acero_static PUBLIC ARROW_ACERO_STATIC) +endif() + +if(ARROW_TEST_LINKAGE STREQUAL "static") + set(ARROW_ACERO_TEST_LINK_LIBS arrow_acero_static ${ARROW_TEST_STATIC_LINK_LIBS}) +else() + set(ARROW_ACERO_TEST_LINK_LIBS arrow_acero_shared ${ARROW_TEST_SHARED_LINK_LIBS}) +endif() + +foreach(LIB_TARGET ${ARROW_ACERO_LIBRARIES}) + target_compile_definitions(${LIB_TARGET} PRIVATE ARROW_ACERO_EXPORTING) +endforeach() + +# Adding unit tests part of the "dataset" portion of the test suite +function(add_arrow_acero_test REL_TEST_NAME) + set(options) + set(one_value_args PREFIX) + set(multi_value_args LABELS) + cmake_parse_arguments(ARG + "${options}" + "${one_value_args}" + "${multi_value_args}" + ${ARGN}) + + if(ARG_PREFIX) + set(PREFIX ${ARG_PREFIX}) + else() + set(PREFIX "arrow-acero") + endif() + + if(ARG_LABELS) + set(LABELS ${ARG_LABELS}) + else() + set(LABELS "arrow_acero") + endif() + + add_arrow_test(${REL_TEST_NAME} + EXTRA_LINK_LIBS + ${ARROW_ACERO_TEST_LINK_LIBS} + PREFIX + ${PREFIX} + SOURCES + test_util_internal.cc + LABELS + ${LABELS} + ${ARG_UNPARSED_ARGUMENTS}) +endfunction() + +add_arrow_acero_test(subtree_test SOURCES subtree_test.cc) +add_arrow_acero_test(plan_test + SOURCES + plan_test.cc + test_nodes_test.cc + test_nodes.cc) +add_arrow_acero_test(fetch_node_test SOURCES fetch_node_test.cc test_nodes.cc) +add_arrow_acero_test(order_by_node_test SOURCES order_by_node_test.cc test_nodes.cc) +add_arrow_acero_test(hash_join_node_test SOURCES hash_join_node_test.cc + bloom_filter_test.cc) +add_arrow_acero_test(pivot_longer_node_test SOURCES pivot_longer_node_test.cc + test_nodes.cc) +add_arrow_acero_test(asof_join_node_test SOURCES asof_join_node_test.cc test_nodes.cc) +add_arrow_acero_test(tpch_node_test SOURCES tpch_node_test.cc) +add_arrow_acero_test(union_node_test SOURCES union_node_test.cc) +add_arrow_acero_test(groupby_test SOURCES groupby_test.cc) +add_arrow_acero_test(util_test SOURCES util_test.cc task_util_test.cc) +#add_arrow_acero_test(hash_aggregate_test SOURCES hash_aggregate_test.cc) + +if(ARROW_BUILD_BENCHMARKS) + function(add_arrow_acero_benchmark REL_BENCHMARK_NAME) + set(options) + set(one_value_args PREFIX) + set(multi_value_args LABELS) + cmake_parse_arguments(ARG + "${options}" + "${one_value_args}" + "${multi_value_args}" + ${ARGN}) + + if(ARG_PREFIX) + set(PREFIX ${ARG_PREFIX}) + else() + set(PREFIX "arrow-acero") + endif() + + if(ARG_LABELS) + set(LABELS ${ARG_LABELS}) + else() + set(LABELS "arrow_acero") + endif() + + add_arrow_benchmark(${REL_BENCHMARK_NAME} + EXTRA_LINK_LIBS + ${ARROW_ACERO_TEST_LINK_LIBS} + PREFIX + ${PREFIX} + SOURCES + test_util_internal.cc + LABELS + ${LABELS} + ${ARG_UNPARSED_ARGUMENTS}) + endfunction() + + add_arrow_acero_benchmark(expression_benchmark SOURCES expression_benchmark.cc) + + add_arrow_acero_benchmark(filter_benchmark SOURCES benchmark_util.cc + filter_benchmark.cc) + + add_arrow_acero_benchmark(project_benchmark SOURCES benchmark_util.cc + project_benchmark.cc) + + add_arrow_acero_benchmark(asof_join_benchmark SOURCES asof_join_benchmark.cc) + + add_arrow_acero_benchmark(tpch_benchmark SOURCES tpch_benchmark.cc) + + add_arrow_acero_benchmark(aggregate_benchmark SOURCES aggregate_benchmark.cc) + + if(ARROW_BUILD_OPENMP_BENCHMARKS) + find_package(OpenMP REQUIRED) + add_arrow_acero_benchmark(hash_join_benchmark + EXTRA_LINK_LIBS + OpenMP::OpenMP_CXX + SOURCES + hash_join_benchmark.cc) + if(MSVC) + target_compile_options(arrow-compute-hash-join-benchmark + PRIVATE "-openmp:experimental -openmp:llvm") + endif() + endif() + + # if(ARROW_BUILD_STATIC) + # target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_static) + # target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_static) + # target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_static) + # target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_static) + # target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_static) + # if(ARROW_BUILD_OPENMP_BENCHMARKS) + # target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_static) + # endif() + # else() + # target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_shared) + # target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_shared) + # target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_shared) + # target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_shared) + # target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_shared) + # if(ARROW_BUILD_OPENMP_BENCHMARKS) + # target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_shared) + # endif() + # endif() +endif() diff --git a/cpp/src/arrow/compute/exec/accumulation_queue.cc b/cpp/src/arrow/acero/accumulation_queue.cc similarity index 98% rename from cpp/src/arrow/compute/exec/accumulation_queue.cc rename to cpp/src/arrow/acero/accumulation_queue.cc index f8e0c74f653..0d17dc2dbda 100644 --- a/cpp/src/arrow/compute/exec/accumulation_queue.cc +++ b/cpp/src/arrow/acero/accumulation_queue.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/accumulation_queue.h" +#include "arrow/acero/accumulation_queue.h" #include #include @@ -26,6 +26,7 @@ #include "arrow/util/logging.h" namespace arrow { +namespace acero { namespace util { using arrow::compute::ExecBatch; AccumulationQueue::AccumulationQueue(AccumulationQueue&& that) { @@ -170,4 +171,5 @@ std::unique_ptr SerialSequencingQueue::Make(Processor* pr } } // namespace util +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/accumulation_queue.h b/cpp/src/arrow/acero/accumulation_queue.h similarity index 99% rename from cpp/src/arrow/compute/exec/accumulation_queue.h rename to cpp/src/arrow/acero/accumulation_queue.h index 5d1b5be5a3a..285790207f9 100644 --- a/cpp/src/arrow/compute/exec/accumulation_queue.h +++ b/cpp/src/arrow/acero/accumulation_queue.h @@ -26,7 +26,9 @@ #include "arrow/result.h" namespace arrow { +namespace acero { namespace util { + using arrow::compute::ExecBatch; /// \brief A container that accumulates batches until they are ready to @@ -154,4 +156,5 @@ class SerialSequencingQueue { }; } // namespace util +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/acero/aggregate_benchmark.cc similarity index 99% rename from cpp/src/arrow/compute/kernels/aggregate_benchmark.cc rename to cpp/src/arrow/acero/aggregate_benchmark.cc index e1736bd9dfa..d65b52fe520 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/acero/aggregate_benchmark.cc @@ -19,9 +19,9 @@ #include +#include "arrow/acero/groupby.h" #include "arrow/array/array_primitive.h" #include "arrow/compute/api.h" -#include "arrow/compute/exec/groupby.h" #include "arrow/table.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/random.h" @@ -31,7 +31,17 @@ #include "arrow/util/string.h" namespace arrow { -namespace compute { + +using compute::Count; +using compute::MinMax; +using compute::Mode; +using compute::QuantileOptions; +using compute::Sum; +using compute::TDigest; +using compute::TDigestOptions; +using compute::VarianceOptions; + +namespace acero { #include #include @@ -824,5 +834,5 @@ BENCHMARK(TDigestKernelDoubleMedian)->Apply(QuantileKernelArgs); BENCHMARK(TDigestKernelDoubleDeciles)->Apply(QuantileKernelArgs); BENCHMARK(TDigestKernelDoubleCentiles)->Apply(QuantileKernelArgs); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/acero/aggregate_node.cc similarity index 98% rename from cpp/src/arrow/compute/exec/aggregate_node.cc rename to cpp/src/arrow/acero/aggregate_node.cc index 62d4ac81d70..23ede24fdff 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/acero/aggregate_node.cc @@ -21,11 +21,11 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/registry.h" #include "arrow/compute/row/grouper.h" @@ -59,7 +59,21 @@ namespace arrow { using internal::checked_cast; -namespace compute { +using compute::ExecSpan; +using compute::ExecValue; +using compute::Function; +using compute::FunctionOptions; +using compute::Grouper; +using compute::HashAggregateKernel; +using compute::Kernel; +using compute::KernelContext; +using compute::KernelInitArgs; +using compute::KernelState; +using compute::RowSegmenter; +using compute::ScalarAggregateKernel; +using compute::Segment; + +namespace acero { namespace { @@ -954,7 +968,7 @@ class GroupByNode : public ExecNode, public TracedNode { } // namespace -namespace internal { +namespace acerointernal { void RegisterAggregateNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory( @@ -972,6 +986,6 @@ void RegisterAggregateNode(ExecFactoryRegistry* registry) { })); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/arrow-acero.pc.in b/cpp/src/arrow/acero/arrow-acero.pc.in new file mode 100644 index 00000000000..d082e179ce7 --- /dev/null +++ b/cpp/src/arrow/acero/arrow-acero.pc.in @@ -0,0 +1,27 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +prefix=@CMAKE_INSTALL_PREFIX@ +includedir=@ARROW_PKG_CONFIG_INCLUDEDIR@ +libdir=@ARROW_PKG_CONFIG_LIBDIR@ + +Name: Apache Arrow Acero Engine +Description: Apache Arrow's Acero Engine. +Version: @ARROW_VERSION@ +Requires: @ARROW_ACERO_PKG_CONFIG_REQUIRES@ +Libs: -L${libdir} -larrow_acero +Cflags.private: -DARROW_ACERO_STATIC diff --git a/cpp/src/arrow/compute/exec/asof_join_benchmark.cc b/cpp/src/arrow/acero/asof_join_benchmark.cc similarity index 96% rename from cpp/src/arrow/compute/exec/asof_join_benchmark.cc rename to cpp/src/arrow/acero/asof_join_benchmark.cc index 7ab4ee7a7ff..9e8f40a415b 100644 --- a/cpp/src/arrow/compute/exec/asof_join_benchmark.cc +++ b/cpp/src/arrow/acero/asof_join_benchmark.cc @@ -19,13 +19,13 @@ #include "benchmark/benchmark.h" -#include "arrow/compute/exec/test_util.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/dataset/file_parquet.h" #include "arrow/table.h" #include "arrow/testing/future_util.h" namespace arrow { -namespace compute { +namespace acero { static const char* kTimeCol = "time"; static const char* kKeyCol = "id"; @@ -77,12 +77,12 @@ static void TableJoinOverhead(benchmark::State& state, state.PauseTiming(); std::vector input_nodes = {Declaration( "table_source", - arrow::compute::TableSourceNodeOptions(left_table_stats.table, batch_size))}; + arrow::acero::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(Declaration( "table_source", - arrow::compute::TableSourceNodeOptions(table_stats.table, batch_size))); + arrow::acero::TableSourceNodeOptions(table_stats.table, batch_size))); } Declaration join_node{factory_name, {input_nodes}, options}; state.ResumeTiming(); @@ -166,5 +166,5 @@ void SetArgs(benchmark::internal::Benchmark* bench) { BENCHMARK(AsOfJoinOverhead)->Apply(SetArgs); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/asof_join_node.cc b/cpp/src/arrow/acero/asof_join_node.cc similarity index 99% rename from cpp/src/arrow/compute/exec/asof_join_node.cc rename to cpp/src/arrow/acero/asof_join_node.cc index 47acc41e889..1cff9159430 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node.cc +++ b/cpp/src/arrow/acero/asof_join_node.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/asof_join_node.h" +#include "arrow/acero/asof_join_node.h" #include #include @@ -27,13 +27,13 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/schema_util.h" +#include "arrow/acero/util.h" #include "arrow/array/builder_binary.h" #include "arrow/array/builder_primitive.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/schema_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/key_hash.h" #include "arrow/compute/light_array.h" #include "arrow/record_batch.h" @@ -49,7 +49,14 @@ namespace arrow { using internal::ToChars; -namespace compute { +using compute::ColumnMetadataFromDataType; +using compute::Hashing64; +using compute::KeyColumnArray; +using compute::KeyColumnMetadata; +using compute::LightContext; +using compute::SortKey; + +namespace acero { template inline typename T::const_iterator std_find(const T& container, const V& val) { @@ -1587,11 +1594,11 @@ AsofJoinNode::AsofJoinNode(ExecPlan* plan, NodeVector inputs, process_(), process_thread_() {} -namespace internal { +namespace acerointernal { void RegisterAsofJoinNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("asofjoin", AsofJoinNode::Make)); } -} // namespace internal +} // namespace acerointernal namespace asofjoin { @@ -1608,5 +1615,5 @@ Result> MakeOutputSchema( } // namespace asofjoin -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/asof_join_node.h b/cpp/src/arrow/acero/asof_join_node.h similarity index 89% rename from cpp/src/arrow/compute/exec/asof_join_node.h rename to cpp/src/arrow/acero/asof_join_node.h index e8a4f62900f..b2ad2edc4a0 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node.h +++ b/cpp/src/arrow/acero/asof_join_node.h @@ -17,13 +17,13 @@ #include +#include "arrow/acero/options.h" +#include "arrow/acero/visibility.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/options.h" #include "arrow/type.h" -#include "arrow/util/visibility.h" namespace arrow { -namespace compute { +namespace acero { namespace asofjoin { using AsofJoinKeys = AsofJoinNodeOptions::Keys; @@ -36,11 +36,11 @@ using AsofJoinKeys = AsofJoinNodeOptions::Keys; /// \param[in] input_schema the schema of each input to the node /// \param[in] input_keys the key of each input to the node /// \param[out] field_output_indices the output index of each field -ARROW_EXPORT Result> MakeOutputSchema( +ARROW_ACERO_EXPORT Result> MakeOutputSchema( const std::vector>& input_schema, const std::vector& input_keys, std::vector* field_output_indices = NULLPTR); } // namespace asofjoin -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/asof_join_node_test.cc b/cpp/src/arrow/acero/asof_join_node_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/asof_join_node_test.cc rename to cpp/src/arrow/acero/asof_join_node_test.cc index b1d99b60f2c..730a82fb097 100644 --- a/cpp/src/arrow/compute/exec/asof_join_node_test.cc +++ b/cpp/src/arrow/acero/asof_join_node_test.cc @@ -24,12 +24,12 @@ #include #include +#include "arrow/acero/options.h" +#include "arrow/acero/test_nodes.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/util.h" #include "arrow/api.h" #include "arrow/compute/api_scalar.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_nodes.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/testing/gtest_util.h" @@ -53,7 +53,13 @@ using testing::UnorderedElementsAreArray; namespace arrow { -namespace compute { + +using compute::Cast; +using compute::Divide; +using compute::Multiply; +using compute::Subtract; + +namespace acero { bool is_temporal_primitive(Type::type type_id) { switch (type_id) { @@ -1282,17 +1288,17 @@ void TestBackpressure(BatchesMaker maker, int num_batches, int batch_size, ASSERT_OK_AND_ASSIGN(auto r0_batches, make_shift(r0_schema, 1)); ASSERT_OK_AND_ASSIGN(auto r1_batches, make_shift(r1_schema, 2)); - compute::Declaration l_src = { + Declaration l_src = { "source", SourceNodeOptions( l_schema, MakeDelayedGen(l_batches, "0:fast", fast_delay, noisy))}; - compute::Declaration r0_src = { + Declaration r0_src = { "source", SourceNodeOptions( r0_schema, MakeDelayedGen(r0_batches, "1:slow", slow_delay, noisy))}; - compute::Declaration r1_src = { + Declaration r1_src = { "source", SourceNodeOptions( r1_schema, MakeDelayedGen(r1_batches, "2:fast", fast_delay, noisy))}; - compute::Declaration asofjoin = { + Declaration asofjoin = { "asofjoin", {l_src, r0_src, r1_src}, GetRepeatedOptions(3, "time", {"key"}, 1000)}; ASSERT_OK_AND_ASSIGN(std::unique_ptr batch_reader, @@ -1331,12 +1337,12 @@ void TestSequencing(BatchesMaker maker, int num_batches, int batch_size) { ASSERT_OK_AND_ASSIGN(auto l_batches, make_shift(l_schema, 0)); ASSERT_OK_AND_ASSIGN(auto r_batches, make_shift(r_schema, 1)); - compute::Declaration l_src = {"source", - SourceNodeOptions(l_schema, l_batches.gen(false, false))}; - compute::Declaration r_src = {"source", - SourceNodeOptions(r_schema, r_batches.gen(false, false))}; + Declaration l_src = {"source", + SourceNodeOptions(l_schema, l_batches.gen(false, false))}; + Declaration r_src = {"source", + SourceNodeOptions(r_schema, r_batches.gen(false, false))}; - compute::Declaration asofjoin = { + Declaration asofjoin = { "asofjoin", {l_src, r_src}, GetRepeatedOptions(2, "time", {"key"}, 1000)}; QueryOptions query_options; @@ -1380,5 +1386,5 @@ TEST(AsofJoinTest, BackpressureWithBatchesGen) { /*fast_delay=*/0.001, /*slow_delay=*/0.01); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/benchmark_util.cc b/cpp/src/arrow/acero/benchmark_util.cc similarity index 72% rename from cpp/src/arrow/compute/exec/benchmark_util.cc rename to cpp/src/arrow/acero/benchmark_util.cc index 17ce54e7b23..d50ac340477 100644 --- a/cpp/src/arrow/compute/exec/benchmark_util.cc +++ b/cpp/src/arrow/acero/benchmark_util.cc @@ -15,20 +15,21 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/benchmark_util.h" +#include "arrow/acero/benchmark_util.h" #include #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/task_util.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/task_util.h" +#include "arrow/acero/util.h" #include "arrow/util/macros.h" namespace arrow { -namespace compute { + +namespace acero { // Generates batches from data, then benchmark rows_per_second and batches_per_second for // an isolated node. We do this by passing in batches through a task scheduler, and @@ -36,31 +37,31 @@ namespace compute { Status BenchmarkIsolatedNodeOverhead( benchmark::State& state, arrow::compute::Expression expr, int32_t num_batches, - int32_t batch_size, arrow::compute::BatchesWithSchema data, std::string factory_name, - arrow::compute::ExecNodeOptions& options, arrow::MemoryPool* pool) { + int32_t batch_size, arrow::acero::BatchesWithSchema data, std::string factory_name, + arrow::acero::ExecNodeOptions& options, arrow::MemoryPool* pool) { for (auto _ : state) { state.PauseTiming(); AsyncGenerator> sink_gen; ExecContext ctx(pool, ::arrow::internal::GetCpuThreadPool()); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - arrow::compute::ExecPlan::Make(ctx)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, + arrow::acero::ExecPlan::Make(ctx)); // 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, + ARROW_ASSIGN_OR_RAISE(arrow::acero::ExecNode * source_node, MakeExecNode("source", plan.get(), {}, - arrow::compute::SourceNodeOptions{ + arrow::acero::SourceNodeOptions{ data.schema, data.gen(/*parallel=*/true, /*slow=*/false)})); - ARROW_ASSIGN_OR_RAISE(arrow::compute::ExecNode * node, + ARROW_ASSIGN_OR_RAISE(arrow::acero::ExecNode * node, MakeExecNode(factory_name, plan.get(), {source_node}, options)); ARROW_RETURN_NOT_OK(MakeExecNode("sink", plan.get(), {node}, - arrow::compute::SinkNodeOptions{&sink_gen})); + arrow::acero::SinkNodeOptions{&sink_gen})); - std::unique_ptr scheduler = - arrow::compute::TaskScheduler::Make(); + std::unique_ptr scheduler = + arrow::acero::TaskScheduler::Make(); std::condition_variable all_tasks_finished_cv; std::mutex mutex; @@ -77,7 +78,7 @@ Status BenchmarkIsolatedNodeOverhead( }); scheduler->RegisterEnd(); - arrow::compute::ThreadIndexer thread_indexer; + arrow::acero::ThreadIndexer thread_indexer; state.ResumeTiming(); arrow::internal::ThreadPool* thread_pool = arrow::internal::GetCpuThreadPool(); @@ -109,27 +110,27 @@ Status BenchmarkIsolatedNodeOverhead( // a source -> node_declarations -> sink sequence. Status BenchmarkNodeOverhead(benchmark::State& state, int32_t num_batches, - int32_t batch_size, arrow::compute::BatchesWithSchema data, - std::vector& node_declarations, + int32_t batch_size, arrow::acero::BatchesWithSchema data, + std::vector& node_declarations, MemoryPool* pool) { ExecContext ctx(pool, ::arrow::internal::GetCpuThreadPool()); for (auto _ : state) { state.PauseTiming(); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, - arrow::compute::ExecPlan::Make(ctx)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr plan, + arrow::acero::ExecPlan::Make(ctx)); AsyncGenerator> sink_gen; - arrow::compute::Declaration source = arrow::compute::Declaration( + arrow::acero::Declaration source = arrow::acero::Declaration( {"source", - arrow::compute::SourceNodeOptions{data.schema, - data.gen(/*parallel=*/true, /*slow=*/false)}, + arrow::acero::SourceNodeOptions{data.schema, + data.gen(/*parallel=*/true, /*slow=*/false)}, "custom_source_label"}); - arrow::compute::Declaration sink = arrow::compute::Declaration( - {"sink", arrow::compute::SinkNodeOptions{&sink_gen}, "custom_sink_label"}); - std::vector sequence = {source}; + arrow::acero::Declaration sink = arrow::acero::Declaration( + {"sink", arrow::acero::SinkNodeOptions{&sink_gen}, "custom_sink_label"}); + std::vector sequence = {source}; sequence.insert(sequence.end(), node_declarations.begin(), node_declarations.end()); sequence.push_back(sink); ARROW_RETURN_NOT_OK( - arrow::compute::Declaration::Sequence(sequence).AddToPlan(plan.get())); + arrow::acero::Declaration::Sequence(sequence).AddToPlan(plan.get())); state.ResumeTiming(); ARROW_RETURN_NOT_OK(StartAndCollect(plan.get(), sink_gen).status()); } @@ -143,5 +144,5 @@ Status BenchmarkNodeOverhead(benchmark::State& state, int32_t num_batches, return Status::OK(); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/benchmark_util.h b/cpp/src/arrow/acero/benchmark_util.h similarity index 76% rename from cpp/src/arrow/compute/exec/benchmark_util.h rename to cpp/src/arrow/acero/benchmark_util.h index 9965b5915d9..0ba8553887c 100644 --- a/cpp/src/arrow/compute/exec/benchmark_util.h +++ b/cpp/src/arrow/acero/benchmark_util.h @@ -23,26 +23,26 @@ #include "benchmark/benchmark.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/test_util.h" namespace arrow { -namespace compute { +namespace acero { Status BenchmarkNodeOverhead(benchmark::State& state, int32_t num_batches, - int32_t batch_size, arrow::compute::BatchesWithSchema data, - std::vector& node_declarations, + int32_t batch_size, arrow::acero::BatchesWithSchema data, + std::vector& node_declarations, arrow::MemoryPool* pool = default_memory_pool()); Status BenchmarkIsolatedNodeOverhead(benchmark::State& state, arrow::compute::Expression expr, int32_t num_batches, int32_t batch_size, - arrow::compute::BatchesWithSchema data, + arrow::acero::BatchesWithSchema data, std::string factory_name, - arrow::compute::ExecNodeOptions& options, + arrow::acero::ExecNodeOptions& options, arrow::MemoryPool* pool = default_memory_pool()); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter.cc b/cpp/src/arrow/acero/bloom_filter.cc similarity index 98% rename from cpp/src/arrow/compute/exec/bloom_filter.cc rename to cpp/src/arrow/acero/bloom_filter.cc index 7b348ff687c..ad5e66ded06 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.cc +++ b/cpp/src/arrow/acero/bloom_filter.cc @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/bloom_filter.h" +#include "arrow/acero/bloom_filter.h" #include -#include "arrow/compute/exec/util.h" // PREFETCH -#include "arrow/util/bit_util.h" // Log2 -#include "arrow/util/bitmap_ops.h" // CountSetBits +#include "arrow/acero/util.h" // PREFETCH +#include "arrow/util/bit_util.h" // Log2 +#include "arrow/util/bitmap_ops.h" // CountSetBits namespace arrow { -namespace compute { +namespace acero { BloomFilterMasks::BloomFilterMasks() { std::seed_seq seed{0, 0, 0, 0, 0, 0, 0, 0}; @@ -439,5 +439,5 @@ std::unique_ptr BloomFilterBuilder::Make( return nullptr; } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter.h b/cpp/src/arrow/acero/bloom_filter.h similarity index 95% rename from cpp/src/arrow/compute/exec/bloom_filter.h rename to cpp/src/arrow/acero/bloom_filter.h index b0227e720d8..b8f7f8cd256 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter.h +++ b/cpp/src/arrow/acero/bloom_filter.h @@ -24,14 +24,14 @@ #include #include #include -#include "arrow/compute/exec/partition_util.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/partition_util.h" +#include "arrow/acero/util.h" #include "arrow/memory_pool.h" #include "arrow/result.h" #include "arrow/status.h" namespace arrow { -namespace compute { +namespace acero { // A set of pre-generated bit masks from a 64-bit word. // @@ -41,7 +41,7 @@ namespace compute { // These bit masks need to look random and need to have a similar fractions of // bits set in order for a Bloom filter to have a low false positives rate. // -struct ARROW_EXPORT BloomFilterMasks { +struct ARROW_ACERO_EXPORT BloomFilterMasks { // Generate all masks as a single bit vector. Each bit offset in this bit // vector corresponds to a single mask. // In each consecutive kBitsPerMask bits, there must be between @@ -51,10 +51,11 @@ struct ARROW_EXPORT BloomFilterMasks { inline uint64_t mask(int bit_offset) { #if ARROW_LITTLE_ENDIAN - return (util::SafeLoadAs(masks_ + bit_offset / 8) >> (bit_offset % 8)) & + return (arrow::util::SafeLoadAs(masks_ + bit_offset / 8) >> + (bit_offset % 8)) & kFullMask; #else - return (BYTESWAP(util::SafeLoadAs(masks_ + bit_offset / 8)) >> + return (BYTESWAP(arrow::util::SafeLoadAs(masks_ + bit_offset / 8)) >> (bit_offset % 8)) & kFullMask; #endif @@ -104,7 +105,7 @@ struct ARROW_EXPORT BloomFilterMasks { // good balance between the size of the filter, the cost of its building and // querying and the rate of false positives. // -class ARROW_EXPORT BlockedBloomFilter { +class ARROW_ACERO_EXPORT BlockedBloomFilter { friend class BloomFilterBuilder_SingleThreaded; friend class BloomFilterBuilder_Parallel; @@ -254,7 +255,7 @@ enum class BloomFilterBuildStrategy { PARALLEL = 1, }; -class ARROW_EXPORT BloomFilterBuilder { +class ARROW_ACERO_EXPORT BloomFilterBuilder { public: virtual ~BloomFilterBuilder() = default; virtual Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, @@ -269,7 +270,7 @@ class ARROW_EXPORT BloomFilterBuilder { static std::unique_ptr Make(BloomFilterBuildStrategy strategy); }; -class ARROW_EXPORT BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { +class ARROW_ACERO_EXPORT BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, @@ -289,7 +290,7 @@ class ARROW_EXPORT BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder BlockedBloomFilter* build_target_; }; -class ARROW_EXPORT BloomFilterBuilder_Parallel : public BloomFilterBuilder { +class ARROW_ACERO_EXPORT BloomFilterBuilder_Parallel : public BloomFilterBuilder { public: Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool, int64_t num_rows, int64_t num_batches, @@ -320,5 +321,5 @@ class ARROW_EXPORT BloomFilterBuilder_Parallel : public BloomFilterBuilder { PartitionLocks prtn_locks_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc b/cpp/src/arrow/acero/bloom_filter_avx2.cc similarity index 98% rename from cpp/src/arrow/compute/exec/bloom_filter_avx2.cc rename to cpp/src/arrow/acero/bloom_filter_avx2.cc index 40069618023..b6c281276db 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_avx2.cc +++ b/cpp/src/arrow/acero/bloom_filter_avx2.cc @@ -16,11 +16,11 @@ // under the License. #include -#include "arrow/compute/exec/bloom_filter.h" +#include "arrow/acero/bloom_filter.h" #include "arrow/util/bit_util.h" namespace arrow { -namespace compute { +namespace acero { #if defined(ARROW_HAVE_AVX2) @@ -134,5 +134,5 @@ int64_t BlockedBloomFilter::Insert_avx2(int64_t num_rows, const uint64_t* hashes #endif -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/bloom_filter_test.cc b/cpp/src/arrow/acero/bloom_filter_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/bloom_filter_test.cc rename to cpp/src/arrow/acero/bloom_filter_test.cc index 3a79c10be2a..de433ac68c1 100644 --- a/cpp/src/arrow/compute/exec/bloom_filter_test.cc +++ b/cpp/src/arrow/acero/bloom_filter_test.cc @@ -22,16 +22,20 @@ #include #include #include -#include "arrow/compute/exec/bloom_filter.h" -#include "arrow/compute/exec/task_util.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/bloom_filter.h" +#include "arrow/acero/task_util.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/util.h" #include "arrow/compute/key_hash.h" #include "arrow/util/bitmap_ops.h" #include "arrow/util/cpu_info.h" namespace arrow { -namespace compute { + +using compute::Hashing32; +using compute::Hashing64; + +namespace acero { constexpr int kBatchSizeMax = 32 * 1024; Status BuildBloomFilter_Serial( @@ -522,5 +526,5 @@ TEST(BloomFilter, Scaling) { } #endif -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_1.jpg b/cpp/src/arrow/acero/doc/img/key_map_1.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_1.jpg rename to cpp/src/arrow/acero/doc/img/key_map_1.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_10.jpg b/cpp/src/arrow/acero/doc/img/key_map_10.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_10.jpg rename to cpp/src/arrow/acero/doc/img/key_map_10.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_11.jpg b/cpp/src/arrow/acero/doc/img/key_map_11.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_11.jpg rename to cpp/src/arrow/acero/doc/img/key_map_11.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_2.jpg b/cpp/src/arrow/acero/doc/img/key_map_2.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_2.jpg rename to cpp/src/arrow/acero/doc/img/key_map_2.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_3.jpg b/cpp/src/arrow/acero/doc/img/key_map_3.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_3.jpg rename to cpp/src/arrow/acero/doc/img/key_map_3.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_4.jpg b/cpp/src/arrow/acero/doc/img/key_map_4.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_4.jpg rename to cpp/src/arrow/acero/doc/img/key_map_4.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_5.jpg b/cpp/src/arrow/acero/doc/img/key_map_5.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_5.jpg rename to cpp/src/arrow/acero/doc/img/key_map_5.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_6.jpg b/cpp/src/arrow/acero/doc/img/key_map_6.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_6.jpg rename to cpp/src/arrow/acero/doc/img/key_map_6.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_7.jpg b/cpp/src/arrow/acero/doc/img/key_map_7.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_7.jpg rename to cpp/src/arrow/acero/doc/img/key_map_7.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_8.jpg b/cpp/src/arrow/acero/doc/img/key_map_8.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_8.jpg rename to cpp/src/arrow/acero/doc/img/key_map_8.jpg diff --git a/cpp/src/arrow/compute/exec/doc/img/key_map_9.jpg b/cpp/src/arrow/acero/doc/img/key_map_9.jpg similarity index 100% rename from cpp/src/arrow/compute/exec/doc/img/key_map_9.jpg rename to cpp/src/arrow/acero/doc/img/key_map_9.jpg diff --git a/cpp/src/arrow/compute/exec/doc/key_map.md b/cpp/src/arrow/acero/doc/key_map.md similarity index 100% rename from cpp/src/arrow/compute/exec/doc/key_map.md rename to cpp/src/arrow/acero/doc/key_map.md diff --git a/cpp/src/arrow/compute/exec/exec_plan.cc b/cpp/src/arrow/acero/exec_plan.cc similarity index 98% rename from cpp/src/arrow/compute/exec/exec_plan.cc rename to cpp/src/arrow/acero/exec_plan.cc index 5a41fd925d1..59cc32d000b 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.cc +++ b/cpp/src/arrow/acero/exec_plan.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/exec_plan.h" #include #include @@ -23,11 +23,11 @@ #include #include +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/task_util.h" +#include "arrow/acero/util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/task_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/expression.h" #include "arrow/compute/registry.h" #include "arrow/datum.h" @@ -50,7 +50,7 @@ using internal::checked_cast; using internal::ThreadPool; using internal::ToChars; -namespace compute { +namespace acero { namespace { @@ -1048,7 +1048,7 @@ Result> DeclarationToReader( return DeclarationToReader(std::move(declaration), std::move(options)); } -namespace internal { +namespace acerointernal { void RegisterSourceNode(ExecFactoryRegistry*); void RegisterFetchNode(ExecFactoryRegistry*); @@ -1062,23 +1062,23 @@ void RegisterSinkNode(ExecFactoryRegistry*); void RegisterHashJoinNode(ExecFactoryRegistry*); void RegisterAsofJoinNode(ExecFactoryRegistry*); -} // namespace internal +} // namespace acerointernal ExecFactoryRegistry* default_exec_factory_registry() { class DefaultRegistry : public ExecFactoryRegistry { public: DefaultRegistry() { - internal::RegisterSourceNode(this); - internal::RegisterFetchNode(this); - internal::RegisterFilterNode(this); - internal::RegisterOrderByNode(this); - internal::RegisterPivotLongerNode(this); - internal::RegisterProjectNode(this); - internal::RegisterUnionNode(this); - internal::RegisterAggregateNode(this); - internal::RegisterSinkNode(this); - internal::RegisterHashJoinNode(this); - internal::RegisterAsofJoinNode(this); + acerointernal::RegisterSourceNode(this); + acerointernal::RegisterFetchNode(this); + acerointernal::RegisterFilterNode(this); + acerointernal::RegisterOrderByNode(this); + acerointernal::RegisterPivotLongerNode(this); + acerointernal::RegisterProjectNode(this); + acerointernal::RegisterUnionNode(this); + acerointernal::RegisterAggregateNode(this); + acerointernal::RegisterSinkNode(this); + acerointernal::RegisterHashJoinNode(this); + acerointernal::RegisterAsofJoinNode(this); } Result GetFactory(const std::string& factory_name) override { @@ -1122,5 +1122,5 @@ Result>()>> MakeReaderGenerator( return MakeBackgroundGenerator(std::move(batch_it), io_executor, max_q, q_restart); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/acero/exec_plan.h similarity index 91% rename from cpp/src/arrow/compute/exec/exec_plan.h rename to cpp/src/arrow/acero/exec_plan.h index b51f38cfa92..33621816938 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/acero/exec_plan.h @@ -26,25 +26,32 @@ #include #include +#include "arrow/acero/type_fwd.h" +#include "arrow/acero/visibility.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/type_fwd.h" #include "arrow/compute/ordering.h" #include "arrow/type_fwd.h" #include "arrow/util/future.h" #include "arrow/util/macros.h" #include "arrow/util/tracing.h" #include "arrow/util/type_fwd.h" -#include "arrow/util/visibility.h" namespace arrow { -namespace compute { +using compute::ExecBatch; +using compute::ExecContext; +using compute::FunctionRegistry; +using compute::GetFunctionRegistry; +using compute::Ordering; +using compute::threaded_exec_context; + +namespace acero { /// \addtogroup execnode-components /// @{ -class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { +class ARROW_ACERO_EXPORT ExecPlan : public std::enable_shared_from_this { public: // This allows operators to rely on signed 16-bit indices static const uint32_t kMaxBatchSize = 1 << 15; @@ -111,7 +118,7 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this { std::string ToString() const; }; -class ARROW_EXPORT ExecNode { +class ARROW_ACERO_EXPORT ExecNode { public: using NodeVector = std::vector; @@ -339,7 +346,7 @@ class ARROW_EXPORT ExecNode { }; /// \brief An extensible registry for factories of ExecNodes -class ARROW_EXPORT ExecFactoryRegistry { +class ARROW_ACERO_EXPORT ExecFactoryRegistry { public: using Factory = std::function(ExecPlan*, std::vector, const ExecNodeOptions&)>; @@ -358,7 +365,7 @@ class ARROW_EXPORT ExecFactoryRegistry { }; /// The default registry, which includes built-in factories. -ARROW_EXPORT +ARROW_ACERO_EXPORT ExecFactoryRegistry* default_exec_factory_registry(); /// \brief Construct an ExecNode using the named factory @@ -375,7 +382,7 @@ inline Result MakeExecNode( /// A Declaration represents an unconstructed ExecNode (and potentially more since its /// inputs may also be Declarations). The node can be constructed and added to a plan /// with Declaration::AddToPlan, which will recursively construct any inputs as necessary. -struct ARROW_EXPORT Declaration { +struct ARROW_ACERO_EXPORT Declaration { using Input = std::variant; Declaration() {} @@ -452,7 +459,7 @@ struct ARROW_EXPORT Declaration { std::string label; }; -struct ARROW_EXPORT QueryOptions { +struct ARROW_ACERO_EXPORT QueryOptions { /// \brief Should the plan use a legacy batching strategy /// /// This is currently in place only to support the Scanner::ToTable @@ -522,7 +529,7 @@ struct ARROW_EXPORT QueryOptions { /// then the default function registry will be used. /// /// \return the schema that batches would have after going through the execution plan -ARROW_EXPORT Result> DeclarationToSchema( +ARROW_ACERO_EXPORT Result> DeclarationToSchema( const Declaration& declaration, FunctionRegistry* function_registry = NULLPTR); /// \brief Create a string representation of a plan @@ -539,7 +546,7 @@ ARROW_EXPORT Result> DeclarationToSchema( /// then the default function registry will be used. /// /// \return a string representation of the plan suitable for debugging output -ARROW_EXPORT Result DeclarationToString( +ARROW_ACERO_EXPORT Result DeclarationToString( const Declaration& declaration, FunctionRegistry* function_registry = NULLPTR); /// \brief Utility method to run a declaration and collect the results into a table @@ -556,12 +563,12 @@ ARROW_EXPORT Result DeclarationToString( /// 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( +ARROW_ACERO_EXPORT Result> DeclarationToTable( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); -ARROW_EXPORT Result> DeclarationToTable( +ARROW_ACERO_EXPORT Result> DeclarationToTable( Declaration declaration, QueryOptions query_options); /// \brief Asynchronous version of \see DeclarationToTable @@ -575,7 +582,7 @@ ARROW_EXPORT Result> DeclarationToTable( /// \param memory_pool The memory pool to use for allocations made while running the plan. /// \param function_registry The function registry to use for function execution. If null /// then the default function registry will be used. -ARROW_EXPORT Future> DeclarationToTableAsync( +ARROW_ACERO_EXPORT Future> DeclarationToTableAsync( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); @@ -584,7 +591,7 @@ ARROW_EXPORT Future> DeclarationToTableAsync( /// /// The executor must be specified (cannot be null) and must be kept alive until the /// returned future finishes. -ARROW_EXPORT Future> DeclarationToTableAsync( +ARROW_ACERO_EXPORT Future> DeclarationToTableAsync( Declaration declaration, ExecContext custom_exec_context); /// \brief a collection of exec batches with a common schema @@ -597,18 +604,18 @@ struct BatchesWithCommonSchema { /// vector /// /// \see DeclarationToTable for details on threading & execution -ARROW_EXPORT Result DeclarationToExecBatches( +ARROW_ACERO_EXPORT Result DeclarationToExecBatches( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); -ARROW_EXPORT Result DeclarationToExecBatches( +ARROW_ACERO_EXPORT Result DeclarationToExecBatches( Declaration declaration, QueryOptions query_options); /// \brief Asynchronous version of \see DeclarationToExecBatches /// /// \see DeclarationToTableAsync for details on threading & execution -ARROW_EXPORT Future DeclarationToExecBatchesAsync( +ARROW_ACERO_EXPORT Future DeclarationToExecBatchesAsync( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); @@ -616,33 +623,33 @@ ARROW_EXPORT Future DeclarationToExecBatchesAsync( /// \brief Overload of \see DeclarationToExecBatchesAsync accepting a custom exec context /// /// \see DeclarationToTableAsync for details on threading & execution -ARROW_EXPORT Future DeclarationToExecBatchesAsync( +ARROW_ACERO_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 on threading & execution -ARROW_EXPORT Result>> DeclarationToBatches( +ARROW_ACERO_EXPORT Result>> DeclarationToBatches( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); -ARROW_EXPORT Result>> DeclarationToBatches( +ARROW_ACERO_EXPORT Result>> DeclarationToBatches( Declaration declaration, QueryOptions query_options); /// \brief Asynchronous version of \see DeclarationToBatches /// /// \see DeclarationToTableAsync for details on threading & execution -ARROW_EXPORT Future>> DeclarationToBatchesAsync( - Declaration declaration, bool use_threads = true, - MemoryPool* memory_pool = default_memory_pool(), - FunctionRegistry* function_registry = NULLPTR); +ARROW_ACERO_EXPORT Future>> +DeclarationToBatchesAsync(Declaration declaration, bool use_threads = true, + MemoryPool* memory_pool = default_memory_pool(), + FunctionRegistry* function_registry = NULLPTR); /// \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); +ARROW_ACERO_EXPORT Future>> +DeclarationToBatchesAsync(Declaration declaration, ExecContext exec_context); /// \brief Utility method to run a declaration and return results as a RecordBatchReader /// @@ -659,12 +666,12 @@ ARROW_EXPORT Future>> DeclarationToBatc /// fills up. /// /// If a custom exec context is provided then the value of `use_threads` will be ignored. -ARROW_EXPORT Result> DeclarationToReader( +ARROW_ACERO_EXPORT Result> DeclarationToReader( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); -ARROW_EXPORT Result> DeclarationToReader( +ARROW_ACERO_EXPORT Result> DeclarationToReader( Declaration declaration, QueryOptions query_options); /// \brief Utility method to run a declaration and ignore results @@ -673,12 +680,13 @@ ARROW_EXPORT Result> DeclarationToReader( /// 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, - MemoryPool* memory_pool = default_memory_pool(), - FunctionRegistry* function_registry = NULLPTR); +ARROW_ACERO_EXPORT Status +DeclarationToStatus(Declaration declaration, bool use_threads = true, + MemoryPool* memory_pool = default_memory_pool(), + FunctionRegistry* function_registry = NULLPTR); -ARROW_EXPORT Status DeclarationToStatus(Declaration declaration, - QueryOptions query_options); +ARROW_ACERO_EXPORT Status DeclarationToStatus(Declaration declaration, + QueryOptions query_options); /// \brief Asynchronous version of \see DeclarationToStatus /// @@ -686,7 +694,7 @@ ARROW_EXPORT Status DeclarationToStatus(Declaration declaration, /// example, when the plan ends with a write node. /// /// \see DeclarationToTableAsync for details on threading & execution -ARROW_EXPORT Future<> DeclarationToStatusAsync( +ARROW_ACERO_EXPORT Future<> DeclarationToStatusAsync( Declaration declaration, bool use_threads = true, MemoryPool* memory_pool = default_memory_pool(), FunctionRegistry* function_registry = NULLPTR); @@ -694,13 +702,13 @@ ARROW_EXPORT Future<> DeclarationToStatusAsync( /// \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); +ARROW_ACERO_EXPORT Future<> DeclarationToStatusAsync(Declaration declaration, + ExecContext exec_context); /// \brief Wrap an ExecBatch generator in a RecordBatchReader. /// /// The RecordBatchReader does not impose any ordering on emitted batches. -ARROW_EXPORT +ARROW_ACERO_EXPORT std::shared_ptr MakeGeneratorReader( std::shared_ptr, std::function>()>, MemoryPool*); @@ -711,12 +719,12 @@ constexpr int kDefaultBackgroundQRestart = 16; /// \brief Make a generator of RecordBatchReaders /// /// Useful as a source node for an Exec plan -ARROW_EXPORT +ARROW_ACERO_EXPORT Result>()>> MakeReaderGenerator( std::shared_ptr reader, arrow::internal::Executor* io_executor, int max_q = kDefaultBackgroundMaxQ, int q_restart = kDefaultBackgroundQRestart); /// @} -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/expression_benchmark.cc b/cpp/src/arrow/acero/expression_benchmark.cc similarity index 98% rename from cpp/src/arrow/compute/exec/expression_benchmark.cc rename to cpp/src/arrow/acero/expression_benchmark.cc index ca418b3eca0..9799446ed65 100644 --- a/cpp/src/arrow/compute/exec/expression_benchmark.cc +++ b/cpp/src/arrow/acero/expression_benchmark.cc @@ -19,8 +19,8 @@ #include +#include "arrow/acero/test_util_internal.h" #include "arrow/compute/cast.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/compute/expression.h" #include "arrow/dataset/partition.h" #include "arrow/testing/generator.h" @@ -28,7 +28,14 @@ #include "arrow/type.h" namespace arrow { -namespace compute { + +using compute::and_; +using compute::call; +using compute::Expression; +using compute::field_ref; +using compute::literal; + +namespace acero { std::shared_ptr ninety_nine_dict = DictionaryScalar::Make(MakeScalar(0), ArrayFromJSON(int64(), "[99]")); @@ -290,5 +297,5 @@ BENCHMARK_CAPTURE(ExecuteScalarExpressionOverhead, ref_only_expression, ->DenseThreadRange(1, std::thread::hardware_concurrency(), std::thread::hardware_concurrency()) ->UseRealTime(); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/fetch_node.cc b/cpp/src/arrow/acero/fetch_node.cc similarity index 95% rename from cpp/src/arrow/compute/exec/fetch_node.cc rename to cpp/src/arrow/acero/fetch_node.cc index 7ab44a3391f..92a9efde96b 100644 --- a/cpp/src/arrow/compute/exec/fetch_node.cc +++ b/cpp/src/arrow/acero/fetch_node.cc @@ -17,14 +17,14 @@ #include +#include "arrow/acero/accumulation_queue.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/map_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/accumulation_queue.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/map_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/expression.h" #include "arrow/datum.h" #include "arrow/result.h" @@ -37,7 +37,7 @@ namespace arrow { using internal::checked_cast; -namespace compute { +namespace acero { namespace { class FetchCounter { @@ -209,12 +209,12 @@ class FetchNode : public ExecNode, public TracedNode, util::SequencingQueue::Pro } // namespace -namespace internal { +namespace acerointernal { void RegisterFetchNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory(std::string(FetchNodeOptions::kName), FetchNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/fetch_node_test.cc b/cpp/src/arrow/acero/fetch_node_test.cc similarity index 94% rename from cpp/src/arrow/compute/exec/fetch_node_test.cc rename to cpp/src/arrow/acero/fetch_node_test.cc index 53afe5b3d1d..3ffd48038d2 100644 --- a/cpp/src/arrow/compute/exec/fetch_node_test.cc +++ b/cpp/src/arrow/acero/fetch_node_test.cc @@ -19,16 +19,16 @@ #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_nodes.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_nodes.h" #include "arrow/table.h" #include "arrow/testing/generator.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/random.h" namespace arrow { -namespace compute { +namespace acero { static constexpr int kRowsPerBatch = 16; static constexpr int kNumBatches = 32; @@ -85,5 +85,5 @@ TEST(FetchNode, Invalid) { CheckFetchInvalid({10, -1}, "`count` must be non-negative"); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/filter_benchmark.cc b/cpp/src/arrow/acero/filter_benchmark.cc similarity index 94% rename from cpp/src/arrow/compute/exec/filter_benchmark.cc rename to cpp/src/arrow/acero/filter_benchmark.cc index bbeb205ca77..25ba42c4d03 100644 --- a/cpp/src/arrow/compute/exec/filter_benchmark.cc +++ b/cpp/src/arrow/acero/filter_benchmark.cc @@ -19,14 +19,18 @@ #include "benchmark/benchmark.h" -#include "arrow/compute/exec/benchmark_util.h" -#include "arrow/compute/exec/options.h" +#include "arrow/acero/benchmark_util.h" +#include "arrow/acero/options.h" #include "arrow/compute/expression.h" #include "arrow/record_batch.h" #include "arrow/testing/random.h" namespace arrow { -namespace compute { + +using compute::call; +using compute::field_ref; + +namespace acero { constexpr int64_t kTotalBatchSize = 1000000; constexpr auto kSeed = 0x94378165; @@ -73,10 +77,10 @@ static void FilterOverhead(benchmark::State& state, std::vector expr const double bool_true_probability = state.range(2) / 100.0; const int32_t num_batches = kTotalBatchSize / batch_size; - arrow::compute::BatchesWithSchema data = MakeRandomBatchesWithNullProbability( + arrow::acero::BatchesWithSchema data = MakeRandomBatchesWithNullProbability( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size, null_prob, bool_true_probability); - std::vector filter_node_dec; + std::vector filter_node_dec; for (Expression expr : expr_vector) { filter_node_dec.push_back({"filter", FilterNodeOptions(expr)}); } @@ -86,7 +90,7 @@ static void FilterOverhead(benchmark::State& state, std::vector expr static void FilterOverheadIsolated(benchmark::State& state, Expression expr) { const int32_t batch_size = static_cast(state.range(0)); const int32_t num_batches = kTotalBatchSize / batch_size; - arrow::compute::BatchesWithSchema data = MakeRandomBatches( + arrow::acero::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); FilterNodeOptions options = FilterNodeOptions{expr}; ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, expr, num_batches, batch_size, data, @@ -159,5 +163,5 @@ BENCHMARK_CAPTURE(FilterOverhead, not_null_and_is_true_singlepass_benchmark, {is_not_null_and_true_expression}) ->Apply(SetMultiPassArgs); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/filter_node.cc b/cpp/src/arrow/acero/filter_node.cc similarity index 94% rename from cpp/src/arrow/compute/exec/filter_node.cc rename to cpp/src/arrow/acero/filter_node.cc index 4162f899fdb..c5d8b181b5f 100644 --- a/cpp/src/arrow/compute/exec/filter_node.cc +++ b/cpp/src/arrow/acero/filter_node.cc @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/map_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/map_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" #include "arrow/compute/expression.h" #include "arrow/datum.h" #include "arrow/result.h" @@ -33,7 +33,9 @@ namespace arrow { using internal::checked_cast; -namespace compute { +using compute::FilterOptions; + +namespace acero { namespace { class FilterNode : public MapNode { @@ -112,11 +114,11 @@ class FilterNode : public MapNode { }; } // namespace -namespace internal { +namespace acerointernal { void RegisterFilterNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("filter", FilterNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/forest_internal.h b/cpp/src/arrow/acero/forest_internal.h similarity index 98% rename from cpp/src/arrow/compute/exec/forest_internal.h rename to cpp/src/arrow/acero/forest_internal.h index 7b55a0aabf3..0cedd6ab9ee 100644 --- a/cpp/src/arrow/compute/exec/forest_internal.h +++ b/cpp/src/arrow/acero/forest_internal.h @@ -25,7 +25,7 @@ #include "arrow/status.h" namespace arrow { -namespace compute { +namespace acero { /// A Forest is a view of a sorted range which carries an ancestry relation in addition /// to an ordering relation: each element's descendants appear directly after it. @@ -121,5 +121,5 @@ class Forest { std::shared_ptr> descendant_counts_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/groupby.cc b/cpp/src/arrow/acero/groupby.cc similarity index 96% rename from cpp/src/arrow/compute/exec/groupby.cc rename to cpp/src/arrow/acero/groupby.cc index 23993e1f112..a0b220d6537 100644 --- a/cpp/src/arrow/compute/exec/groupby.cc +++ b/cpp/src/arrow/acero/groupby.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/groupby.h" +#include "arrow/acero/groupby.h" #include #include @@ -33,7 +33,7 @@ namespace arrow { -namespace compute { +namespace acero { Result> TableGroupBy(std::shared_ptr table, std::vector aggregates, @@ -57,5 +57,5 @@ Result> BatchGroupBy(std::shared_ptr batch, return DeclarationToTable(std::move(plan), use_threads, memory_pool); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/groupby.h b/cpp/src/arrow/acero/groupby.h similarity index 92% rename from cpp/src/arrow/compute/exec/groupby.h rename to cpp/src/arrow/acero/groupby.h index 5e0fe51b09b..c24990ad69d 100644 --- a/cpp/src/arrow/compute/exec/groupby.h +++ b/cpp/src/arrow/acero/groupby.h @@ -20,17 +20,17 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/visibility.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" #include "arrow/compute/kernel.h" #include "arrow/datum.h" #include "arrow/result.h" -#include "arrow/util/visibility.h" namespace arrow { -namespace compute { +namespace acero { /// Convenience function to perform a group-by on a table /// @@ -46,7 +46,7 @@ namespace compute { /// /// \return a table that will have one column for each aggregate, named after they /// aggregate function, and one column for each key -ARROW_EXPORT +ARROW_ACERO_EXPORT Result> TableGroupBy( std::shared_ptr
table, std::vector aggregates, std::vector keys, bool use_threads = false, @@ -55,11 +55,11 @@ Result> TableGroupBy( /// Convenience function to perform a group-by on a record batch /// /// \see GroupByTable -ARROW_EXPORT +ARROW_ACERO_EXPORT Result> BatchGroupBy( std::shared_ptr record_batch, std::vector aggregates, std::vector keys, bool use_threads = false, MemoryPool* memory_pool = default_memory_pool()); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/groupby_test.cc b/cpp/src/arrow/acero/groupby_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/groupby_test.cc rename to cpp/src/arrow/acero/groupby_test.cc index 0196499d297..1284dbae2bd 100644 --- a/cpp/src/arrow/compute/exec/groupby_test.cc +++ b/cpp/src/arrow/acero/groupby_test.cc @@ -14,7 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/groupby.h" +#include "arrow/acero/groupby.h" #include #include @@ -25,7 +25,7 @@ namespace arrow { -namespace compute { +namespace acero { TEST(GroupByConvenienceFunc, Basic) { std::shared_ptr in_schema = @@ -121,5 +121,5 @@ TEST(GroupByConvenienceFunc, Invalid) { TableGroupBy(in_table, {{"add", {"value"}, "value_add"}}, {})); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/acero/hash_aggregate_test.cc similarity index 99% rename from cpp/src/arrow/compute/kernels/hash_aggregate_test.cc rename to cpp/src/arrow/acero/hash_aggregate_test.cc index fd631e0dc51..226b8a633a3 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/acero/hash_aggregate_test.cc @@ -25,6 +25,9 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/array.h" #include "arrow/array/concatenate.h" #include "arrow/chunked_array.h" @@ -33,13 +36,9 @@ #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/codegen_internal.h" -#include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" #include "arrow/compute/row/grouper.h" #include "arrow/compute/row/grouper_internal.h" @@ -70,7 +69,23 @@ using internal::checked_cast; using internal::checked_pointer_cast; using internal::ToChars; -namespace compute { +using compute::CallFunction; +using compute::CountOptions; +using compute::default_exec_context; +using compute::ExecSpan; +using compute::FunctionOptions; +using compute::Grouper; +using compute::RowSegmenter; +using compute::ScalarAggregateOptions; +using compute::Segment; +using compute::SortIndices; +using compute::SortKey; +using compute::SortOrder; +using compute::Take; +using compute::TDigestOptions; +using compute::VarianceOptions; + +namespace acero { namespace { using GroupByFunction = std::function( @@ -1171,7 +1186,7 @@ class GroupBy : public ::testing::TestWithParam { void ValidateGroupBy(const std::vector& aggregates, std::vector arguments, std::vector keys, bool naive = true) { - compute::ValidateGroupBy(GetParam(), aggregates, arguments, keys, naive); + ValidateGroupBy(GetParam(), aggregates, arguments, keys, naive); } Result GroupByTest(const std::vector& arguments, @@ -1179,15 +1194,15 @@ class GroupBy : public ::testing::TestWithParam { const std::vector& segment_keys, const std::vector& aggregates, bool use_threads) { - return compute::GroupByTest(GetParam(), arguments, keys, segment_keys, aggregates, - use_threads); + return GroupByTest(GetParam(), arguments, keys, segment_keys, aggregates, + use_threads); } Result GroupByTest(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, bool use_threads) { - return compute::GroupByTest(GetParam(), arguments, keys, aggregates, use_threads); + return GroupByTest(GetParam(), arguments, keys, aggregates, use_threads); } Result AltGroupBy(const std::vector& arguments, @@ -1201,7 +1216,7 @@ class GroupBy : public ::testing::TestWithParam { void TestSegmentKey(const std::shared_ptr
& table, Datum output, const std::vector& segment_keys) { - return compute::TestSegmentKey(GetParam(), table, output, segment_keys); + return TestSegmentKey(GetParam(), table, output, segment_keys); } }; @@ -4904,5 +4919,5 @@ INSTANTIATE_TEST_SUITE_P(SegmentedScalarGroupBy, SegmentedScalarGroupBy, INSTANTIATE_TEST_SUITE_P(SegmentedKeyGroupBy, SegmentedKeyGroupBy, ::testing::Values(RunSegmentedGroupByImpl)); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/acero/hash_join.cc similarity index 99% rename from cpp/src/arrow/compute/exec/hash_join.cc rename to cpp/src/arrow/acero/hash_join.cc index 7a654a05148..296b2c56e00 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/acero/hash_join.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/hash_join.h" +#include "arrow/acero/hash_join.h" #include #include @@ -25,14 +25,17 @@ #include #include -#include "arrow/compute/exec/hash_join_dict.h" -#include "arrow/compute/exec/task_util.h" +#include "arrow/acero/hash_join_dict.h" +#include "arrow/acero/task_util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/row/encode_internal.h" #include "arrow/util/tracing_internal.h" namespace arrow { -namespace compute { + +using compute::ExecSpan; + +namespace acero { class HashJoinBasicImpl : public HashJoinImpl { private: @@ -49,7 +52,7 @@ class HashJoinBasicImpl : public HashJoinImpl { FinishedCallback finished_callback) override { START_COMPUTE_SPAN(span_, "HashJoinBasicImpl", {{"detail", filter.ToString()}, - {"join.kind", arrow::compute::ToString(join_type)}, + {"join.kind", arrow::acero::ToString(join_type)}, {"join.threads", static_cast(num_threads)}}); num_threads_ = num_threads; @@ -791,5 +794,5 @@ Result> HashJoinImpl::MakeBasic() { return std::move(impl); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join.h b/cpp/src/arrow/acero/hash_join.h similarity index 87% rename from cpp/src/arrow/compute/exec/hash_join.h rename to cpp/src/arrow/acero/hash_join.h index 52c12983d00..a81ff274e5e 100644 --- a/cpp/src/arrow/compute/exec/hash_join.h +++ b/cpp/src/arrow/acero/hash_join.h @@ -21,21 +21,21 @@ #include #include -#include "arrow/compute/exec/accumulation_queue.h" -#include "arrow/compute/exec/bloom_filter.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/schema_util.h" -#include "arrow/compute/exec/task_util.h" +#include "arrow/acero/accumulation_queue.h" +#include "arrow/acero/bloom_filter.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/schema_util.h" +#include "arrow/acero/task_util.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/type.h" #include "arrow/util/tracing.h" namespace arrow { -namespace compute { +namespace acero { -using arrow::util::AccumulationQueue; +using util::AccumulationQueue; class HashJoinImpl { public: @@ -68,8 +68,8 @@ class HashJoinImpl { static Result> MakeSwiss(); protected: - util::tracing::Span span_; + arrow::util::tracing::Span span_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc b/cpp/src/arrow/acero/hash_join_benchmark.cc similarity index 98% rename from cpp/src/arrow/compute/exec/hash_join_benchmark.cc rename to cpp/src/arrow/acero/hash_join_benchmark.cc index 156f55034ca..7f1653a98cb 100644 --- a/cpp/src/arrow/compute/exec/hash_join_benchmark.cc +++ b/cpp/src/arrow/acero/hash_join_benchmark.cc @@ -17,12 +17,12 @@ #include "benchmark/benchmark.h" +#include "arrow/acero/hash_join.h" +#include "arrow/acero/hash_join_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/util.h" #include "arrow/api.h" -#include "arrow/compute/exec/hash_join.h" -#include "arrow/compute/exec/hash_join_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/testing/random.h" #include "arrow/util/thread_pool.h" @@ -34,7 +34,7 @@ #include namespace arrow { -namespace compute { +namespace aceroacero { struct BenchmarkSettings { int num_threads = 1; JoinType join_type = JoinType::INNER; @@ -453,5 +453,5 @@ BENCHMARK(BM_HashJoinBasic_ProbeParallelism) #endif // ARROW_BUILD_DETAILED_BENCHMARKS -} // namespace compute +} // namespace aceroacero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_dict.cc b/cpp/src/arrow/acero/hash_join_dict.cc similarity index 98% rename from cpp/src/arrow/compute/exec/hash_join_dict.cc rename to cpp/src/arrow/acero/hash_join_dict.cc index 4ce89446d3c..3aef08e6e9c 100644 --- a/cpp/src/arrow/compute/exec/hash_join_dict.cc +++ b/cpp/src/arrow/acero/hash_join_dict.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/hash_join_dict.h" +#include "arrow/acero/hash_join_dict.h" #include #include @@ -28,7 +28,12 @@ #include "arrow/util/checked_cast.h" namespace arrow { -namespace compute { + +using compute::ExecSpan; +using compute::internal::KeyEncoder; +using compute::internal::RowEncoder; + +namespace acero { bool HashJoinDictUtil::KeyDataTypesValid( const std::shared_ptr& probe_data_type, @@ -223,7 +228,7 @@ Status HashJoinDictBuild::Init(ExecContext* ctx, std::shared_ptr dictiona dictionary_ = dictionary; // Initialize encoder - internal::RowEncoder encoder; + RowEncoder encoder; std::vector encoder_types; encoder_types.emplace_back(value_type_); encoder.Init(encoder_types, ctx); @@ -254,7 +259,7 @@ Status HashJoinDictBuild::Init(ExecContext* ctx, std::shared_ptr dictiona // Null values will always be represented as null not an id pointing to a // dictionary entry for null. // - if (internal::KeyEncoder::IsNull(reinterpret_cast(str.data()))) { + if (KeyEncoder::IsNull(reinterpret_cast(str.data()))) { ids[i] = HashJoinDictUtil::kNullId; bit_util::ClearBit(non_nulls, i); continue; @@ -284,7 +289,7 @@ Result> HashJoinDictBuild::RemapInputValues( ExecContext* ctx, const Datum& values, int64_t batch_length) const { // Initialize encoder // - internal::RowEncoder encoder; + RowEncoder encoder; std::vector encoder_types = {value_type_}; encoder.Init(encoder_types, ctx); @@ -311,7 +316,7 @@ Result> HashJoinDictBuild::RemapInputValues( // for (int64_t i = 0; i < encoded_length; ++i) { std::string str = encoder.encoded_row(static_cast(i)); - if (internal::KeyEncoder::IsNull(reinterpret_cast(str.data()))) { + if (KeyEncoder::IsNull(reinterpret_cast(str.data()))) { // Map nulls to nulls bit_util::ClearBit(non_nulls, i); ids[i] = HashJoinDictUtil::kNullId; @@ -445,7 +450,7 @@ Result> HashJoinDictProbe::RemapInput( const uint8_t* non_nulls = row_ids_arr->buffers[0]->data(); for (int64_t i = 0; i < batch_length; ++i) { if (!bit_util::GetBit(non_nulls, i)) { - row_ids[i] = internal::RowEncoder::kRowIdForNulls(); + row_ids[i] = RowEncoder::kRowIdForNulls(); } } @@ -659,5 +664,5 @@ Status HashJoinDictProbeMulti::EncodeBatch( return Status::OK(); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_dict.h b/cpp/src/arrow/acero/hash_join_dict.h similarity index 98% rename from cpp/src/arrow/compute/exec/hash_join_dict.h rename to cpp/src/arrow/acero/hash_join_dict.h index 6c6a4014f42..c7d8d785d07 100644 --- a/cpp/src/arrow/compute/exec/hash_join_dict.h +++ b/cpp/src/arrow/acero/hash_join_dict.h @@ -20,8 +20,8 @@ #include #include +#include "arrow/acero/schema_util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/schema_util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/result.h" #include "arrow/status.h" @@ -58,9 +58,12 @@ // namespace arrow { -namespace compute { -using internal::RowEncoder; +using compute::ExecBatch; +using compute::ExecContext; +using compute::internal::RowEncoder; + +namespace acero { /// Helper class with operations that are stateless and common to processing of dictionary /// keys on both build and probe side. @@ -249,7 +252,7 @@ class HashJoinDictProbe { std::shared_ptr remapped_ids_; // Encoder of key columns that uses unified representation instead of original data type // for key columns that need to use it (have dictionaries on either side of the join). - internal::RowEncoder encoder_; + RowEncoder encoder_; }; // Encapsulates dictionary handling logic for build side of hash join. @@ -311,5 +314,5 @@ class HashJoinDictProbeMulti { std::vector local_states_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_graphs.py b/cpp/src/arrow/acero/hash_join_graphs.py similarity index 100% rename from cpp/src/arrow/compute/exec/hash_join_graphs.py rename to cpp/src/arrow/acero/hash_join_graphs.py diff --git a/cpp/src/arrow/compute/exec/hash_join_node.cc b/cpp/src/arrow/acero/hash_join_node.cc similarity index 98% rename from cpp/src/arrow/compute/exec/hash_join_node.cc rename to cpp/src/arrow/acero/hash_join_node.cc index 6da58330e22..5fc950c0144 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.cc +++ b/cpp/src/arrow/acero/hash_join_node.cc @@ -20,13 +20,13 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/hash_join.h" -#include "arrow/compute/exec/hash_join_dict.h" -#include "arrow/compute/exec/hash_join_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/schema_util.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/hash_join.h" +#include "arrow/acero/hash_join_dict.h" +#include "arrow/acero/hash_join_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/schema_util.h" +#include "arrow/acero/util.h" #include "arrow/compute/key_hash.h" #include "arrow/util/checked_cast.h" #include "arrow/util/future.h" @@ -37,7 +37,12 @@ namespace arrow { using internal::checked_cast; -namespace compute { +using compute::field_ref; +using compute::FilterOptions; +using compute::Hashing32; +using compute::KeyColumnArray; + +namespace acero { // Check if a type is supported in a join (as either a key or non-key column) bool HashJoinSchema::IsTypeSupported(const DataType& type) { @@ -554,7 +559,7 @@ struct BloomFilterPushdownContext { std::vector hashes(batch.length); std::vector bv(bit_vector_bytes); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * stack, ctx_->GetTempStack(thread_index)); // Start with full selection for the current batch @@ -1121,12 +1126,16 @@ Status BloomFilterPushdownContext::BuildBloomFilter_exec_task(size_t thread_inde } ARROW_ASSIGN_OR_RAISE(ExecBatch key_batch, ExecBatch::Make(std::move(key_columns))); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * stack, ctx_->GetTempStack(thread_index)); - util::TempVectorHolder hash_holder(stack, util::MiniBatch::kMiniBatchLength); + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * stack, + ctx_->GetTempStack(thread_index)); + arrow::util::TempVectorHolder hash_holder( + stack, arrow::util::MiniBatch::kMiniBatchLength); uint32_t* hashes = hash_holder.mutable_data(); - for (int64_t i = 0; i < key_batch.length; i += util::MiniBatch::kMiniBatchLength) { - int64_t length = std::min(static_cast(key_batch.length - i), - static_cast(util::MiniBatch::kMiniBatchLength)); + for (int64_t i = 0; i < key_batch.length; + i += arrow::util::MiniBatch::kMiniBatchLength) { + int64_t length = + std::min(static_cast(key_batch.length - i), + static_cast(arrow::util::MiniBatch::kMiniBatchLength)); std::vector temp_column_arrays; RETURN_NOT_OK(Hashing32::HashBatch(key_batch, hashes, temp_column_arrays, @@ -1241,11 +1250,11 @@ std::pair> BloomFilterPushdownContext::GetPushdo #endif // ARROW_LITTLE_ENDIAN } -namespace internal { +namespace acerointernal { void RegisterHashJoinNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("hashjoin", HashJoinNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_node.h b/cpp/src/arrow/acero/hash_join_node.h similarity index 95% rename from cpp/src/arrow/compute/exec/hash_join_node.h rename to cpp/src/arrow/acero/hash_join_node.h index 8dc7ea0b8bf..cca64d59830 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node.h +++ b/cpp/src/arrow/acero/hash_join_node.h @@ -19,15 +19,18 @@ #include -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/schema_util.h" +#include "arrow/acero/options.h" +#include "arrow/acero/schema_util.h" #include "arrow/result.h" #include "arrow/status.h" namespace arrow { -namespace compute { -class ARROW_EXPORT HashJoinSchema { +using compute::ExecContext; + +namespace acero { + +class ARROW_ACERO_EXPORT HashJoinSchema { public: Status Init(JoinType join_type, const Schema& left_schema, const std::vector& left_keys, const Schema& right_schema, @@ -95,5 +98,5 @@ class ARROW_EXPORT HashJoinSchema { const std::vector& key); }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/hash_join_node_test.cc b/cpp/src/arrow/acero/hash_join_node_test.cc similarity index 99% rename from cpp/src/arrow/compute/exec/hash_join_node_test.cc rename to cpp/src/arrow/acero/hash_join_node_test.cc index d8466dc0ea4..960d3838fb8 100644 --- a/cpp/src/arrow/compute/exec/hash_join_node_test.cc +++ b/cpp/src/arrow/acero/hash_join_node_test.cc @@ -22,10 +22,10 @@ #include #include +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/util.h" #include "arrow/api.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/testing/extension_type.h" @@ -38,7 +38,17 @@ using testing::UnorderedElementsAreArray; namespace arrow { -namespace compute { + +using compute::call; +using compute::default_exec_context; +using compute::ExecSpan; +using compute::field_ref; +using compute::SortIndices; +using compute::SortKey; +using compute::Take; +using compute::internal::RowEncoder; + +namespace acero { BatchesWithSchema GenerateBatchesFromString( const std::shared_ptr& schema, @@ -476,7 +486,7 @@ std::vector> GenRandomUniqueRecords( for (size_t i = 0; i < result.size(); ++i) { val_types.push_back(result[i]->type()); } - internal::RowEncoder encoder; + RowEncoder encoder; encoder.Init(val_types, ctx); ExecBatch batch({}, num_desired); batch.values.resize(result.size()); @@ -2114,5 +2124,5 @@ TEST(HashJoin, ChainedIntegerHashJoins) { } } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/map_node.cc b/cpp/src/arrow/acero/map_node.cc similarity index 97% rename from cpp/src/arrow/compute/exec/map_node.cc rename to cpp/src/arrow/acero/map_node.cc index 79abcb88f10..d2dffea72f3 100644 --- a/cpp/src/arrow/compute/exec/map_node.cc +++ b/cpp/src/arrow/acero/map_node.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/map_node.h" +#include "arrow/acero/map_node.h" #include #include @@ -31,7 +31,7 @@ #include "arrow/util/tracing_internal.h" namespace arrow { -namespace compute { +namespace acero { MapNode::MapNode(ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema) @@ -85,5 +85,5 @@ Status MapNode::InputReceived(ExecNode* input, ExecBatch batch) { void MapNode::Finish() {} -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/map_node.h b/cpp/src/arrow/acero/map_node.h similarity index 91% rename from cpp/src/arrow/compute/exec/map_node.h rename to cpp/src/arrow/acero/map_node.h index 96f91f149da..8bdd0ab2ca3 100644 --- a/cpp/src/arrow/compute/exec/map_node.h +++ b/cpp/src/arrow/acero/map_node.h @@ -22,17 +22,17 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/util.h" +#include "arrow/acero/visibility.h" #include "arrow/compute/type_fwd.h" #include "arrow/status.h" #include "arrow/type_fwd.h" #include "arrow/util/cancel.h" #include "arrow/util/type_fwd.h" -#include "arrow/util/visibility.h" namespace arrow { -namespace compute { +namespace acero { /// A utility base class for simple exec nodes with one input /// @@ -41,7 +41,7 @@ namespace compute { /// /// An AtomicCounter is used to keep track of when all data has arrived. When it /// has the Finish() method will be invoked -class ARROW_EXPORT MapNode : public ExecNode, public TracedNode { +class ARROW_ACERO_EXPORT MapNode : public ExecNode, public TracedNode { public: MapNode(ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema); @@ -77,5 +77,5 @@ class ARROW_EXPORT MapNode : public ExecNode, public TracedNode { AtomicCounter input_counter_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/options.cc b/cpp/src/arrow/acero/options.cc similarity index 96% rename from cpp/src/arrow/compute/exec/options.cc rename to cpp/src/arrow/acero/options.cc index 08064af5805..b57ff9ea1a7 100644 --- a/cpp/src/arrow/compute/exec/options.cc +++ b/cpp/src/arrow/acero/options.cc @@ -15,15 +15,15 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/exec_plan.h" #include "arrow/io/util_internal.h" #include "arrow/table.h" #include "arrow/util/async_generator.h" #include "arrow/util/logging.h" namespace arrow { -namespace compute { +namespace acero { constexpr int64_t TableSourceNodeOptions::kDefaultMaxBatchSize; @@ -97,5 +97,5 @@ ExecBatchSourceNodeOptions::ExecBatchSourceNodeOptions(std::shared_ptr s : SchemaSourceNodeOptions(std::move(schema), VecToItMaker(std::move(batches)), requires_io) {} -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/acero/options.h similarity index 93% rename from cpp/src/arrow/compute/exec/options.h rename to cpp/src/arrow/acero/options.h index 5faebd21efd..635f8a1a97b 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/acero/options.h @@ -23,32 +23,40 @@ #include #include +#include "arrow/acero/type_fwd.h" +#include "arrow/acero/visibility.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/type_fwd.h" #include "arrow/compute/expression.h" #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/util/async_generator.h" #include "arrow/util/async_util.h" -#include "arrow/util/visibility.h" namespace arrow { +using compute::Aggregate; +using compute::ExecBatch; +using compute::Expression; +using compute::literal; +using compute::Ordering; +using compute::SelectKOptions; +using compute::SortOptions; + namespace internal { class Executor; } // namespace internal -namespace compute { +namespace acero { using AsyncExecBatchGenerator = AsyncGenerator>; /// \addtogroup execnode-options /// @{ -class ARROW_EXPORT ExecNodeOptions { +class ARROW_ACERO_EXPORT ExecNodeOptions { public: virtual ~ExecNodeOptions() = default; }; @@ -57,7 +65,7 @@ class ARROW_EXPORT ExecNodeOptions { /// /// plan->exec_context()->executor() will be used to parallelize pushing to /// outputs, if provided. -class ARROW_EXPORT SourceNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT SourceNodeOptions : public ExecNodeOptions { public: SourceNodeOptions(std::shared_ptr output_schema, std::function>()> generator) @@ -75,7 +83,7 @@ class ARROW_EXPORT SourceNodeOptions : public ExecNodeOptions { }; /// \brief An extended Source node which accepts a table -class ARROW_EXPORT TableSourceNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT TableSourceNodeOptions : public ExecNodeOptions { public: static constexpr int64_t kDefaultMaxBatchSize = 1 << 20; TableSourceNodeOptions(std::shared_ptr
table, @@ -96,7 +104,7 @@ class ARROW_EXPORT TableSourceNodeOptions : public ExecNodeOptions { /// the plan is to be consumed. /// /// This node is for serialization purposes only and can never be executed. -class ARROW_EXPORT NamedTableNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT NamedTableNodeOptions : public ExecNodeOptions { public: NamedTableNodeOptions(std::vector names, std::shared_ptr schema) : names(std::move(names)), schema(schema) {} @@ -109,7 +117,7 @@ class ARROW_EXPORT NamedTableNodeOptions : public ExecNodeOptions { /// /// ItMaker is a maker of an iterator of tabular data. template -class ARROW_EXPORT SchemaSourceNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT SchemaSourceNodeOptions : public ExecNodeOptions { public: SchemaSourceNodeOptions(std::shared_ptr schema, ItMaker it_maker, arrow::internal::Executor* io_executor) @@ -142,7 +150,7 @@ class ARROW_EXPORT SchemaSourceNodeOptions : public ExecNodeOptions { bool requires_io; }; -class ARROW_EXPORT RecordBatchReaderSourceNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT RecordBatchReaderSourceNodeOptions : public ExecNodeOptions { public: RecordBatchReaderSourceNodeOptions(std::shared_ptr reader, arrow::internal::Executor* io_executor = NULLPTR) @@ -159,14 +167,14 @@ class ARROW_EXPORT RecordBatchReaderSourceNodeOptions : public ExecNodeOptions { using ArrayVectorIteratorMaker = std::function>()>; /// \brief An extended Source node which accepts a schema and array-vectors -class ARROW_EXPORT ArrayVectorSourceNodeOptions +class ARROW_ACERO_EXPORT ArrayVectorSourceNodeOptions : public SchemaSourceNodeOptions { using SchemaSourceNodeOptions::SchemaSourceNodeOptions; }; using ExecBatchIteratorMaker = std::function>()>; /// \brief An extended Source node which accepts a schema and exec-batches -class ARROW_EXPORT ExecBatchSourceNodeOptions +class ARROW_ACERO_EXPORT ExecBatchSourceNodeOptions : public SchemaSourceNodeOptions { public: using SchemaSourceNodeOptions::SchemaSourceNodeOptions; @@ -179,7 +187,7 @@ class ARROW_EXPORT ExecBatchSourceNodeOptions using RecordBatchIteratorMaker = std::function>()>; /// \brief An extended Source node which accepts a schema and record-batches -class ARROW_EXPORT RecordBatchSourceNodeOptions +class ARROW_ACERO_EXPORT RecordBatchSourceNodeOptions : public SchemaSourceNodeOptions { using SchemaSourceNodeOptions::SchemaSourceNodeOptions; }; @@ -189,7 +197,7 @@ class ARROW_EXPORT RecordBatchSourceNodeOptions /// filter_expression will be evaluated against each batch which is pushed to /// this node. Any rows for which filter_expression does not evaluate to `true` will be /// excluded in the batch emitted by this node. -class ARROW_EXPORT FilterNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT FilterNodeOptions : public ExecNodeOptions { public: explicit FilterNodeOptions(Expression filter_expression) : filter_expression(std::move(filter_expression)) {} @@ -197,7 +205,7 @@ class ARROW_EXPORT FilterNodeOptions : public ExecNodeOptions { Expression filter_expression; }; -class ARROW_EXPORT FetchNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT FetchNodeOptions : public ExecNodeOptions { public: static constexpr std::string_view kName = "fetch"; FetchNodeOptions(int64_t offset, int64_t count) : offset(offset), count(count) {} @@ -212,7 +220,7 @@ class ARROW_EXPORT FetchNodeOptions : public ExecNodeOptions { /// this node to produce a corresponding output column. /// /// If names are not provided, the string representations of exprs will be used. -class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT ProjectNodeOptions : public ExecNodeOptions { public: explicit ProjectNodeOptions(std::vector expressions, std::vector names = {}) @@ -238,7 +246,7 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { /// described above, applies. /// /// The keys and segment_keys vectors must be disjoint. -class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT AggregateNodeOptions : public ExecNodeOptions { public: explicit AggregateNodeOptions(std::vector aggregates, std::vector keys = {}, @@ -258,7 +266,7 @@ class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { constexpr int32_t kDefaultBackpressureHighBytes = 1 << 30; // 1GiB constexpr int32_t kDefaultBackpressureLowBytes = 1 << 28; // 256MiB -class ARROW_EXPORT BackpressureMonitor { +class ARROW_ACERO_EXPORT BackpressureMonitor { public: virtual ~BackpressureMonitor() = default; virtual uint64_t bytes_in_use() = 0; @@ -266,7 +274,7 @@ class ARROW_EXPORT BackpressureMonitor { }; /// \brief Options to control backpressure behavior -struct ARROW_EXPORT BackpressureOptions { +struct ARROW_ACERO_EXPORT BackpressureOptions { /// \brief Create default options that perform no backpressure BackpressureOptions() : resume_if_below(0), pause_if_above(0) {} /// \brief Create options that will perform backpressure @@ -293,7 +301,7 @@ struct ARROW_EXPORT BackpressureOptions { /// /// Emitted batches will only be ordered if there is a meaningful ordering /// and sequence_output is not set to false. -class ARROW_EXPORT SinkNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT SinkNodeOptions : public ExecNodeOptions { public: explicit SinkNodeOptions(std::function>()>* generator, std::shared_ptr* schema, @@ -349,7 +357,7 @@ class ARROW_EXPORT SinkNodeOptions : public ExecNodeOptions { /// Callers should ensure that they do not call Pause and Resume simultaneously and they /// should sequence things so that a call to Pause() is always followed by an eventual /// call to Resume() -class ARROW_EXPORT BackpressureControl { +class ARROW_ACERO_EXPORT BackpressureControl { public: virtual ~BackpressureControl() = default; /// \brief Ask the input to pause @@ -361,7 +369,7 @@ class ARROW_EXPORT BackpressureControl { virtual void Resume() = 0; }; -class ARROW_EXPORT SinkNodeConsumer { +class ARROW_ACERO_EXPORT SinkNodeConsumer { public: virtual ~SinkNodeConsumer() = default; /// \brief Prepare any consumer state @@ -383,7 +391,7 @@ class ARROW_EXPORT SinkNodeConsumer { }; /// \brief Add a sink node which consumes data within the exec plan run -class ARROW_EXPORT ConsumingSinkNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT ConsumingSinkNodeOptions : public ExecNodeOptions { public: explicit ConsumingSinkNodeOptions(std::shared_ptr consumer, std::vector names = {}, @@ -410,7 +418,7 @@ class ARROW_EXPORT ConsumingSinkNodeOptions : public ExecNodeOptions { /// /// All batches pushed to this node will be accumulated, then sorted, by the given /// fields. Then sorted batches will be forwarded to the generator in sorted order. -class ARROW_EXPORT OrderBySinkNodeOptions : public SinkNodeOptions { +class ARROW_ACERO_EXPORT OrderBySinkNodeOptions : public SinkNodeOptions { public: explicit OrderBySinkNodeOptions( SortOptions sort_options, @@ -426,7 +434,7 @@ class ARROW_EXPORT OrderBySinkNodeOptions : public SinkNodeOptions { /// the new data with an updated batch index. /// /// Larger-than-memory sort is not currently supported. -class ARROW_EXPORT OrderByNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT OrderByNodeOptions : public ExecNodeOptions { public: static constexpr std::string_view kName = "order_by"; explicit OrderByNodeOptions(Ordering ordering) : ordering(std::move(ordering)) {} @@ -456,7 +464,7 @@ enum class JoinKeyCmp { EQ, IS }; /// @{ /// \brief Make a node which implements join operation using hash join strategy. -class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT HashJoinNodeOptions : public ExecNodeOptions { public: static constexpr const char* default_output_suffix_for_left = ""; static constexpr const char* default_output_suffix_for_right = ""; @@ -574,7 +582,7 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions { /// This node takes one left table and any number of right tables, and asof joins them /// together. Batches produced by each input must be ordered by the "on" key. /// This node will output one row for each row in the left table. -class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT AsofJoinNodeOptions : public ExecNodeOptions { public: /// \brief Keys for one input table of the AsofJoin operation /// @@ -621,7 +629,7 @@ class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions { /// /// All batches pushed to this node will be accumulated, then selected, by the given /// fields. Then sorted batches will be forwarded to the generator in sorted order. -class ARROW_EXPORT SelectKSinkNodeOptions : public SinkNodeOptions { +class ARROW_ACERO_EXPORT SelectKSinkNodeOptions : public SinkNodeOptions { public: explicit SelectKSinkNodeOptions( SelectKOptions select_k_options, @@ -636,7 +644,7 @@ class ARROW_EXPORT SelectKSinkNodeOptions : public SinkNodeOptions { /// /// obtains the output of an execution plan to /// a table pointer. -class ARROW_EXPORT TableSinkNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT TableSinkNodeOptions : public ExecNodeOptions { public: explicit TableSinkNodeOptions(std::shared_ptr
* output_table, std::optional sequence_output = std::nullopt) @@ -656,7 +664,7 @@ class ARROW_EXPORT TableSinkNodeOptions : public ExecNodeOptions { std::vector names; }; -struct ARROW_EXPORT PivotLongerRowTemplate { +struct ARROW_ACERO_EXPORT PivotLongerRowTemplate { PivotLongerRowTemplate(std::vector feature_values, std::vector> measurement_values) : feature_values(std::move(feature_values)), @@ -730,7 +738,7 @@ struct ARROW_EXPORT PivotLongerRowTemplate { /// Finishing it off we name our new columns: /// feature_field_names={"a/b","x/y"} /// measurement_field_names={"f1", "f2"} -class ARROW_EXPORT PivotLongerNodeOptions : public ExecNodeOptions { +class ARROW_ACERO_EXPORT PivotLongerNodeOptions : public ExecNodeOptions { public: static constexpr std::string_view kName = "pivot_longer"; /// One or more row templates to create new output rows @@ -746,5 +754,5 @@ class ARROW_EXPORT PivotLongerNodeOptions : public ExecNodeOptions { /// @} -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/order_by_impl.cc b/cpp/src/arrow/acero/order_by_impl.cc similarity index 96% rename from cpp/src/arrow/compute/exec/order_by_impl.cc rename to cpp/src/arrow/acero/order_by_impl.cc index 4afcf884f53..2c624f6ab89 100644 --- a/cpp/src/arrow/compute/exec/order_by_impl.cc +++ b/cpp/src/arrow/acero/order_by_impl.cc @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/order_by_impl.h" +#include "arrow/acero/order_by_impl.h" #include #include #include #include +#include "arrow/acero/options.h" #include "arrow/compute/api_vector.h" -#include "arrow/compute/exec/options.h" #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/status.h" @@ -31,9 +31,12 @@ #include "arrow/util/checked_cast.h" namespace arrow { + using internal::checked_cast; -namespace compute { +using compute::TakeOptions; + +namespace acero { class SortBasicImpl : public OrderByImpl { public: @@ -100,5 +103,5 @@ Result> OrderByImpl::MakeSelectK( return std::move(impl); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/order_by_impl.h b/cpp/src/arrow/acero/order_by_impl.h similarity index 93% rename from cpp/src/arrow/compute/exec/order_by_impl.h rename to cpp/src/arrow/acero/order_by_impl.h index afc92aeddb2..9b5a0f69a69 100644 --- a/cpp/src/arrow/compute/exec/order_by_impl.h +++ b/cpp/src/arrow/acero/order_by_impl.h @@ -21,14 +21,17 @@ #include #include -#include "arrow/compute/exec/options.h" +#include "arrow/acero/options.h" #include "arrow/record_batch.h" #include "arrow/result.h" #include "arrow/status.h" #include "arrow/type.h" namespace arrow { -namespace compute { + +using compute::ExecContext; + +namespace acero { class OrderByImpl { public: @@ -49,5 +52,5 @@ class OrderByImpl { const SelectKOptions& options); }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/order_by_node.cc b/cpp/src/arrow/acero/order_by_node.cc similarity index 95% rename from cpp/src/arrow/compute/exec/order_by_node.cc rename to cpp/src/arrow/acero/order_by_node.cc index beca15fc49e..bfcfccee585 100644 --- a/cpp/src/arrow/compute/exec/order_by_node.cc +++ b/cpp/src/arrow/acero/order_by_node.cc @@ -22,10 +22,10 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/result.h" #include "arrow/table.h" #include "arrow/util/checked_cast.h" @@ -35,7 +35,9 @@ namespace arrow { using internal::checked_cast; -namespace compute { +using compute::TakeOptions; + +namespace acero { namespace { class OrderByNode : public ExecNode, public TracedNode { @@ -154,13 +156,13 @@ class OrderByNode : public ExecNode, public TracedNode { } // namespace -namespace internal { +namespace acerointernal { void RegisterOrderByNode(ExecFactoryRegistry* registry) { DCHECK_OK( registry->AddFactory(std::string(OrderByNodeOptions::kName), OrderByNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/order_by_node_test.cc b/cpp/src/arrow/acero/order_by_node_test.cc similarity index 95% rename from cpp/src/arrow/compute/exec/order_by_node_test.cc rename to cpp/src/arrow/acero/order_by_node_test.cc index 04d1eaa7b20..d77b0f3184f 100644 --- a/cpp/src/arrow/compute/exec/order_by_node_test.cc +++ b/cpp/src/arrow/acero/order_by_node_test.cc @@ -19,9 +19,9 @@ #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_nodes.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_nodes.h" #include "arrow/table.h" #include "arrow/testing/generator.h" #include "arrow/testing/gtest_util.h" @@ -32,7 +32,10 @@ namespace arrow { using internal::checked_pointer_cast; -namespace compute { +using compute::SortKey; +using compute::SortOrder; + +namespace acero { // Sorting is slow, don't use too many rows static constexpr int kRowsPerBatch = 4; @@ -115,5 +118,5 @@ TEST(OrderByNode, Invalid) { "`ordering` must be an explicit non-empty ordering"); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/partition_util.cc b/cpp/src/arrow/acero/partition_util.cc similarity index 96% rename from cpp/src/arrow/compute/exec/partition_util.cc rename to cpp/src/arrow/acero/partition_util.cc index e99007c45a3..a060a4b26b1 100644 --- a/cpp/src/arrow/compute/exec/partition_util.cc +++ b/cpp/src/arrow/acero/partition_util.cc @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/partition_util.h" +#include "arrow/acero/partition_util.h" #include namespace arrow { -namespace compute { +namespace acero { PartitionLocks::PartitionLocks() : num_prtns_(0), locks_(nullptr), rngs_(nullptr) {} @@ -85,5 +85,5 @@ void PartitionLocks::ReleasePartitionLock(int prtn_id) { lock->store(false, std::memory_order_release); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/partition_util.h b/cpp/src/arrow/acero/partition_util.h similarity index 98% rename from cpp/src/arrow/compute/exec/partition_util.h rename to cpp/src/arrow/acero/partition_util.h index b3f302511a7..27cde61d587 100644 --- a/cpp/src/arrow/compute/exec/partition_util.h +++ b/cpp/src/arrow/acero/partition_util.h @@ -22,12 +22,12 @@ #include #include #include +#include "arrow/acero/util.h" #include "arrow/buffer.h" -#include "arrow/compute/exec/util.h" #include "arrow/util/pcg_random.h" namespace arrow { -namespace compute { +namespace acero { class PartitionSort { public: @@ -180,5 +180,5 @@ class PartitionLocks { std::unique_ptr rngs_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/pch.h b/cpp/src/arrow/acero/pch.h new file mode 100644 index 00000000000..ddb4c120f2a --- /dev/null +++ b/cpp/src/arrow/acero/pch.h @@ -0,0 +1,23 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Often-used headers, for precompiling. +// If updating this header, please make sure you check compilation speed +// before checking in. Adding headers which are not used extremely often +// may incur a slowdown, since it makes the precompiled header heavier to load. + +#include "arrow/pch.h" diff --git a/cpp/src/arrow/compute/exec/pivot_longer_node.cc b/cpp/src/arrow/acero/pivot_longer_node.cc similarity index 98% rename from cpp/src/arrow/compute/exec/pivot_longer_node.cc rename to cpp/src/arrow/acero/pivot_longer_node.cc index 9f4e5c41a73..32f08922884 100644 --- a/cpp/src/arrow/compute/exec/pivot_longer_node.cc +++ b/cpp/src/arrow/acero/pivot_longer_node.cc @@ -22,9 +22,9 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" #include "arrow/status.h" #include "arrow/type.h" #include "arrow/util/checked_cast.h" @@ -35,7 +35,7 @@ namespace arrow { using internal::checked_cast; -namespace compute { +namespace acero { namespace { // A row template that's been bound to a schema @@ -270,13 +270,13 @@ class PivotLongerNode : public ExecNode, public TracedNode { } // namespace -namespace internal { +namespace acerointernal { void RegisterPivotLongerNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory(std::string(PivotLongerNodeOptions::kName), PivotLongerNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/pivot_longer_node_test.cc b/cpp/src/arrow/acero/pivot_longer_node_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/pivot_longer_node_test.cc rename to cpp/src/arrow/acero/pivot_longer_node_test.cc index 60ba27d9d07..9c548a2f23f 100644 --- a/cpp/src/arrow/compute/exec/pivot_longer_node_test.cc +++ b/cpp/src/arrow/acero/pivot_longer_node_test.cc @@ -19,8 +19,8 @@ #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/testing/generator.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" @@ -28,7 +28,10 @@ #include "arrow/table.h" namespace arrow { -namespace compute { + +using compute::field_ref; + +namespace acero { constexpr int kNumBatches = 64; constexpr int kRowsPerBatch = 64; @@ -282,5 +285,5 @@ TEST(PivotLongerNode, ExamplesFromTidyr4) { AssertTablesEqual(*expected, *output, /*same_chunk_layout=*/false); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/acero/plan_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/plan_test.cc rename to cpp/src/arrow/acero/plan_test.cc index ff6b5d96e6e..a3ba1946a1a 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/acero/plan_test.cc @@ -20,12 +20,12 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_nodes.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_nodes.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/expression.h" #include "arrow/io/util_internal.h" #include "arrow/record_batch.h" @@ -50,7 +50,16 @@ using testing::UnorderedElementsAreArray; namespace arrow { -namespace compute { +using compute::call; +using compute::CountOptions; +using compute::field_ref; +using compute::ScalarAggregateOptions; +using compute::SortKey; +using compute::SortOrder; +using compute::Take; +using compute::TDigestOptions; + +namespace acero { TEST(ExecPlanConstruction, Empty) { ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make()); @@ -411,7 +420,7 @@ TEST(ExecPlanExecution, SinkNodeBackpressure) { BackpressureOptions backpressure_options(resume_if_below_bytes, pause_if_above_bytes); std::shared_ptr schema_ = schema({field("data", uint32())}); ARROW_EXPECT_OK( - compute::Declaration::Sequence( + acero::Declaration::Sequence( { {"source", SourceNodeOptions(schema_, batch_producer)}, {"sink", SinkNodeOptions{&sink_gen, /*schema=*/nullptr, @@ -555,7 +564,7 @@ TEST(ExecPlanExecution, CustomFieldNames) { std::vector<::arrow::compute::ExecBatch> ebatches = generator->ExecBatches(/*rows_per_batch=*/1, /*num_batches=*/1); Declaration source = - Declaration("exec_batch_source", ::arrow::compute::ExecBatchSourceNodeOptions( + Declaration("exec_batch_source", ::arrow::acero::ExecBatchSourceNodeOptions( generator->Schema(), std::move(ebatches))); QueryOptions opts; opts.field_names = {"y"}; @@ -1024,7 +1033,7 @@ TEST(ExecPlanExecution, ProjectMaintainsOrder) { std::vector<::arrow::compute::ExecBatch> ebatches = generator->ExecBatches(kRowsPerBatch, kNumBatches); Declaration source_node = - Declaration("exec_batch_source", ::arrow::compute::ExecBatchSourceNodeOptions( + Declaration("exec_batch_source", ::arrow::acero::ExecBatchSourceNodeOptions( generator->Schema(), std::move(ebatches))); Declaration plan = @@ -1048,7 +1057,7 @@ TEST(ExecPlanExecution, FilterMaintainsOrder) { std::vector<::arrow::compute::ExecBatch> ebatches = generator->ExecBatches(kRowsPerBatch, kNumBatches); Declaration source_node = - Declaration("exec_batch_source", ::arrow::compute::ExecBatchSourceNodeOptions( + Declaration("exec_batch_source", ::arrow::acero::ExecBatchSourceNodeOptions( generator->Schema(), std::move(ebatches))); Declaration plan = Declaration::Sequence( @@ -1694,5 +1703,5 @@ TEST(ExecPlanExecution, SegmentedAggregationWithBatchCrossingSegment) { {expected}); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/project_benchmark.cc b/cpp/src/arrow/acero/project_benchmark.cc similarity index 89% rename from cpp/src/arrow/compute/exec/project_benchmark.cc rename to cpp/src/arrow/acero/project_benchmark.cc index d289353ca07..bcc07f19cbc 100644 --- a/cpp/src/arrow/compute/exec/project_benchmark.cc +++ b/cpp/src/arrow/acero/project_benchmark.cc @@ -20,12 +20,12 @@ #include "benchmark/benchmark.h" +#include "arrow/acero/benchmark_util.h" +#include "arrow/acero/options.h" +#include "arrow/acero/task_util.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/compute/cast.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/benchmark_util.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/task_util.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/compute/expression.h" #include "arrow/dataset/partition.h" #include "arrow/testing/future_util.h" @@ -34,7 +34,11 @@ #include "arrow/type.h" namespace arrow { -namespace compute { + +using compute::call; +using compute::field_ref; + +namespace acero { static constexpr int64_t kTotalBatchSize = 1000000; @@ -42,9 +46,9 @@ static void ProjectionOverhead(benchmark::State& state, Expression expr) { const int32_t batch_size = static_cast(state.range(0)); const int32_t num_batches = kTotalBatchSize / batch_size; - arrow::compute::BatchesWithSchema data = MakeRandomBatches( + arrow::acero::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); - std::vector project_node_dec = { + std::vector project_node_dec = { {"project", ProjectNodeOptions{{expr}}}}; ASSERT_OK( BenchmarkNodeOverhead(state, num_batches, batch_size, data, project_node_dec)); @@ -54,7 +58,7 @@ static void ProjectionOverheadIsolated(benchmark::State& state, Expression expr) const int32_t batch_size = static_cast(state.range(0)); const int32_t num_batches = kTotalBatchSize / batch_size; - arrow::compute::BatchesWithSchema data = MakeRandomBatches( + arrow::acero::BatchesWithSchema data = MakeRandomBatches( schema({field("i64", int64()), field("bool", boolean())}), num_batches, batch_size); ProjectNodeOptions options = ProjectNodeOptions{{expr}}; ASSERT_OK(BenchmarkIsolatedNodeOverhead(state, expr, num_batches, batch_size, data, @@ -93,5 +97,5 @@ BENCHMARK_CAPTURE(ProjectionOverhead, zero_copy_expression, zero_copy_expression BENCHMARK_CAPTURE(ProjectionOverhead, ref_only_expression, ref_only_expression) ->Apply(SetArgs); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/project_node.cc b/cpp/src/arrow/acero/project_node.cc similarity index 92% rename from cpp/src/arrow/compute/exec/project_node.cc rename to cpp/src/arrow/acero/project_node.cc index 56d3474aa76..321c4434ff3 100644 --- a/cpp/src/arrow/compute/exec/project_node.cc +++ b/cpp/src/arrow/acero/project_node.cc @@ -17,13 +17,13 @@ #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/map_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/map_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/expression.h" #include "arrow/datum.h" #include "arrow/result.h" @@ -34,9 +34,9 @@ namespace arrow { -using internal::checked_cast; +using arrow::internal::checked_cast; -namespace compute { +namespace acero { namespace { class ProjectNode : public MapNode { @@ -117,12 +117,12 @@ class ProjectNode : public MapNode { } // namespace -namespace internal { +namespace acerointernal { void RegisterProjectNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("project", ProjectNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/query_context.cc b/cpp/src/arrow/acero/query_context.cc similarity index 96% rename from cpp/src/arrow/compute/exec/query_context.cc rename to cpp/src/arrow/acero/query_context.cc index 384bf3f3ab5..d1a9a7774e7 100644 --- a/cpp/src/arrow/compute/exec/query_context.cc +++ b/cpp/src/arrow/acero/query_context.cc @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/query_context.h" +#include "arrow/acero/query_context.h" #include "arrow/util/cpu_info.h" #include "arrow/util/io_util.h" namespace arrow { -using internal::CpuInfo; -namespace compute { +using arrow::internal::CpuInfo; +namespace acero { QueryContext::QueryContext(QueryOptions opts, ExecContext exec_context) : options_(opts), @@ -88,5 +88,5 @@ int QueryContext::RegisterTaskGroup(std::function task, Status QueryContext::StartTaskGroup(int task_group_id, int64_t num_tasks) { return task_scheduler_->StartTaskGroup(GetThreadIndex(), task_group_id, num_tasks); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/query_context.h b/cpp/src/arrow/acero/query_context.h similarity index 91% rename from cpp/src/arrow/compute/exec/query_context.h rename to cpp/src/arrow/acero/query_context.h index 453a9a7d797..9ea11679cba 100644 --- a/cpp/src/arrow/compute/exec/query_context.h +++ b/cpp/src/arrow/acero/query_context.h @@ -18,25 +18,27 @@ #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/task_util.h" +#include "arrow/acero/util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/task_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/io/interfaces.h" #include "arrow/util/async_util.h" #include "arrow/util/type_fwd.h" namespace arrow { +using compute::default_exec_context; using io::IOContext; -namespace compute { -class ARROW_EXPORT QueryContext { +namespace acero { + +class ARROW_ACERO_EXPORT QueryContext { public: QueryContext(QueryOptions opts = {}, ExecContext exec_context = *default_exec_context()); - Status Init(size_t max_num_threads, util::AsyncTaskScheduler* scheduler); + Status Init(size_t max_num_threads, arrow::util::AsyncTaskScheduler* scheduler); const ::arrow::internal::CpuInfo* cpu_info() const; int64_t hardware_flags() const; @@ -46,11 +48,11 @@ class ARROW_EXPORT QueryContext { ExecContext* exec_context() { return &exec_context_; } IOContext* io_context() { return &io_context_; } TaskScheduler* scheduler() { return task_scheduler_.get(); } - util::AsyncTaskScheduler* async_scheduler() { return async_scheduler_; } + arrow::util::AsyncTaskScheduler* async_scheduler() { return async_scheduler_; } size_t GetThreadIndex(); size_t max_concurrency() const; - Result GetTempStack(size_t thread_index); + Result GetTempStack(size_t thread_index); /// \brief Start an external task /// @@ -139,17 +141,17 @@ class ARROW_EXPORT QueryContext { ExecContext exec_context_; IOContext io_context_; - util::AsyncTaskScheduler* async_scheduler_ = NULLPTR; + arrow::util::AsyncTaskScheduler* async_scheduler_ = NULLPTR; std::unique_ptr task_scheduler_ = TaskScheduler::Make(); ThreadIndexer thread_indexer_; struct ThreadLocalData { bool is_init = false; - util::TempVectorStack stack; + arrow::util::TempVectorStack stack; }; std::vector tld_; std::atomic in_flight_bytes_to_disk_{0}; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/schema_util.h b/cpp/src/arrow/acero/schema_util.h similarity index 99% rename from cpp/src/arrow/compute/exec/schema_util.h rename to cpp/src/arrow/acero/schema_util.h index f2b14aa5450..6760022feb4 100644 --- a/cpp/src/arrow/compute/exec/schema_util.h +++ b/cpp/src/arrow/acero/schema_util.h @@ -29,7 +29,7 @@ namespace arrow { using internal::checked_cast; -namespace compute { +namespace acero { // Identifiers for all different row schemas that are used in a join // @@ -222,5 +222,5 @@ class SchemaProjectionMaps { using HashJoinProjectionMaps = SchemaProjectionMaps; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/sink_node.cc b/cpp/src/arrow/acero/sink_node.cc similarity index 97% rename from cpp/src/arrow/compute/exec/sink_node.cc rename to cpp/src/arrow/acero/sink_node.cc index f3fe1252cec..4bd55b0b3ea 100644 --- a/cpp/src/arrow/compute/exec/sink_node.cc +++ b/cpp/src/arrow/acero/sink_node.cc @@ -21,14 +21,14 @@ #include #include +#include "arrow/acero/accumulation_queue.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/order_by_impl.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/accumulation_queue.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/order_by_impl.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/expression.h" #include "arrow/datum.h" @@ -47,9 +47,9 @@ using namespace std::string_view_literals; // NOLINT namespace arrow { -using internal::checked_cast; +using arrow::internal::checked_cast; -namespace compute { +namespace acero { namespace { class BackpressureReservoir : public BackpressureMonitor { @@ -423,9 +423,9 @@ class ConsumingSinkNode : public ExecNode, std::atomic backpressure_counter_ = 0; std::unique_ptr sequencer_; }; -static Result MakeTableConsumingSinkNode( - compute::ExecPlan* plan, std::vector inputs, - const compute::ExecNodeOptions& options) { +static Result MakeTableConsumingSinkNode(ExecPlan* plan, + std::vector inputs, + const ExecNodeOptions& options) { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "TableConsumingSinkNode")); const auto& sink_options = checked_cast(options); MemoryPool* pool = plan->query_context()->memory_pool(); @@ -538,7 +538,7 @@ struct OrderBySinkNode final : public SinkNode { } Status Finish() override { - util::tracing::Span span; + arrow::util::tracing::Span span; ARROW_RETURN_NOT_OK(DoFinish()); return SinkNode::Finish(); } @@ -554,7 +554,7 @@ struct OrderBySinkNode final : public SinkNode { } // namespace -namespace internal { +namespace acerointernal { void RegisterSinkNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("select_k_sink", OrderBySinkNode::MakeSelectK)); @@ -564,6 +564,6 @@ void RegisterSinkNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("table_sink", MakeTableConsumingSinkNode)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/source_node.cc b/cpp/src/arrow/acero/source_node.cc similarity index 97% rename from cpp/src/arrow/compute/exec/source_node.cc rename to cpp/src/arrow/acero/source_node.cc index 73eea80196f..24898d686bb 100644 --- a/cpp/src/arrow/compute/exec/source_node.cc +++ b/cpp/src/arrow/acero/source_node.cc @@ -19,11 +19,11 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/expression.h" #include "arrow/datum.h" @@ -46,10 +46,10 @@ using namespace std::string_view_literals; // NOLINT namespace arrow { -using internal::checked_cast; -using internal::MapVector; +using arrow::internal::checked_cast; +using arrow::internal::MapVector; -namespace compute { +namespace acero { namespace { struct SourceNode : ExecNode, public TracedNode { @@ -506,9 +506,8 @@ struct ArrayVectorSourceNode const char ArrayVectorSourceNode::kKindName[] = "ArrayVectorSourceNode"; -Result MakeNamedTableNode(compute::ExecPlan* plan, - std::vector inputs, - const compute::ExecNodeOptions& options) { +Result MakeNamedTableNode(ExecPlan* plan, std::vector inputs, + const ExecNodeOptions& options) { return Status::Invalid( "The named table node is for serialization purposes only and can never be " "converted into an exec plan or executed"); @@ -516,7 +515,7 @@ Result MakeNamedTableNode(compute::ExecPlan* plan, } // namespace -namespace internal { +namespace acerointernal { void RegisterSourceNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("source", SourceNode::Make)); @@ -529,6 +528,6 @@ void RegisterSourceNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("named_table", MakeNamedTableNode)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/subtree_internal.h b/cpp/src/arrow/acero/subtree_internal.h similarity index 98% rename from cpp/src/arrow/compute/exec/subtree_internal.h rename to cpp/src/arrow/acero/subtree_internal.h index ef0f19ac5cc..a6595e77937 100644 --- a/cpp/src/arrow/compute/exec/subtree_internal.h +++ b/cpp/src/arrow/acero/subtree_internal.h @@ -27,7 +27,10 @@ #include "arrow/compute/expression.h" namespace arrow { -namespace compute { + +using compute::Expression; + +namespace acero { // Helper class for efficiently detecting subtrees given expressions. // // Using fragment partition expressions as an example: @@ -174,5 +177,5 @@ inline bool operator==(const SubtreeImpl::Encoded& l, const SubtreeImpl::Encoded return l.index == r.index && l.guarantee == r.guarantee; } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/subtree_test.cc b/cpp/src/arrow/acero/subtree_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/subtree_test.cc rename to cpp/src/arrow/acero/subtree_test.cc index 908af3be7ef..4d526ee0e74 100644 --- a/cpp/src/arrow/compute/exec/subtree_test.cc +++ b/cpp/src/arrow/acero/subtree_test.cc @@ -24,8 +24,8 @@ #include #include -#include "arrow/compute/exec/forest_internal.h" -#include "arrow/compute/exec/subtree_internal.h" +#include "arrow/acero/forest_internal.h" +#include "arrow/acero/subtree_internal.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/string.h" @@ -33,7 +33,10 @@ namespace arrow { using internal::StartsWith; -namespace compute { +using compute::field_ref; +using compute::literal; + +namespace acero { using testing::ContainerEq; @@ -377,5 +380,5 @@ TEST(Subtree, EncodeFragments) { SubtreeImpl::Encoded{std::nullopt, SubtreeImpl::expression_codes({2, 3})}, })); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/swiss_join.cc b/cpp/src/arrow/acero/swiss_join.cc similarity index 95% rename from cpp/src/arrow/compute/exec/swiss_join.cc rename to cpp/src/arrow/acero/swiss_join.cc index 8bf2ee1df47..ed1608e67d1 100644 --- a/cpp/src/arrow/compute/exec/swiss_join.cc +++ b/cpp/src/arrow/acero/swiss_join.cc @@ -20,10 +20,10 @@ #include #include #include +#include "arrow/acero/hash_join.h" +#include "arrow/acero/swiss_join_internal.h" +#include "arrow/acero/util.h" #include "arrow/array/util.h" // MakeArrayFromScalar -#include "arrow/compute/exec/hash_join.h" -#include "arrow/compute/exec/swiss_join_internal.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/key_hash.h" #include "arrow/compute/row/compare_internal.h" @@ -33,7 +33,14 @@ #include "arrow/util/tracing_internal.h" namespace arrow { -namespace compute { + +using compute::ColumnMetadataFromDataType; +using compute::Hashing32; +using compute::KeyCompare; +using compute::LightContext; +using compute::SwissTable; + +namespace acero { int RowArrayAccessor::VarbinaryColumnId(const RowTableMetadata& row_metadata, int column_id) { @@ -232,7 +239,7 @@ void RowArray::Compare(const ExecBatch& batch, int begin_row_id, int end_row_id, int num_selected, const uint16_t* batch_selection_maybe_null, const uint32_t* array_row_ids, uint32_t* out_num_not_equal, uint16_t* out_not_equal_selection, int64_t hardware_flags, - util::TempVectorStack* temp_stack, + arrow::util::TempVectorStack* temp_stack, std::vector& temp_column_arrays, uint8_t* out_match_bitvector_maybe_null) { Status status = ColumnArraysFromExecBatch( @@ -310,7 +317,8 @@ Status RowArray::DecodeSelected(ResizableArrayData* output, int column_id, const uint64_t* src = reinterpret_cast(ptr); for (uint32_t word_id = 0; word_id < bit_util::CeilDiv(num_bytes, sizeof(uint64_t)); ++word_id) { - util::SafeStore(dst + word_id, util::SafeLoad(src + word_id)); + arrow::util::SafeStore(dst + word_id, + arrow::util::SafeLoad(src + word_id)); } }); break; @@ -339,7 +347,8 @@ Status RowArray::DecodeSelected(ResizableArrayData* output, int column_id, const uint64_t* src = reinterpret_cast(ptr); for (uint32_t word_id = 0; word_id < bit_util::CeilDiv(num_bytes, sizeof(uint64_t)); ++word_id) { - util::SafeStore(dst + word_id, util::SafeLoad(src + word_id)); + arrow::util::SafeStore(dst + word_id, + arrow::util::SafeLoad(src + word_id)); } }); } @@ -634,7 +643,7 @@ Status SwissTableMerge::PrepareForMerge(SwissTable* target, // int log_blocks_max = 1; for (size_t i = 0; i < sources.size(); ++i) { - log_blocks_max = std::max(log_blocks_max, sources[i]->log_blocks_); + log_blocks_max = std::max(log_blocks_max, sources[i]->log_blocks()); } int log_blocks = log_num_sources + log_blocks_max; @@ -644,7 +653,7 @@ Status SwissTableMerge::PrepareForMerge(SwissTable* target, // Target will be used in read-only mode and that array is only needed when // resizing table which may occur only after new inserts. // - RETURN_NOT_OK(target->init(sources[0]->hardware_flags_, pool, log_blocks, + RETURN_NOT_OK(target->init(sources[0]->hardware_flags(), pool, log_blocks, /*no_hash_array=*/true)); // Calculate and output the first group id index for each source. @@ -654,9 +663,9 @@ Status SwissTableMerge::PrepareForMerge(SwissTable* target, first_target_group_id->resize(sources.size()); for (size_t i = 0; i < sources.size(); ++i) { (*first_target_group_id)[i] = num_groups; - num_groups += sources[i]->num_inserted_; + num_groups += sources[i]->num_inserted(); } - target->num_inserted_ = num_groups; + target->num_inserted(num_groups); } return Status::OK(); @@ -670,7 +679,7 @@ void SwissTableMerge::MergePartition(SwissTable* target, const SwissTable* sourc // Prepare parameters needed for scanning full slots in source. // int source_group_id_bits = - SwissTable::num_groupid_bits_from_log_blocks(source->log_blocks_); + SwissTable::num_groupid_bits_from_log_blocks(source->log_blocks()); uint64_t source_group_id_mask = ~0ULL >> (64 - source_group_id_bits); int64_t source_block_bytes = source_group_id_bits + 8; ARROW_DCHECK(source_block_bytes % sizeof(uint64_t) == 0); @@ -678,17 +687,17 @@ void SwissTableMerge::MergePartition(SwissTable* target, const SwissTable* sourc // Compute index of the last block in target that corresponds to the given // partition. // - ARROW_DCHECK(num_partition_bits <= target->log_blocks_); + ARROW_DCHECK(num_partition_bits <= target->log_blocks()); int64_t target_max_block_id = - ((partition_id + 1) << (target->log_blocks_ - num_partition_bits)) - 1; + ((partition_id + 1) << (target->log_blocks() - num_partition_bits)) - 1; overflow_group_ids->clear(); overflow_hashes->clear(); // For each source block... - int64_t source_blocks = 1LL << source->log_blocks_; + int64_t source_blocks = 1LL << source->log_blocks(); for (int64_t block_id = 0; block_id < source_blocks; ++block_id) { - uint8_t* block_bytes = source->blocks_ + block_id * source_block_bytes; + uint8_t* block_bytes = source->blocks() + block_id * source_block_bytes; uint64_t block = *reinterpret_cast(block_bytes); // For each non-empty source slot... @@ -702,7 +711,7 @@ void SwissTableMerge::MergePartition(SwissTable* target, const SwissTable* sourc uint64_t group_id = source->extract_group_id(block_bytes, local_slot_id, source_group_id_mask); int64_t global_slot_id = block_id * kSlotsPerBlock + local_slot_id; - uint32_t hash = source->hashes_[global_slot_id]; + uint32_t hash = source->hashes()[global_slot_id]; // Insert partition id into the highest bits of hash, shifting the // remaining hash bits right. // @@ -728,13 +737,13 @@ inline bool SwissTableMerge::InsertNewGroup(SwissTable* target, uint64_t group_i uint32_t hash, int64_t max_block_id) { // Load the first block to visit for this hash // - int64_t block_id = hash >> (SwissTable::bits_hash_ - target->log_blocks_); - int64_t block_id_mask = ((1LL << target->log_blocks_) - 1); + int64_t block_id = hash >> (SwissTable::bits_hash_ - target->log_blocks()); + int64_t block_id_mask = ((1LL << target->log_blocks()) - 1); int num_group_id_bits = - SwissTable::num_groupid_bits_from_log_blocks(target->log_blocks_); + SwissTable::num_groupid_bits_from_log_blocks(target->log_blocks()); int64_t num_block_bytes = num_group_id_bits + sizeof(uint64_t); ARROW_DCHECK(num_block_bytes % sizeof(uint64_t) == 0); - uint8_t* block_bytes = target->blocks_ + block_id * num_block_bytes; + uint8_t* block_bytes = target->blocks() + block_id * num_block_bytes; uint64_t block = *reinterpret_cast(block_bytes); // Search for the first block with empty slots. @@ -743,7 +752,7 @@ inline bool SwissTableMerge::InsertNewGroup(SwissTable* target, uint64_t group_i constexpr uint64_t kHighBitOfEachByte = 0x8080808080808080ULL; while ((block & kHighBitOfEachByte) == 0 && block_id < max_block_id) { block_id = (block_id + 1) & block_id_mask; - block_bytes = target->blocks_ + block_id * num_block_bytes; + block_bytes = target->blocks() + block_id * num_block_bytes; block = *reinterpret_cast(block_bytes); } if ((block & kHighBitOfEachByte) == 0) { @@ -761,7 +770,7 @@ inline bool SwissTableMerge::InsertNewGroup(SwissTable* target, uint64_t group_i void SwissTableMerge::InsertNewGroups(SwissTable* target, const std::vector& group_ids, const std::vector& hashes) { - int64_t num_blocks = 1LL << target->log_blocks_; + int64_t num_blocks = 1LL << target->log_blocks(); for (size_t i = 0; i < group_ids.size(); ++i) { std::ignore = InsertNewGroup(target, group_ids[i], hashes[i], num_blocks); } @@ -769,7 +778,7 @@ void SwissTableMerge::InsertNewGroups(SwissTable* target, SwissTableWithKeys::Input::Input(const ExecBatch* in_batch, int in_batch_start_row, int in_batch_end_row, - util::TempVectorStack* in_temp_stack, + arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays) : batch(in_batch), batch_start_row(in_batch_start_row), @@ -781,7 +790,7 @@ SwissTableWithKeys::Input::Input(const ExecBatch* in_batch, int in_batch_start_r temp_group_ids(nullptr) {} SwissTableWithKeys::Input::Input(const ExecBatch* in_batch, - util::TempVectorStack* in_temp_stack, + arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays) : batch(in_batch), batch_start_row(0), @@ -794,7 +803,7 @@ SwissTableWithKeys::Input::Input(const ExecBatch* in_batch, SwissTableWithKeys::Input::Input(const ExecBatch* in_batch, int in_num_selected, const uint16_t* in_selection, - util::TempVectorStack* in_temp_stack, + arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays, std::vector* in_temp_group_ids) : batch(in_batch), @@ -856,7 +865,7 @@ void SwissTableWithKeys::EqualCallback(int num_keys, const uint16_t* selection_m if (in->selection_maybe_null) { auto selection_to_use_buf = - util::TempVectorHolder(in->temp_stack, num_keys); + arrow::util::TempVectorHolder(in->temp_stack, num_keys); ARROW_DCHECK(in->temp_group_ids); in->temp_group_ids->resize(in->batch->length); @@ -879,7 +888,8 @@ void SwissTableWithKeys::EqualCallback(int num_keys, const uint16_t* selection_m batch_end_to_use = static_cast(in->batch->length); group_ids_to_use = in->temp_group_ids->data(); - auto match_bitvector_buf = util::TempVectorHolder(in->temp_stack, num_keys); + auto match_bitvector_buf = + arrow::util::TempVectorHolder(in->temp_stack, num_keys); uint8_t* match_bitvector = match_bitvector_buf.mutable_data(); keys_.Compare(*in->batch, batch_start_to_use, batch_end_to_use, num_keys, @@ -888,14 +898,14 @@ void SwissTableWithKeys::EqualCallback(int num_keys, const uint16_t* selection_m if (selection_maybe_null) { int num_keys_mismatch = 0; - util::bit_util::bits_filter_indexes(0, hardware_flags, num_keys, match_bitvector, - selection_maybe_null, &num_keys_mismatch, - out_selection_mismatch); + arrow::util::bit_util::bits_filter_indexes( + 0, hardware_flags, num_keys, match_bitvector, selection_maybe_null, + &num_keys_mismatch, out_selection_mismatch); *out_num_keys_mismatch = num_keys_mismatch; } else { int num_keys_mismatch = 0; - util::bit_util::bits_to_indexes(0, hardware_flags, num_keys, match_bitvector, - &num_keys_mismatch, out_selection_mismatch); + arrow::util::bit_util::bits_to_indexes(0, hardware_flags, num_keys, match_bitvector, + &num_keys_mismatch, out_selection_mismatch); *out_num_keys_mismatch = num_keys_mismatch; } @@ -924,7 +934,7 @@ Status SwissTableWithKeys::AppendCallback(int num_keys, const uint16_t* selectio if (in->selection_maybe_null) { auto selection_to_use_buf = - util::TempVectorHolder(in->temp_stack, num_keys); + arrow::util::TempVectorHolder(in->temp_stack, num_keys); for (int i = 0; i < num_keys; ++i) { selection_to_use_buf.mutable_data()[i] = in->selection_maybe_null[selection[i]]; } @@ -982,7 +992,7 @@ Status SwissTableWithKeys::MapWithInserts(Input* input, const uint32_t* hashes, Status SwissTableWithKeys::Map(Input* input, bool insert_missing, const uint32_t* hashes, uint8_t* match_bitvector_maybe_null, uint32_t* key_ids) { - util::TempVectorStack* temp_stack = input->temp_stack; + arrow::util::TempVectorStack* temp_stack = input->temp_stack; // Split into smaller mini-batches // @@ -990,8 +1000,8 @@ Status SwissTableWithKeys::Map(Input* input, bool insert_missing, const uint32_t int num_rows_to_process = input->selection_maybe_null ? input->num_selected : input->batch_end_row - input->batch_start_row; - auto hashes_buf = util::TempVectorHolder(temp_stack, minibatch_size); - auto match_bitvector_buf = util::TempVectorHolder( + auto hashes_buf = arrow::util::TempVectorHolder(temp_stack, minibatch_size); + auto match_bitvector_buf = arrow::util::TempVectorHolder( temp_stack, static_cast(bit_util::BytesForBits(minibatch_size)) + sizeof(uint64_t)); for (int minibatch_start = 0; minibatch_start < num_rows_to_process;) { @@ -1017,7 +1027,8 @@ Status SwissTableWithKeys::Map(Input* input, bool insert_missing, const uint32_t // Lookup existing keys. { - auto slots = util::TempVectorHolder(temp_stack, minibatch_size_next); + auto slots = + arrow::util::TempVectorHolder(temp_stack, minibatch_size_next); swiss_table_.early_filter(minibatch_size_next, minibatch_hashes, minibatch_match_bitvector, slots.mutable_data()); swiss_table_.find(minibatch_size_next, minibatch_hashes, minibatch_match_bitvector, @@ -1028,11 +1039,12 @@ Status SwissTableWithKeys::Map(Input* input, bool insert_missing, const uint32_t // Perform inserts of missing keys if required. // if (insert_missing) { - auto ids_buf = util::TempVectorHolder(temp_stack, minibatch_size_next); + auto ids_buf = + arrow::util::TempVectorHolder(temp_stack, minibatch_size_next); int num_ids; - util::bit_util::bits_to_indexes(0, swiss_table_.hardware_flags(), - minibatch_size_next, minibatch_match_bitvector, - &num_ids, ids_buf.mutable_data()); + arrow::util::bit_util::bits_to_indexes( + 0, swiss_table_.hardware_flags(), minibatch_size_next, + minibatch_match_bitvector, &num_ids, ids_buf.mutable_data()); RETURN_NOT_OK(swiss_table_.map_new_keys( num_ids, ids_buf.mutable_data(), minibatch_hashes, minibatch_key_ids, @@ -1182,7 +1194,7 @@ Status SwissTableForJoinBuild::Init(SwissTableForJoin* target, int dop, int64_t Status SwissTableForJoinBuild::PushNextBatch(int64_t thread_id, const ExecBatch& key_batch, const ExecBatch* payload_batch_maybe_null, - util::TempVectorStack* temp_stack) { + arrow::util::TempVectorStack* temp_stack) { ARROW_DCHECK(thread_id < dop_); ThreadState& locals = thread_states_[thread_id]; @@ -1254,7 +1266,7 @@ Status SwissTableForJoinBuild::PushNextBatch(int64_t thread_id, Status SwissTableForJoinBuild::ProcessPartition(int64_t thread_id, const ExecBatch& key_batch, const ExecBatch* payload_batch_maybe_null, - util::TempVectorStack* temp_stack, + arrow::util::TempVectorStack* temp_stack, int prtn_id) { ARROW_DCHECK(thread_id < dop_); ThreadState& locals = thread_states_[thread_id]; @@ -1472,7 +1484,7 @@ void SwissTableForJoinBuild::PrtnMerge(int prtn_id) { } } -void SwissTableForJoinBuild::FinishPrtnMerge(util::TempVectorStack* temp_stack) { +void SwissTableForJoinBuild::FinishPrtnMerge(arrow::util::TempVectorStack* temp_stack) { // Process overflow key ids // for (int prtn_id = 0; prtn_id < num_prtns_; ++prtn_id) { @@ -1855,7 +1867,7 @@ void JoinProbeProcessor::Init(int num_key_columns, JoinType join_type, Status JoinProbeProcessor::OnNextBatch(int64_t thread_id, const ExecBatch& keypayload_batch, - util::TempVectorStack* temp_stack, + arrow::util::TempVectorStack* temp_stack, std::vector* temp_column_arrays) { const SwissTable* swiss_table = hash_table_->keys()->swiss_table(); int64_t hardware_flags = swiss_table->hardware_flags(); @@ -1872,16 +1884,16 @@ Status JoinProbeProcessor::OnNextBatch(int64_t thread_id, // // Start by allocating mini-batch buffers // - auto hashes_buf = util::TempVectorHolder(temp_stack, minibatch_size); - auto match_bitvector_buf = util::TempVectorHolder( + auto hashes_buf = arrow::util::TempVectorHolder(temp_stack, minibatch_size); + auto match_bitvector_buf = arrow::util::TempVectorHolder( temp_stack, static_cast(bit_util::BytesForBits(minibatch_size))); - auto key_ids_buf = util::TempVectorHolder(temp_stack, minibatch_size); + auto key_ids_buf = arrow::util::TempVectorHolder(temp_stack, minibatch_size); auto materialize_batch_ids_buf = - util::TempVectorHolder(temp_stack, minibatch_size); + arrow::util::TempVectorHolder(temp_stack, minibatch_size); auto materialize_key_ids_buf = - util::TempVectorHolder(temp_stack, minibatch_size); + arrow::util::TempVectorHolder(temp_stack, minibatch_size); auto materialize_payload_ids_buf = - util::TempVectorHolder(temp_stack, minibatch_size); + arrow::util::TempVectorHolder(temp_stack, minibatch_size); for (int minibatch_start = 0; minibatch_start < num_rows;) { uint32_t minibatch_size_next = std::min(minibatch_size, num_rows - minibatch_start); @@ -1905,7 +1917,7 @@ Status JoinProbeProcessor::OnNextBatch(int64_t thread_id, if (join_type_ == JoinType::LEFT_SEMI || join_type_ == JoinType::LEFT_ANTI || join_type_ == JoinType::RIGHT_SEMI || join_type_ == JoinType::RIGHT_ANTI) { int num_passing_ids = 0; - util::bit_util::bits_to_indexes( + arrow::util::bit_util::bits_to_indexes( (join_type_ == JoinType::LEFT_ANTI) ? 0 : 1, hardware_flags, minibatch_size_next, match_bitvector_buf.mutable_data(), &num_passing_ids, materialize_batch_ids_buf.mutable_data()); @@ -1986,7 +1998,7 @@ Status JoinProbeProcessor::OnNextBatch(int64_t thread_id, // if (join_type_ == JoinType::LEFT_OUTER || join_type_ == JoinType::FULL_OUTER) { int num_passing_ids = 0; - util::bit_util::bits_to_indexes( + arrow::util::bit_util::bits_to_indexes( /*bit_to_search=*/0, hardware_flags, minibatch_size_next, match_bitvector_buf.mutable_data(), &num_passing_ids, materialize_batch_ids_buf.mutable_data()); @@ -2037,7 +2049,7 @@ class SwissJoin : public HashJoinImpl { FinishedCallback finished_callback) override { START_COMPUTE_SPAN(span_, "SwissJoinImpl", {{"detail", filter.ToString()}, - {"join.kind", arrow::compute::ToString(join_type)}, + {"join.kind", arrow::acero::ToString(join_type)}, {"join.threads", static_cast(num_threads)}}); num_threads_ = static_cast(num_threads); @@ -2119,7 +2131,7 @@ class SwissJoin : public HashJoinImpl { ExecBatch keypayload_batch; ARROW_ASSIGN_OR_RAISE(keypayload_batch, KeyPayloadFromInput(/*side=*/0, &batch)); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * temp_stack, + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * temp_stack, ctx_->GetTempStack(thread_index)); return CancelIfNotOK( @@ -2230,7 +2242,7 @@ class SwissJoin : public HashJoinImpl { input_batch.values[schema->num_cols(HashJoinProjection::KEY) + icol]; } } - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * temp_stack, + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * temp_stack, ctx_->GetTempStack(thread_id)); RETURN_NOT_OK(CancelIfNotOK(hash_table_build_.PushNextBatch( static_cast(thread_id), key_batch, no_payload ? nullptr : &payload_batch, @@ -2266,7 +2278,7 @@ class SwissJoin : public HashJoinImpl { Status MergeFinished(size_t thread_id) { RETURN_NOT_OK(status()); - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * temp_stack, + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * temp_stack, ctx_->GetTempStack(thread_id)); hash_table_build_.FinishPrtnMerge(temp_stack); return CancelIfNotOK(OnBuildHashTableFinished(static_cast(thread_id))); @@ -2320,23 +2332,23 @@ class SwissJoin : public HashJoinImpl { std::min((task_id + 1) * kNumRowsPerScanTask, hash_table_.num_rows()); // Get thread index and related temp vector stack // - ARROW_ASSIGN_OR_RAISE(util::TempVectorStack * temp_stack, + ARROW_ASSIGN_OR_RAISE(arrow::util::TempVectorStack * temp_stack, ctx_->GetTempStack(thread_id)); // Split into mini-batches // - auto payload_ids_buf = - util::TempVectorHolder(temp_stack, util::MiniBatch::kMiniBatchLength); - auto key_ids_buf = - util::TempVectorHolder(temp_stack, util::MiniBatch::kMiniBatchLength); - auto selection_buf = - util::TempVectorHolder(temp_stack, util::MiniBatch::kMiniBatchLength); + auto payload_ids_buf = arrow::util::TempVectorHolder( + temp_stack, arrow::util::MiniBatch::kMiniBatchLength); + auto key_ids_buf = arrow::util::TempVectorHolder( + temp_stack, arrow::util::MiniBatch::kMiniBatchLength); + auto selection_buf = arrow::util::TempVectorHolder( + temp_stack, arrow::util::MiniBatch::kMiniBatchLength); for (int64_t mini_batch_start = start_row; mini_batch_start < end_row;) { // Compute the size of the next mini-batch // int64_t mini_batch_size_next = std::min(end_row - mini_batch_start, - static_cast(util::MiniBatch::kMiniBatchLength)); + static_cast(arrow::util::MiniBatch::kMiniBatchLength)); // Get the list of key and payload ids from this mini-batch to output. // @@ -2535,5 +2547,5 @@ Result> HashJoinImpl::MakeSwiss() { return std::move(impl); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/swiss_join_avx2.cc b/cpp/src/arrow/acero/swiss_join_avx2.cc similarity index 98% rename from cpp/src/arrow/compute/exec/swiss_join_avx2.cc rename to cpp/src/arrow/acero/swiss_join_avx2.cc index da43370918c..d5c0b7817f5 100644 --- a/cpp/src/arrow/compute/exec/swiss_join_avx2.cc +++ b/cpp/src/arrow/acero/swiss_join_avx2.cc @@ -17,11 +17,11 @@ #include -#include "arrow/compute/exec/swiss_join_internal.h" +#include "arrow/acero/swiss_join_internal.h" #include "arrow/util/bit_util.h" namespace arrow { -namespace compute { +namespace acero { #if defined(ARROW_HAVE_AVX2) @@ -193,5 +193,5 @@ int RowArrayAccessor::VisitNulls_avx2(const RowTableImpl& rows, int column_id, #endif -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/swiss_join_internal.h b/cpp/src/arrow/acero/swiss_join_internal.h similarity index 96% rename from cpp/src/arrow/compute/exec/swiss_join_internal.h rename to cpp/src/arrow/acero/swiss_join_internal.h index 766f40e131c..17bacaf1446 100644 --- a/cpp/src/arrow/compute/exec/swiss_join_internal.h +++ b/cpp/src/arrow/acero/swiss_join_internal.h @@ -18,17 +18,27 @@ #pragma once #include -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/partition_util.h" -#include "arrow/compute/exec/schema_util.h" -#include "arrow/compute/exec/task_util.h" +#include "arrow/acero/options.h" +#include "arrow/acero/partition_util.h" +#include "arrow/acero/schema_util.h" +#include "arrow/acero/task_util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/key_map.h" #include "arrow/compute/light_array.h" #include "arrow/compute/row/encode_internal.h" namespace arrow { -namespace compute { + +using compute::ExecBatchBuilder; +using compute::KeyColumnArray; +using compute::KeyColumnMetadata; +using compute::ResizableArrayData; +using compute::RowTableEncoder; +using compute::RowTableImpl; +using compute::RowTableMetadata; +using compute::SwissTable; + +namespace acero { class RowArrayAccessor { public: @@ -112,7 +122,7 @@ struct RowArray { void Compare(const ExecBatch& batch, int begin_row_id, int end_row_id, int num_selected, const uint16_t* batch_selection_maybe_null, const uint32_t* array_row_ids, uint32_t* out_num_not_equal, uint16_t* out_not_equal_selection, - int64_t hardware_flags, util::TempVectorStack* temp_stack, + int64_t hardware_flags, arrow::util::TempVectorStack* temp_stack, std::vector& temp_column_arrays, uint8_t* out_match_bitvector_maybe_null = NULLPTR); @@ -276,14 +286,14 @@ class SwissTableMerge { struct SwissTableWithKeys { struct Input { Input(const ExecBatch* in_batch, int in_batch_start_row, int in_batch_end_row, - util::TempVectorStack* in_temp_stack, + arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays); - Input(const ExecBatch* in_batch, util::TempVectorStack* in_temp_stack, + Input(const ExecBatch* in_batch, arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays); Input(const ExecBatch* in_batch, int in_num_selected, const uint16_t* in_selection, - util::TempVectorStack* in_temp_stack, + arrow::util::TempVectorStack* in_temp_stack, std::vector* in_temp_column_arrays, std::vector* in_temp_group_ids); @@ -302,7 +312,7 @@ struct SwissTableWithKeys { const uint16_t* selection_maybe_null; // Thread specific scratch buffers for storing temporary data. // - util::TempVectorStack* temp_stack; + arrow::util::TempVectorStack* temp_stack; std::vector* temp_column_arrays; std::vector* temp_group_ids; }; @@ -420,7 +430,7 @@ class SwissTableForJoinBuild { // Status PushNextBatch(int64_t thread_id, const ExecBatch& key_batch, const ExecBatch* payload_batch_maybe_null, - util::TempVectorStack* temp_stack); + arrow::util::TempVectorStack* temp_stack); // Allocate memory and initialize counters required for parallel merging of // hash table partitions. @@ -438,7 +448,7 @@ class SwissTableForJoinBuild { // parallel merging phase, due to hash table search resulting in crossing // partition boundaries. // - void FinishPrtnMerge(util::TempVectorStack* temp_stack); + void FinishPrtnMerge(arrow::util::TempVectorStack* temp_stack); // The number of partitions is the number of parallel tasks to execute during // the final phase of hash table build process. @@ -451,7 +461,7 @@ class SwissTableForJoinBuild { void InitRowArray(); Status ProcessPartition(int64_t thread_id, const ExecBatch& key_batch, const ExecBatch* payload_batch_maybe_null, - util::TempVectorStack* temp_stack, int prtn_id); + arrow::util::TempVectorStack* temp_stack, int prtn_id); SwissTableForJoin* target_; // DOP stands for Degree Of Parallelism - the maximum number of participating @@ -737,7 +747,7 @@ class JoinProbeProcessor { std::vector materialize, const std::vector* cmp, OutputBatchFn output_batch_fn); Status OnNextBatch(int64_t thread_id, const ExecBatch& keypayload_batch, - util::TempVectorStack* temp_stack, + arrow::util::TempVectorStack* temp_stack, std::vector* temp_column_arrays); // Must be called by a single-thread having exclusive access to the instance @@ -757,5 +767,5 @@ class JoinProbeProcessor { OutputBatchFn output_batch_fn_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/task_util.cc b/cpp/src/arrow/acero/task_util.cc similarity index 99% rename from cpp/src/arrow/compute/exec/task_util.cc rename to cpp/src/arrow/acero/task_util.cc index 8568450119a..8127902e69e 100644 --- a/cpp/src/arrow/compute/exec/task_util.cc +++ b/cpp/src/arrow/acero/task_util.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/task_util.h" +#include "arrow/acero/task_util.h" #include #include @@ -23,7 +23,7 @@ #include "arrow/util/logging.h" namespace arrow { -namespace compute { +namespace acero { class TaskSchedulerImpl : public TaskScheduler { public: @@ -422,5 +422,5 @@ std::unique_ptr TaskScheduler::Make() { return std::move(impl); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/task_util.h b/cpp/src/arrow/acero/task_util.h similarity index 96% rename from cpp/src/arrow/compute/exec/task_util.h rename to cpp/src/arrow/acero/task_util.h index 5dc66aee167..bc19396bd24 100644 --- a/cpp/src/arrow/compute/exec/task_util.h +++ b/cpp/src/arrow/acero/task_util.h @@ -22,11 +22,12 @@ #include #include +#include "arrow/acero/visibility.h" #include "arrow/status.h" #include "arrow/util/logging.h" namespace arrow { -namespace compute { +namespace acero { // Atomic value surrounded by padding bytes to avoid cache line invalidation // whenever it is modified by a concurrent thread on a different CPU core. @@ -53,7 +54,7 @@ class AtomicWithPadding { // // Also allows for executing next pending tasks immediately using a caller thread. // -class ARROW_EXPORT TaskScheduler { +class ARROW_ACERO_EXPORT TaskScheduler { public: using TaskImpl = std::function; using TaskGroupContinuationImpl = std::function; @@ -96,5 +97,5 @@ class ARROW_EXPORT TaskScheduler { static std::unique_ptr Make(); }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/task_util_test.cc b/cpp/src/arrow/acero/task_util_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/task_util_test.cc rename to cpp/src/arrow/acero/task_util_test.cc index fd9502bdfac..dafb6b24b4c 100644 --- a/cpp/src/arrow/compute/exec/task_util_test.cc +++ b/cpp/src/arrow/acero/task_util_test.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/task_util.h" +#include "arrow/acero/task_util.h" #include #include @@ -25,7 +25,7 @@ #include -#include "arrow/compute/exec/util.h" +#include "arrow/acero/util.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/thread_pool.h" @@ -33,7 +33,7 @@ namespace arrow { using internal::ThreadPool; -namespace compute { +namespace acero { /// \brief Create a thread pool and start all threads /// @@ -224,5 +224,5 @@ TEST(TaskScheduler, StressTwo) { } } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_nodes.cc b/cpp/src/arrow/acero/test_nodes.cc similarity index 96% rename from cpp/src/arrow/compute/exec/test_nodes.cc rename to cpp/src/arrow/acero/test_nodes.cc index 94a0df58c36..ff95f72e6e6 100644 --- a/cpp/src/arrow/compute/exec/test_nodes.cc +++ b/cpp/src/arrow/acero/test_nodes.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/test_nodes.h" +#include "arrow/acero/test_nodes.h" #include #include @@ -23,10 +23,10 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/api.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" #include "arrow/io/interfaces.h" #include "arrow/testing/random.h" #include "arrow/util/checked_cast.h" @@ -34,9 +34,9 @@ namespace arrow { -using internal::checked_cast; +using arrow::internal::checked_cast; -namespace compute { +namespace acero { AsyncGenerator> MakeDelayedGen( Iterator> src, std::string label, double delay_sec, @@ -209,5 +209,5 @@ void RegisterTestNodes() { }); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_nodes.h b/cpp/src/arrow/acero/test_nodes.h similarity index 94% rename from cpp/src/arrow/compute/exec/test_nodes.h rename to cpp/src/arrow/acero/test_nodes.h index afe37f8c9f9..2d1d630b3b7 100644 --- a/cpp/src/arrow/compute/exec/test_nodes.h +++ b/cpp/src/arrow/acero/test_nodes.h @@ -19,12 +19,12 @@ #include -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/testing/random.h" namespace arrow { -namespace compute { +namespace acero { // \brief Make a delaying source that is optionally noisy (prints when it emits) AsyncGenerator> MakeDelayedGen( @@ -55,5 +55,5 @@ struct JitterNodeOptions : public ExecNodeOptions { void RegisterTestNodes(); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_nodes_test.cc b/cpp/src/arrow/acero/test_nodes_test.cc similarity index 94% rename from cpp/src/arrow/compute/exec/test_nodes_test.cc rename to cpp/src/arrow/acero/test_nodes_test.cc index 201c46c42f2..967bd63769e 100644 --- a/cpp/src/arrow/compute/exec/test_nodes_test.cc +++ b/cpp/src/arrow/acero/test_nodes_test.cc @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/test_nodes.h" +#include "arrow/acero/test_nodes.h" #include -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/exec_plan.h" #include "arrow/table.h" #include "arrow/testing/generator.h" #include "arrow/testing/random.h" namespace arrow { -namespace compute { +namespace acero { TEST(JitterNode, Basic) { static constexpr random::SeedType kTestSeed = 42; @@ -56,5 +56,5 @@ TEST(JitterNode, Basic) { ASSERT_GT(numOutOfPlace, 0); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/acero/test_util_internal.cc similarity index 92% rename from cpp/src/arrow/compute/exec/test_util.cc rename to cpp/src/arrow/acero/test_util_internal.cc index ddfc58426d1..46116d66d8f 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/acero/test_util_internal.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/test_util.h" +#include "arrow/acero/test_util_internal.h" #include #include @@ -31,11 +31,12 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" +#include "arrow/array/validate.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/function_internal.h" #include "arrow/datum.h" #include "arrow/io/interfaces.h" @@ -53,9 +54,68 @@ namespace arrow { -using internal::Executor; +using arrow::internal::Executor; + +using compute::SortKey; +using compute::Take; + +namespace acero { + +namespace { +void ValidateOutputImpl(const ArrayData& output) { + ASSERT_OK(::arrow::internal::ValidateArrayFull(output)); + TestInitialized(output); +} + +void ValidateOutputImpl(const ChunkedArray& output) { + ASSERT_OK(output.ValidateFull()); + for (const auto& chunk : output.chunks()) { + TestInitialized(*chunk); + } +} + +void ValidateOutputImpl(const RecordBatch& output) { + ASSERT_OK(output.ValidateFull()); + for (const auto& column : output.column_data()) { + TestInitialized(*column); + } +} + +void ValidateOutputImpl(const Table& output) { + ASSERT_OK(output.ValidateFull()); + for (const auto& column : output.columns()) { + for (const auto& chunk : column->chunks()) { + TestInitialized(*chunk); + } + } +} + +void ValidateOutputImpl(const Scalar& output) { ASSERT_OK(output.ValidateFull()); } + +} // namespace + +void ValidateOutput(const Datum& output) { + switch (output.kind()) { + case Datum::ARRAY: + ValidateOutputImpl(*output.array()); + break; + case Datum::CHUNKED_ARRAY: + ValidateOutputImpl(*output.chunked_array()); + break; + case Datum::RECORD_BATCH: + ValidateOutputImpl(*output.record_batch()); + break; + case Datum::TABLE: + ValidateOutputImpl(*output.table()); + break; + case Datum::SCALAR: + ValidateOutputImpl(*output.scalar()); + break; + default: + break; + } +} -namespace compute { namespace { struct DummyNode : ExecNode { @@ -624,5 +684,5 @@ Result> MakeRandomTimeSeriesTable( return Table::Make(schema, columns, num_rows); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/acero/test_util_internal.h similarity index 90% rename from cpp/src/arrow/compute/exec/test_util.h rename to cpp/src/arrow/acero/test_util_internal.h index 7da3cba5aca..03f41702865 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/acero/test_util_internal.h @@ -17,8 +17,8 @@ #pragma once -#include -#include +#include "arrow/testing/gtest_util.h" +#include "arrow/util/vector.h" #include #include @@ -26,26 +26,27 @@ #include #include +#include "arrow/acero/exec_plan.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" #include "arrow/compute/kernel.h" #include "arrow/testing/visibility.h" #include "arrow/util/async_generator.h" #include "arrow/util/pcg_random.h" namespace arrow { -namespace compute { + +namespace acero { + +void ValidateOutput(const Datum& output); using StartProducingFunc = std::function; using StopProducingFunc = std::function; // Make a dummy node that has no execution behaviour -ARROW_TESTING_EXPORT ExecNode* MakeDummyNode(ExecPlan* plan, std::string label, std::vector inputs, bool is_sink = false, StartProducingFunc = {}, StopProducingFunc = {}); -ARROW_TESTING_EXPORT ExecBatch ExecBatchFromJSON(const std::vector& types, std::string_view json); /// \brief Shape qualifier for value types. In certain instances @@ -53,7 +54,6 @@ ExecBatch ExecBatchFromJSON(const std::vector& types, std::string_vi /// other kernels arguments can be arrays or scalars enum class ArgShape { ANY, ARRAY, SCALAR }; -ARROW_TESTING_EXPORT ExecBatch ExecBatchFromJSON(const std::vector& types, const std::vector& shapes, std::string_view json); @@ -92,92 +92,70 @@ struct BatchesWithSchema { } }; -ARROW_TESTING_EXPORT Future<> StartAndFinish(ExecPlan* plan); -ARROW_TESTING_EXPORT Future> StartAndCollect( ExecPlan* plan, AsyncGenerator> gen); -ARROW_TESTING_EXPORT AsyncGenerator> MakeIntegerBatchGen( const std::vector>& gens, const std::shared_ptr& schema, int num_batches, int batch_size); -ARROW_TESTING_EXPORT BatchesWithSchema MakeBasicBatches(); -ARROW_TESTING_EXPORT BatchesWithSchema MakeNestedBatches(); -ARROW_TESTING_EXPORT BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, int num_batches = 10, int batch_size = 4, int64_t alignment = kDefaultBufferAlignment, MemoryPool* memory_pool = nullptr); -ARROW_TESTING_EXPORT Result MakeIntegerBatches( const std::vector>& gens, const std::shared_ptr& schema, int num_batches, int batch_size); -ARROW_TESTING_EXPORT BatchesWithSchema MakeBatchesFromString(const std::shared_ptr& schema, const std::vector& json_strings, int multiplicity = 1); -ARROW_TESTING_EXPORT Result>> ToArrayVectors( const BatchesWithSchema& batches_with_schema); -ARROW_TESTING_EXPORT Result>> ToExecBatches( const BatchesWithSchema& batches); -ARROW_TESTING_EXPORT Result>> ToRecordBatches( const BatchesWithSchema& batches); -ARROW_TESTING_EXPORT Result> ToRecordBatchReader( const BatchesWithSchema& batches_with_schema); -ARROW_TESTING_EXPORT Result>> ToArrayVectors( const BatchesWithSchema& batches_with_schema); -ARROW_TESTING_EXPORT Result>> ToExecBatches( const BatchesWithSchema& batches); -ARROW_TESTING_EXPORT Result>> ToRecordBatches( const BatchesWithSchema& batches); -ARROW_TESTING_EXPORT Result> SortTableOnAllFields(const std::shared_ptr
& tab); -ARROW_TESTING_EXPORT void AssertTablesEqualIgnoringOrder(const std::shared_ptr
& exp, const std::shared_ptr
& act); -ARROW_TESTING_EXPORT void AssertExecBatchesEqualIgnoringOrder(const std::shared_ptr& schema, const std::vector& exp, const std::vector& act); -ARROW_TESTING_EXPORT void AssertExecBatchesEqual(const std::shared_ptr& schema, const std::vector& exp, const std::vector& act); -ARROW_TESTING_EXPORT void AssertExecBatchesSequenced( - const std::vector& batches); +void AssertExecBatchesSequenced(const std::vector& batches); -ARROW_TESTING_EXPORT bool operator==(const Declaration&, const Declaration&); -ARROW_TESTING_EXPORT void PrintTo(const Declaration& decl, std::ostream* os); class Random64Bit { @@ -223,9 +201,8 @@ struct TableGenerationProperties { /// where idx is in [0, properties.num_columns) /// Each id has rows corresponding to a singular data point in the time range (start, end, /// time_frequency). The table is sorted by time. -ARROW_TESTING_EXPORT Result> MakeRandomTimeSeriesTable( const TableGenerationProperties& properties); -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/tpch_benchmark.cc b/cpp/src/arrow/acero/tpch_benchmark.cc similarity index 93% rename from cpp/src/arrow/compute/exec/tpch_benchmark.cc rename to cpp/src/arrow/acero/tpch_benchmark.cc index 2adee26a425..feb16ea4bda 100644 --- a/cpp/src/arrow/compute/exec/tpch_benchmark.cc +++ b/cpp/src/arrow/acero/tpch_benchmark.cc @@ -17,17 +17,24 @@ #include +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/tpch_node.h" #include "arrow/compute/cast.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/tpch_node.h" #include "arrow/testing/future_util.h" #include namespace arrow { -namespace compute { -namespace internal { + +using compute::call; +using compute::CountOptions; +using compute::field_ref; +using compute::ScalarAggregateOptions; +using compute::SortKey; + +namespace acero { +namespace acerointernal { std::shared_ptr Plan_Q1(AsyncGenerator>* sink_gen, int scale_factor) { @@ -117,6 +124,6 @@ static void BM_Tpch_Q1(benchmark::State& st) { } BENCHMARK(BM_Tpch_Q1)->Args({1})->ArgNames({"ScaleFactor"}); -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/tpch_node.cc b/cpp/src/arrow/acero/tpch_node.cc similarity index 99% rename from cpp/src/arrow/compute/exec/tpch_node.cc rename to cpp/src/arrow/acero/tpch_node.cc index cba370ab014..36f894c19bc 100644 --- a/cpp/src/arrow/compute/exec/tpch_node.cc +++ b/cpp/src/arrow/acero/tpch_node.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/tpch_node.h" +#include "arrow/acero/tpch_node.h" #include #include @@ -28,10 +28,10 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/query_context.h" #include "arrow/buffer.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/query_context.h" #include "arrow/datum.h" #include "arrow/util/async_util.h" #include "arrow/util/formatting.h" @@ -43,11 +43,11 @@ namespace arrow { -using internal::checked_cast; -using internal::GetRandomSeed; +using arrow::internal::checked_cast; +using arrow::internal::GetRandomSeed; -namespace compute { -namespace internal { +namespace acero { +namespace acerointernal { /* Architecture of the generator: This is a multithreaded implementation of TPC-H's DBGen data generator. For each table @@ -3538,6 +3538,6 @@ Result> TpchGen::Make(ExecPlan* plan, double scale_fact return std::make_unique(plan, scale_factor, batch_size, *seed); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/tpch_node.h b/cpp/src/arrow/acero/tpch_node.h similarity index 92% rename from cpp/src/arrow/compute/exec/tpch_node.h rename to cpp/src/arrow/acero/tpch_node.h index c99d61d950b..280e05fe5ed 100644 --- a/cpp/src/arrow/compute/exec/tpch_node.h +++ b/cpp/src/arrow/acero/tpch_node.h @@ -22,15 +22,16 @@ #include #include -#include "arrow/compute/exec/type_fwd.h" +#include "arrow/acero/type_fwd.h" +#include "arrow/acero/visibility.h" #include "arrow/result.h" #include "arrow/status.h" namespace arrow { -namespace compute { -namespace internal { +namespace acero { +namespace acerointernal { -class ARROW_EXPORT TpchGen { +class ARROW_ACERO_EXPORT TpchGen { public: virtual ~TpchGen() = default; @@ -59,6 +60,6 @@ class ARROW_EXPORT TpchGen { virtual Result Region(std::vector columns = {}) = 0; }; -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/tpch_node_test.cc b/cpp/src/arrow/acero/tpch_node_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/tpch_node_test.cc rename to cpp/src/arrow/acero/tpch_node_test.cc index 640a200e4ef..015066f9740 100644 --- a/cpp/src/arrow/compute/exec/tpch_node_test.cc +++ b/cpp/src/arrow/acero/tpch_node_test.cc @@ -23,10 +23,10 @@ #include #include -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/tpch_node.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" +#include "arrow/acero/tpch_node.h" +#include "arrow/acero/util.h" #include "arrow/compute/kernels/row_encoder_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/testing/gtest_util.h" @@ -39,10 +39,10 @@ namespace arrow { -using internal::StartsWith; +using arrow::internal::StartsWith; -namespace compute { -namespace internal { +namespace acero { +namespace acerointernal { static constexpr uint32_t kStartDate = 8035; // January 1, 1992 is 8035 days after January 1, 1970 @@ -642,6 +642,6 @@ TEST(TpchNode, AllTables) { } } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/type_fwd.h b/cpp/src/arrow/acero/type_fwd.h similarity index 96% rename from cpp/src/arrow/compute/exec/type_fwd.h rename to cpp/src/arrow/acero/type_fwd.h index c4026032619..f0410de9f78 100644 --- a/cpp/src/arrow/compute/exec/type_fwd.h +++ b/cpp/src/arrow/acero/type_fwd.h @@ -21,7 +21,7 @@ namespace arrow { -namespace compute { +namespace acero { class ExecNode; class ExecPlan; @@ -32,5 +32,5 @@ struct QueryOptions; struct Declaration; class SinkNodeConsumer; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/union_node.cc b/cpp/src/arrow/acero/union_node.cc similarity index 95% rename from cpp/src/arrow/compute/exec/union_node.cc rename to cpp/src/arrow/acero/union_node.cc index 59c1f914281..f65027e02e6 100644 --- a/cpp/src/arrow/compute/exec/union_node.cc +++ b/cpp/src/arrow/acero/union_node.cc @@ -17,10 +17,10 @@ #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" #include "arrow/compute/api.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/util.h" #include "arrow/util/bitmap_ops.h" #include "arrow/util/checked_cast.h" #include "arrow/util/future.h" @@ -34,7 +34,7 @@ namespace arrow { using internal::checked_cast; using internal::ToChars; -namespace compute { +namespace acero { namespace { std::vector GetInputLabels(const ExecNode::NodeVector& inputs) { @@ -119,12 +119,12 @@ class UnionNode : public ExecNode, public TracedNode { std::atomic total_batches_{0}; }; -namespace internal { +namespace acerointernal { void RegisterUnionNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("union", UnionNode::Make)); } -} // namespace internal -} // namespace compute +} // namespace acerointernal +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/union_node_test.cc b/cpp/src/arrow/acero/union_node_test.cc similarity index 97% rename from cpp/src/arrow/compute/exec/union_node_test.cc rename to cpp/src/arrow/acero/union_node_test.cc index d14bfe16e5f..8c07ece939a 100644 --- a/cpp/src/arrow/compute/exec/union_node_test.cc +++ b/cpp/src/arrow/acero/union_node_test.cc @@ -18,9 +18,9 @@ #include #include +#include "arrow/acero/options.h" +#include "arrow/acero/test_util_internal.h" #include "arrow/api.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" #include "arrow/testing/random.h" @@ -28,7 +28,7 @@ using testing::UnorderedElementsAreArray; namespace arrow { -namespace compute { +namespace acero { struct TestUnionNode : public ::testing::Test { static constexpr int kNumBatches = 10; @@ -146,5 +146,5 @@ TEST_F(TestUnionNode, TestEmpty) { this->CheckUnionExecNode(/*num_input_nodes*/ 0, /*num_batches=*/0, /*parallel=*/false); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util.cc b/cpp/src/arrow/acero/util.cc similarity index 97% rename from cpp/src/arrow/compute/exec/util.cc rename to cpp/src/arrow/acero/util.cc index 6a1fd37aa19..d0b448ad6c3 100644 --- a/cpp/src/arrow/compute/exec/util.cc +++ b/cpp/src/arrow/acero/util.cc @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/util.h" +#include "arrow/acero/util.h" -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/exec_plan.h" #include "arrow/table.h" #include "arrow/util/bit_util.h" #include "arrow/util/bitmap_ops.h" @@ -25,7 +25,7 @@ #include "arrow/util/ubsan.h" namespace arrow { -namespace compute { +namespace acero { Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inputs, int expected_num_inputs, const char* kind_name) { @@ -134,5 +134,5 @@ void TracedNode::NoteInputReceived(const ExecBatch& batch) const { {{"node.label", node_->label()}}); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/acero/util.h similarity index 94% rename from cpp/src/arrow/compute/exec/util.h rename to cpp/src/arrow/acero/util.h index 0ef8ab264e0..b83b8712618 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/acero/util.h @@ -24,9 +24,9 @@ #include #include +#include "arrow/acero/options.h" +#include "arrow/acero/type_fwd.h" #include "arrow/buffer.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/type_fwd.h" #include "arrow/compute/expression.h" #include "arrow/compute/util.h" #include "arrow/memory_pool.h" @@ -40,17 +40,18 @@ #include "arrow/util/type_fwd.h" namespace arrow { -namespace compute { -ARROW_EXPORT +namespace acero { + +ARROW_ACERO_EXPORT Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector& inputs, int expected_num_inputs, const char* kind_name); -ARROW_EXPORT +ARROW_ACERO_EXPORT Result> TableFromExecBatches( const std::shared_ptr& schema, const std::vector& exec_batches); -class ARROW_EXPORT AtomicCounter { +class ARROW_ACERO_EXPORT AtomicCounter { public: AtomicCounter() = default; @@ -94,7 +95,7 @@ class ARROW_EXPORT AtomicCounter { std::atomic complete_{false}; }; -class ARROW_EXPORT ThreadIndexer { +class ARROW_ACERO_EXPORT ThreadIndexer { public: size_t operator()(); @@ -103,7 +104,7 @@ class ARROW_EXPORT ThreadIndexer { private: static size_t Check(size_t thread_index); - util::Mutex mutex_; + arrow::util::Mutex mutex_; std::unordered_map id_to_index_; }; @@ -154,7 +155,7 @@ class TailSkipForSIMD { }; /// \brief A consumer that collects results into an in-memory table -struct ARROW_EXPORT TableSinkNodeConsumer : public SinkNodeConsumer { +struct ARROW_ACERO_EXPORT TableSinkNodeConsumer : public SinkNodeConsumer { public: TableSinkNodeConsumer(std::shared_ptr
* out, MemoryPool* pool) : out_(out), pool_(pool) {} @@ -168,10 +169,10 @@ struct ARROW_EXPORT TableSinkNodeConsumer : public SinkNodeConsumer { MemoryPool* pool_; std::shared_ptr schema_; std::vector> batches_; - util::Mutex consume_mutex_; + arrow::util::Mutex consume_mutex_; }; -class ARROW_EXPORT NullSinkNodeConsumer : public SinkNodeConsumer { +class ARROW_ACERO_EXPORT NullSinkNodeConsumer : public SinkNodeConsumer { public: Status Init(const std::shared_ptr&, BackpressureControl*, ExecPlan* plan) override { @@ -188,7 +189,7 @@ class ARROW_EXPORT NullSinkNodeConsumer : public SinkNodeConsumer { /// CRTP helper for tracing helper functions -class ARROW_EXPORT TracedNode { +class ARROW_ACERO_EXPORT TracedNode { public: // All nodes should call TraceStartProducing or NoteStartProducing exactly once // Most nodes will be fine with a call to NoteStartProducing since the StartProducing @@ -225,5 +226,5 @@ class ARROW_EXPORT TracedNode { ExecNode* node_; }; -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/compute/exec/util_avx2.cc b/cpp/src/arrow/acero/util_avx2.cc similarity index 99% rename from cpp/src/arrow/compute/exec/util_avx2.cc rename to cpp/src/arrow/acero/util_avx2.cc index 2357bc936a2..7c2a3782545 100644 --- a/cpp/src/arrow/compute/exec/util_avx2.cc +++ b/cpp/src/arrow/acero/util_avx2.cc @@ -17,7 +17,7 @@ #include -#include "arrow/compute/exec/util.h" +#include "arrow/acero/util.h" #include "arrow/util/bit_util.h" namespace arrow { diff --git a/cpp/src/arrow/compute/exec/util_test.cc b/cpp/src/arrow/acero/util_test.cc similarity index 98% rename from cpp/src/arrow/compute/exec/util_test.cc rename to cpp/src/arrow/acero/util_test.cc index ca5118dc1aa..a291075a0a9 100644 --- a/cpp/src/arrow/compute/exec/util_test.cc +++ b/cpp/src/arrow/acero/util_test.cc @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/hash_join_node.h" -#include "arrow/compute/exec/schema_util.h" +#include "arrow/acero/hash_join_node.h" +#include "arrow/acero/schema_util.h" #include "arrow/testing/extension_type.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" @@ -24,7 +24,7 @@ using testing::Eq; namespace arrow { -namespace compute { +namespace acero { const char* kLeftSuffix = ".left"; const char* kRightSuffix = ".right"; @@ -184,5 +184,5 @@ TEST(FieldMap, ExtensionTypeHashJoin) { EXPECT_EQ(i.get(0), 0); } -} // namespace compute +} // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/visibility.h b/cpp/src/arrow/acero/visibility.h new file mode 100644 index 00000000000..02382232b69 --- /dev/null +++ b/cpp/src/arrow/acero/visibility.h @@ -0,0 +1,50 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// This API is EXPERIMENTAL. + +#pragma once + +#if defined(_WIN32) || defined(__CYGWIN__) +#if defined(_MSC_VER) +#pragma warning(push) +#pragma warning(disable : 4251) +#else +#pragma GCC diagnostic ignored "-Wattributes" +#endif + +#ifdef ARROW_ACERO_STATIC +#define ARROW_ACERO_EXPORT +#elif defined(ARROW_ACERO_EXPORTING) +#define ARROW_ACERO_EXPORT __declspec(dllexport) +#else +#define ARROW_ACERO_EXPORT __declspec(dllimport) +#endif + +#define ARROW_ACERO_NO_EXPORT +#else // Not Windows +#ifndef ARROW_ACERO_EXPORT +#define ARROW_ACERO_EXPORT __attribute__((visibility("default"))) +#endif +#ifndef ARROW_ACERO_NO_EXPORT +#define ARROW_ACERO_NO_EXPORT __attribute__((visibility("hidden"))) +#endif +#endif // Not-Windows + +#if defined(_MSC_VER) +#pragma warning(pop) +#endif diff --git a/cpp/src/arrow/compute/CMakeLists.txt b/cpp/src/arrow/compute/CMakeLists.txt index f459dd2cc76..1d71c14dca7 100644 --- a/cpp/src/arrow/compute/CMakeLists.txt +++ b/cpp/src/arrow/compute/CMakeLists.txt @@ -95,5 +95,4 @@ add_arrow_benchmark(function_benchmark PREFIX "arrow-compute") add_subdirectory(kernels) -add_subdirectory(exec) add_subdirectory(row) diff --git a/cpp/src/arrow/compute/exec/CMakeLists.txt b/cpp/src/arrow/compute/exec/CMakeLists.txt deleted file mode 100644 index 61cf1903b62..00000000000 --- a/cpp/src/arrow/compute/exec/CMakeLists.txt +++ /dev/null @@ -1,70 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -arrow_install_all_headers("arrow/compute/exec") - -add_arrow_compute_test(subtree_test SOURCES subtree_test.cc) - -add_arrow_compute_test(plan_test - SOURCES - plan_test.cc - test_nodes_test.cc - test_nodes.cc) -add_arrow_compute_test(fetch_node_test SOURCES fetch_node_test.cc test_nodes.cc) -add_arrow_compute_test(hash_join_node_test SOURCES hash_join_node_test.cc - bloom_filter_test.cc) -add_arrow_compute_test(order_by_node_test SOURCES order_by_node_test.cc test_nodes.cc) -add_arrow_compute_test(pivot_longer_node_test SOURCES pivot_longer_node_test.cc - test_nodes.cc) -add_arrow_compute_test(asof_join_node_test SOURCES asof_join_node_test.cc test_nodes.cc) -add_arrow_compute_test(tpch_node_test) -add_arrow_compute_test(union_node_test) -add_arrow_compute_test(groupby_test) -add_arrow_compute_test(util_test SOURCES util_test.cc task_util_test.cc) - -add_arrow_benchmark(expression_benchmark PREFIX "arrow-compute") - -add_arrow_benchmark(filter_benchmark - PREFIX - "arrow-compute" - SOURCES - benchmark_util.cc - filter_benchmark.cc) - -add_arrow_benchmark(project_benchmark - PREFIX - "arrow-compute" - SOURCES - benchmark_util.cc - project_benchmark.cc) - -add_arrow_benchmark(asof_join_benchmark PREFIX "arrow-compute") - -add_arrow_benchmark(tpch_benchmark PREFIX "arrow-compute") - -if(ARROW_BUILD_OPENMP_BENCHMARKS) - find_package(OpenMP REQUIRED) - add_arrow_benchmark(hash_join_benchmark - PREFIX - "arrow-compute" - EXTRA_LINK_LIBS - OpenMP::OpenMP_CXX) - if(MSVC) - target_compile_options(arrow-compute-hash-join-benchmark - PRIVATE "-openmp:experimental -openmp:llvm") - endif() -endif() diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index baec4a26a95..a642130cd7d 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -473,7 +473,7 @@ using KernelInit = std::function>( /// \brief Base type for kernels. Contains the function signature and /// optionally the state initialization function, along with some common /// attributes -struct Kernel { +struct ARROW_EXPORT Kernel { Kernel() = default; Kernel(std::shared_ptr sig, KernelInit init) @@ -524,7 +524,7 @@ using ArrayKernelExec = Status (*)(KernelContext*, const ExecSpan&, ExecResult*) /// \brief Kernel data structure for implementations of ScalarFunction. In /// addition to the members found in Kernel, contains the null handling /// and memory pre-allocation preferences. -struct ScalarKernel : public Kernel { +struct ARROW_EXPORT ScalarKernel : public Kernel { ScalarKernel() = default; ScalarKernel(std::shared_ptr sig, ArrayKernelExec exec, @@ -561,7 +561,7 @@ struct ScalarKernel : public Kernel { /// contains an optional finalizer function, the null handling and memory /// pre-allocation preferences (which have different defaults from /// ScalarKernel), and some other execution-related options. -struct VectorKernel : public Kernel { +struct ARROW_EXPORT VectorKernel : public Kernel { /// \brief See VectorKernel::finalize member for usage using FinalizeFunc = std::function*)>; @@ -643,7 +643,7 @@ using ScalarAggregateFinalize = Status (*)(KernelContext*, Datum*); /// * merge: combines one KernelState with another. /// * finalize: produces the end result of the aggregation using the /// KernelState in the KernelContext. -struct ScalarAggregateKernel : public Kernel { +struct ARROW_EXPORT ScalarAggregateKernel : public Kernel { ScalarAggregateKernel() = default; ScalarAggregateKernel(std::shared_ptr sig, KernelInit init, @@ -694,7 +694,7 @@ using HashAggregateFinalize = Status (*)(KernelContext*, Datum*); /// * merge: combines one KernelState with another. /// * finalize: produces the end result of the aggregation using the /// KernelState in the KernelContext. -struct HashAggregateKernel : public Kernel { +struct ARROW_EXPORT HashAggregateKernel : public Kernel { HashAggregateKernel() = default; HashAggregateKernel(std::shared_ptr sig, KernelInit init, diff --git a/cpp/src/arrow/compute/kernels/CMakeLists.txt b/cpp/src/arrow/compute/kernels/CMakeLists.txt index 0c5a3b7d787..dcb02408947 100644 --- a/cpp/src/arrow/compute/kernels/CMakeLists.txt +++ b/cpp/src/arrow/compute/kernels/CMakeLists.txt @@ -24,9 +24,6 @@ add_arrow_test(scalar_cast_test scalar_cast_test.cc test_util.cc) -add_arrow_test(kernel_utility_test ${ARROW_COMPUTE_TEST_ARGS} SOURCES - codegen_internal_test.cc) - # ---------------------------------------------------------------------- # Scalar kernels @@ -93,9 +90,9 @@ add_arrow_benchmark(vector_selection_benchmark PREFIX "arrow-compute") # Aggregates -add_arrow_compute_test(aggregate_test - SOURCES - aggregate_test.cc - hash_aggregate_test.cc - test_util.cc) -add_arrow_benchmark(aggregate_benchmark PREFIX "arrow-compute") +add_arrow_compute_test(aggregate_test SOURCES aggregate_test.cc test_util.cc) + +# ---------------------------------------------------------------------- +# Utilities + +add_arrow_compute_test(kernel_utility_test SOURCES codegen_internal_test.cc) diff --git a/cpp/src/arrow/compute/key_hash.h b/cpp/src/arrow/compute/key_hash.h index ddf86dfcdc0..e43d7b8df52 100644 --- a/cpp/src/arrow/compute/key_hash.h +++ b/cpp/src/arrow/compute/key_hash.h @@ -53,6 +53,10 @@ class ARROW_EXPORT Hashing32 { int64_t hardware_flags, util::TempVectorStack* temp_stack, int64_t offset, int64_t length); + static void HashFixed(int64_t hardware_flags, bool combine_hashes, uint32_t num_keys, + uint64_t length_key, const uint8_t* keys, uint32_t* hashes, + uint32_t* temp_hashes_for_combine); + private: static const uint32_t PRIME32_1 = 0x9E3779B1; static const uint32_t PRIME32_2 = 0x85EBCA77; @@ -62,10 +66,6 @@ class ARROW_EXPORT Hashing32 { static const uint32_t kCombineConst = 0x9e3779b9UL; static const int64_t kStripeSize = 4 * sizeof(uint32_t); - static void HashFixed(int64_t hardware_flags, bool combine_hashes, uint32_t num_keys, - uint64_t length_key, const uint8_t* keys, uint32_t* hashes, - uint32_t* temp_hashes_for_combine); - static void HashVarLen(int64_t hardware_flags, bool combine_hashes, uint32_t num_rows, const uint32_t* offsets, const uint8_t* concatenated_keys, uint32_t* hashes, uint32_t* temp_hashes_for_combine); @@ -166,6 +166,9 @@ class ARROW_EXPORT Hashing64 { int64_t hardware_flags, util::TempVectorStack* temp_stack, int64_t offset, int64_t length); + static void HashFixed(bool combine_hashes, uint32_t num_keys, uint64_t length_key, + const uint8_t* keys, uint64_t* hashes); + private: static const uint64_t PRIME64_1 = 0x9E3779B185EBCA87ULL; static const uint64_t PRIME64_2 = 0xC2B2AE3D27D4EB4FULL; @@ -175,9 +178,6 @@ class ARROW_EXPORT Hashing64 { static const uint32_t kCombineConst = 0x9e3779b9UL; static const int64_t kStripeSize = 4 * sizeof(uint64_t); - static void HashFixed(bool combine_hashes, uint32_t num_keys, uint64_t length_key, - const uint8_t* keys, uint64_t* hashes); - static void HashVarLen(bool combine_hashes, uint32_t num_rows, const uint32_t* offsets, const uint8_t* concatenated_keys, uint64_t* hashes); diff --git a/cpp/src/arrow/compute/key_hash_test.cc b/cpp/src/arrow/compute/key_hash_test.cc index d030e622641..d10645391b4 100644 --- a/cpp/src/arrow/compute/key_hash_test.cc +++ b/cpp/src/arrow/compute/key_hash_test.cc @@ -19,11 +19,11 @@ #include #include +#include #include #include "arrow/array/builder_binary.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/key_hash.h" +#include "arrow/testing/gtest_util.h" #include "arrow/util/cpu_info.h" #include "arrow/util/pcg_random.h" diff --git a/cpp/src/arrow/compute/key_map.h b/cpp/src/arrow/compute/key_map.h index 4702c5ecc8b..c34e198e543 100644 --- a/cpp/src/arrow/compute/key_map.h +++ b/cpp/src/arrow/compute/key_map.h @@ -36,7 +36,7 @@ namespace compute { // slots, stamps) and operations provided by this class is given in the document: // arrow/compute/exec/doc/key_map.md. // -class SwissTable { +class ARROW_EXPORT SwissTable { friend class SwissTableMerge; public: @@ -76,6 +76,32 @@ class SwissTable { MemoryPool* pool() const { return pool_; } + int log_blocks() const { return log_blocks_; } + + void num_inserted(uint32_t i) { num_inserted_ = i; } + + uint8_t* blocks() const { return blocks_; } + + uint32_t* hashes() const { return hashes_; } + + /// \brief Extract group id for a given slot in a given block. + /// + inline uint64_t extract_group_id(const uint8_t* block_ptr, int slot, + uint64_t group_id_mask) const; + + inline void insert_into_empty_slot(uint32_t slot_id, uint32_t hash, uint32_t group_id); + + static int num_groupid_bits_from_log_blocks(int log_blocks) { + int required_bits = log_blocks + 3; + return required_bits <= 8 ? 8 + : required_bits <= 16 ? 16 + : required_bits <= 32 ? 32 + : 64; + } + + // Use 32-bit hash for now + static constexpr int bits_hash_ = 32; + private: // Lookup helpers @@ -106,10 +132,6 @@ class SwissTable { inline void search_block(uint64_t block, int stamp, int start_slot, int* out_slot, int* out_match_found) const; - /// \brief Extract group id for a given slot in a given block. - /// - inline uint64_t extract_group_id(const uint8_t* block_ptr, int slot, - uint64_t group_id_mask) const; void extract_group_ids(const int num_keys, const uint16_t* optional_selection, const uint32_t* hashes, const uint8_t* local_slots, uint32_t* out_group_ids) const; @@ -160,8 +182,6 @@ class SwissTable { inline bool find_next_stamp_match(const uint32_t hash, const uint32_t in_slot_id, uint32_t* out_slot_id, uint32_t* out_group_id) const; - inline void insert_into_empty_slot(uint32_t slot_id, uint32_t hash, uint32_t group_id); - // Slow processing of input keys in the most generic case. // Handles inserting new keys. // Pre-existing keys will be handled correctly, although the intended use is for this @@ -179,17 +199,6 @@ class SwissTable { // Resize large hash tables when 75% full. Status grow_double(); - static int num_groupid_bits_from_log_blocks(int log_blocks) { - int required_bits = log_blocks + 3; - return required_bits <= 8 ? 8 - : required_bits <= 16 ? 16 - : required_bits <= 32 ? 32 - : 64; - } - - // Use 32-bit hash for now - static constexpr int bits_hash_ = 32; - // Number of hash bits stored in slots in a block. // The highest bits of hash determine block id. // The next set of highest bits is a "stamp" stored in a slot in a block. diff --git a/cpp/src/arrow/compute/row/compare_internal.h b/cpp/src/arrow/compute/row/compare_internal.h index 778485e5c46..638b8c2ec72 100644 --- a/cpp/src/arrow/compute/row/compare_internal.h +++ b/cpp/src/arrow/compute/row/compare_internal.h @@ -30,7 +30,7 @@ namespace arrow { namespace compute { -class KeyCompare { +class ARROW_EXPORT KeyCompare { public: // Returns a single 16-bit selection vector of rows that failed comparison. // If there is input selection on the left, the resulting selection is a filtered image diff --git a/cpp/src/arrow/compute/row/encode_internal.h b/cpp/src/arrow/compute/row/encode_internal.h index bdf38df4fc3..b83767b694c 100644 --- a/cpp/src/arrow/compute/row/encode_internal.h +++ b/cpp/src/arrow/compute/row/encode_internal.h @@ -44,7 +44,7 @@ namespace compute { /// be accessed together, as in the case of hash table key. /// /// Does not support nested types -class RowTableEncoder { +class ARROW_EXPORT RowTableEncoder { public: void Init(const std::vector& cols, int row_alignment, int string_alignment); diff --git a/cpp/src/arrow/compute/util.h b/cpp/src/arrow/compute/util.h index 94d8a4d28d8..d10320e61c2 100644 --- a/cpp/src/arrow/compute/util.h +++ b/cpp/src/arrow/compute/util.h @@ -81,7 +81,7 @@ class MiniBatch { /// Temporary vectors should resemble allocating temporary variables on the stack /// but in the context of vectorized processing where we need to store a vector of /// temporaries instead of a single value. -class TempVectorStack { +class ARROW_EXPORT TempVectorStack { template friend class TempVectorHolder; @@ -139,7 +139,7 @@ class TempVectorHolder { uint32_t num_elements_; }; -class bit_util { +class ARROW_EXPORT bit_util { public: static void bits_to_indexes(int bit_to_search, int64_t hardware_flags, const int num_bits, const uint8_t* bits, int* num_indexes, diff --git a/cpp/src/arrow/dataset/ArrowDatasetConfig.cmake.in b/cpp/src/arrow/dataset/ArrowDatasetConfig.cmake.in index 6816f2c837d..66b0302cbca 100644 --- a/cpp/src/arrow/dataset/ArrowDatasetConfig.cmake.in +++ b/cpp/src/arrow/dataset/ArrowDatasetConfig.cmake.in @@ -28,6 +28,7 @@ include(CMakeFindDependencyMacro) find_dependency(Arrow) +find_dependency(ArrowAcero) find_dependency(Parquet) include("${CMAKE_CURRENT_LIST_DIR}/ArrowDatasetTargets.cmake") diff --git a/cpp/src/arrow/dataset/CMakeLists.txt b/cpp/src/arrow/dataset/CMakeLists.txt index 858542f3548..fa6e1b1deeb 100644 --- a/cpp/src/arrow/dataset/CMakeLists.txt +++ b/cpp/src/arrow/dataset/CMakeLists.txt @@ -34,7 +34,7 @@ set(ARROW_DATASET_SRCS if(ARROW_PARQUET) set(ARROW_DATASET_PKG_CONFIG_REQUIRES parquet) else() - set(ARROW_DATASET_PKG_CONFIG_REQUIRES arrow) + set(ARROW_DATASET_PKG_CONFIG_REQUIRES arrow arrow_acero) endif() set(ARROW_DATASET_STATIC_LINK_LIBS) @@ -66,8 +66,12 @@ else() list(APPEND ARROW_DATASET_SHARED_INSTALL_INTERFACE_LIBS Arrow::arrow_shared) endif() -list(APPEND ARROW_DATASET_STATIC_LINK_LIBS arrow_static ${ARROW_STATIC_LINK_LIBS}) -list(APPEND ARROW_DATASET_SHARED_LINK_LIBS arrow_shared) +list(APPEND + ARROW_DATASET_STATIC_LINK_LIBS + arrow_static + arrow_acero_static + ${ARROW_STATIC_LINK_LIBS}) +list(APPEND ARROW_DATASET_SHARED_LINK_LIBS arrow_shared arrow_acero_shared) add_arrow_lib(arrow_dataset CMAKE_PACKAGE_NAME @@ -135,6 +139,8 @@ function(ADD_ARROW_DATASET_TEST REL_TEST_NAME) ${ARROW_DATASET_TEST_LINK_LIBS} PREFIX ${PREFIX} + SOURCES + test_util_internal.cc LABELS ${LABELS} ${ARG_UNPARSED_ARGUMENTS}) diff --git a/cpp/src/arrow/dataset/dataset.cc b/cpp/src/arrow/dataset/dataset.cc index 6f3787b886c..0e7bc3da842 100644 --- a/cpp/src/arrow/dataset/dataset.cc +++ b/cpp/src/arrow/dataset/dataset.cc @@ -18,7 +18,7 @@ #include #include -#include "arrow/compute/exec/util.h" +#include "arrow/acero/util.h" #include "arrow/dataset/dataset.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/scanner.h" diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index 29e9ab1f0c4..aada6ad8657 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -17,7 +17,7 @@ #include "arrow/dataset/file_base.h" -#include +#include "arrow/acero/exec_plan.h" #include #include @@ -26,12 +26,12 @@ #include #include +#include "arrow/acero/forest_internal.h" +#include "arrow/acero/map_node.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/subtree_internal.h" +#include "arrow/acero/util.h" #include "arrow/compute/api_scalar.h" -#include "arrow/compute/exec/forest_internal.h" -#include "arrow/compute/exec/map_node.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/subtree_internal.h" -#include "arrow/compute/exec/util.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/dataset_writer.h" #include "arrow/dataset/scanner.h" @@ -211,7 +211,7 @@ bool FileFragment::Equals(const FileFragment& other) const { struct FileSystemDataset::FragmentSubtrees { // Forest for skipping fragments based on extracted subtree expressions - compute::Forest forest; + acero::Forest forest; // fragment indices and subtree expressions in forest order std::vector> fragments_and_subtrees; }; @@ -268,13 +268,13 @@ std::string FileSystemDataset::ToString() const { void FileSystemDataset::SetupSubtreePruning() { subtrees_ = std::make_shared(); - compute::SubtreeImpl impl; + acero::SubtreeImpl impl; auto encoded = impl.EncodeGuarantees( [&](int index) { return fragments_[index]->partition_expression(); }, static_cast(fragments_.size())); - std::sort(encoded.begin(), encoded.end(), compute::SubtreeImpl::ByGuarantee()); + std::sort(encoded.begin(), encoded.end(), acero::SubtreeImpl::ByGuarantee()); for (const auto& e : encoded) { if (e.index) { @@ -284,8 +284,8 @@ void FileSystemDataset::SetupSubtreePruning() { } } - subtrees_->forest = compute::Forest(static_cast(encoded.size()), - compute::SubtreeImpl::IsAncestor{encoded}); + subtrees_->forest = acero::Forest(static_cast(encoded.size()), + acero::SubtreeImpl::IsAncestor{encoded}); } Result FileSystemDataset::GetFragmentsImpl( @@ -299,7 +299,7 @@ Result FileSystemDataset::GetFragmentsImpl( std::vector predicates{predicate}; RETURN_NOT_OK(subtrees_->forest.Visit( - [&](compute::Forest::Ref ref) -> Result { + [&](acero::Forest::Ref ref) -> Result { if (auto fragment_index = std::get_if(&subtrees_->fragments_and_subtrees[ref.i])) { fragment_indices.push_back(*fragment_index); @@ -318,7 +318,7 @@ Result FileSystemDataset::GetFragmentsImpl( predicates.push_back(std::move(simplified)); return true; }, - [&](compute::Forest::Ref ref) { predicates.pop_back(); })); + [&](acero::Forest::Ref ref) { predicates.pop_back(); })); std::sort(fragment_indices.begin(), fragment_indices.end()); @@ -385,7 +385,7 @@ Status WriteBatch( return Status::OK(); } -class DatasetWritingSinkNodeConsumer : public compute::SinkNodeConsumer { +class DatasetWritingSinkNodeConsumer : public acero::SinkNodeConsumer { public: DatasetWritingSinkNodeConsumer(std::shared_ptr custom_metadata, FileSystemDatasetWriteOptions write_options) @@ -393,8 +393,8 @@ class DatasetWritingSinkNodeConsumer : public compute::SinkNodeConsumer { write_options_(std::move(write_options)) {} Status Init(const std::shared_ptr& schema, - compute::BackpressureControl* backpressure_control, - compute::ExecPlan* plan) override { + acero::BackpressureControl* backpressure_control, + acero::ExecPlan* plan) override { if (custom_metadata_) { schema_ = schema->WithMetadata(custom_metadata_); } else { @@ -455,19 +455,19 @@ Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_optio // when reading from a single input file. const auto& custom_metadata = scanner->options()->projected_schema->metadata(); - compute::Declaration plan = compute::Declaration::Sequence({ + acero::Declaration plan = acero::Declaration::Sequence({ {"scan", ScanNodeOptions{dataset, scanner->options()}}, - {"filter", compute::FilterNodeOptions{scanner->options()->filter}}, - {"project", compute::ProjectNodeOptions{std::move(exprs), std::move(names)}}, + {"filter", acero::FilterNodeOptions{scanner->options()->filter}}, + {"project", acero::ProjectNodeOptions{std::move(exprs), std::move(names)}}, {"write", WriteNodeOptions{write_options, custom_metadata}}, }); - return compute::DeclarationToStatus(std::move(plan), scanner->options()->use_threads); + return acero::DeclarationToStatus(std::move(plan), scanner->options()->use_threads); } -Result MakeWriteNode(compute::ExecPlan* plan, - std::vector inputs, - const compute::ExecNodeOptions& options) { +Result MakeWriteNode(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { if (inputs.size() != 1) { return Status::Invalid("Write SinkNode requires exactly 1 input, got ", inputs.size()); @@ -488,17 +488,17 @@ Result MakeWriteNode(compute::ExecPlan* plan, ARROW_ASSIGN_OR_RAISE( auto node, - compute::MakeExecNode("consuming_sink", plan, std::move(inputs), - compute::ConsumingSinkNodeOptions{std::move(consumer)})); + acero::MakeExecNode("consuming_sink", plan, std::move(inputs), + acero::ConsumingSinkNodeOptions{std::move(consumer)})); return node; } namespace { -class TeeNode : public compute::MapNode { +class TeeNode : public acero::MapNode { public: - TeeNode(compute::ExecPlan* plan, std::vector inputs, + TeeNode(acero::ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema, FileSystemDatasetWriteOptions write_options) : MapNode(plan, std::move(inputs), std::move(output_schema)), @@ -513,9 +513,9 @@ class TeeNode : public compute::MapNode { return MapNode::StartProducing(); } - static Result Make(compute::ExecPlan* plan, - std::vector inputs, - const compute::ExecNodeOptions& options) { + static Result Make(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "TeeNode")); const WriteNodeOptions write_node_options = @@ -567,7 +567,7 @@ class TeeNode : public compute::MapNode { } // namespace namespace internal { -void InitializeDatasetWriter(arrow::compute::ExecFactoryRegistry* registry) { +void InitializeDatasetWriter(arrow::acero::ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("write", MakeWriteNode)); DCHECK_OK(registry->AddFactory("tee", TeeNode::Make)); } diff --git a/cpp/src/arrow/dataset/file_base.h b/cpp/src/arrow/dataset/file_base.h index 2b8421ce16b..ca28cb1e4ad 100644 --- a/cpp/src/arrow/dataset/file_base.h +++ b/cpp/src/arrow/dataset/file_base.h @@ -455,7 +455,7 @@ struct ARROW_DS_EXPORT FileSystemDatasetWriteOptions { }; /// \brief Wraps FileSystemDatasetWriteOptions for consumption as compute::ExecNodeOptions -class ARROW_DS_EXPORT WriteNodeOptions : public compute::ExecNodeOptions { +class ARROW_DS_EXPORT WriteNodeOptions : public acero::ExecNodeOptions { public: explicit WriteNodeOptions( FileSystemDatasetWriteOptions options, @@ -471,8 +471,7 @@ class ARROW_DS_EXPORT WriteNodeOptions : public compute::ExecNodeOptions { /// @} namespace internal { -ARROW_DS_EXPORT void InitializeDatasetWriter( - arrow::compute::ExecFactoryRegistry* registry); +ARROW_DS_EXPORT void InitializeDatasetWriter(arrow::acero::ExecFactoryRegistry* registry); } } // namespace dataset diff --git a/cpp/src/arrow/dataset/file_test.cc b/cpp/src/arrow/dataset/file_test.cc index 914bfa28b5e..79c11fdffd4 100644 --- a/cpp/src/arrow/dataset/file_test.cc +++ b/cpp/src/arrow/dataset/file_test.cc @@ -25,7 +25,6 @@ #include #include "arrow/array/array_primitive.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/dataset/api.h" #include "arrow/dataset/partition.h" #include "arrow/dataset/plan.h" diff --git a/cpp/src/arrow/dataset/plan.cc b/cpp/src/arrow/dataset/plan.cc index 805fb25aa71..fe47e2c483a 100644 --- a/cpp/src/arrow/dataset/plan.cc +++ b/cpp/src/arrow/dataset/plan.cc @@ -17,7 +17,7 @@ #include "arrow/dataset/plan.h" -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/exec_plan.h" #include "arrow/dataset/file_base.h" #include "arrow/dataset/scanner.h" @@ -30,7 +30,7 @@ namespace internal { void Initialize() { static std::once_flag flag; std::call_once(flag, [] { - auto registry = compute::default_exec_factory_registry(); + auto registry = acero::default_exec_factory_registry(); if (registry) { InitializeScanner(registry); InitializeScannerV2(registry); diff --git a/cpp/src/arrow/dataset/scan_node.cc b/cpp/src/arrow/dataset/scan_node.cc index 4f574930be0..46cc1dc6c8c 100644 --- a/cpp/src/arrow/dataset/scan_node.cc +++ b/cpp/src/arrow/dataset/scan_node.cc @@ -22,9 +22,9 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/query_context.h" -#include "arrow/compute/exec/util.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/query_context.h" +#include "arrow/acero/util.h" #include "arrow/compute/expression.h" #include "arrow/compute/expression_internal.h" #include "arrow/dataset/scanner.h" @@ -40,8 +40,6 @@ using namespace std::string_view_literals; // NOLINT -namespace cp = arrow::compute; - namespace arrow { using internal::checked_cast; @@ -57,8 +55,8 @@ Result> OutputSchemaFromOptions(const ScanV2Options& opt // In the future we should support async scanning of fragments. The // Dataset class doesn't support this yet but we pretend it does here to // ease future adoption of the feature. -Future>> GetFragments(Dataset* dataset, - cp::Expression predicate) { +Future>> GetFragments( + Dataset* dataset, compute::Expression predicate) { // In the future the dataset should be responsible for figuring out // the I/O context. This will allow different I/O contexts to be used // when scanning different datasets. For example, if we are scanning a @@ -121,12 +119,12 @@ Future>> GetFragments(Dataset* dataset, /// fragments. On destruction we continue consuming the fragments until they complete /// (which should be fairly quick since we cancelled the fragment). This ensures the /// I/O work is completely finished before the node is destroyed. -class ScanNode : public cp::ExecNode, public cp::TracedNode { +class ScanNode : public acero::ExecNode, public acero::TracedNode { public: - ScanNode(cp::ExecPlan* plan, ScanV2Options options, + ScanNode(acero::ExecPlan* plan, ScanV2Options options, std::shared_ptr output_schema) - : cp::ExecNode(plan, {}, {}, std::move(output_schema)), - cp::TracedNode(this), + : acero::ExecNode(plan, {}, {}, std::move(output_schema)), + acero::TracedNode(this), options_(options) {} static Result NormalizeAndValidate(const ScanV2Options& options, @@ -170,8 +168,9 @@ class ScanNode : public cp::ExecNode, public cp::TracedNode { return std::move(normalized); } - static Result Make(cp::ExecPlan* plan, std::vector inputs, - const cp::ExecNodeOptions& options) { + static Result Make(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 0, "ScanNode")); const auto& scan_options = checked_cast(options); ARROW_ASSIGN_OR_RAISE( @@ -188,8 +187,10 @@ class ScanNode : public cp::ExecNode, public cp::TracedNode { [[noreturn]] static void NoInputs() { Unreachable("no inputs; this should never be called"); } - [[noreturn]] Status InputReceived(cp::ExecNode*, cp::ExecBatch) override { NoInputs(); } - [[noreturn]] Status InputFinished(cp::ExecNode*, int) override { NoInputs(); } + [[noreturn]] Status InputReceived(acero::ExecNode*, compute::ExecBatch) override { + NoInputs(); + } + [[noreturn]] Status InputFinished(acero::ExecNode*, int) override { NoInputs(); } Status Init() override { return Status::OK(); } @@ -468,7 +469,7 @@ class ScanNode : public cp::ExecNode, public cp::TracedNode { } // namespace namespace internal { -void InitializeScannerV2(arrow::compute::ExecFactoryRegistry* registry) { +void InitializeScannerV2(arrow::acero::ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("scan2", ScanNode::Make)); } } // namespace internal diff --git a/cpp/src/arrow/dataset/scanner.cc b/cpp/src/arrow/dataset/scanner.cc index fcefeda582b..18981d14519 100644 --- a/cpp/src/arrow/dataset/scanner.cc +++ b/cpp/src/arrow/dataset/scanner.cc @@ -25,15 +25,15 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/query_context.h" #include "arrow/array/array_primitive.h" #include "arrow/array/util.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/query_context.h" #include "arrow/dataset/dataset.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/plan.h" @@ -431,11 +431,11 @@ Result AsyncScanner::ScanBatchesUnorderedAsync( auto exec_context = std::make_shared(scan_options_->pool, cpu_executor); - compute::QueryOptions query_options; + acero::QueryOptions query_options; query_options.use_legacy_batching = use_legacy_batching; ARROW_ASSIGN_OR_RAISE(auto plan, - compute::ExecPlan::Make(query_options, *exec_context.get())); + acero::ExecPlan::Make(query_options, *exec_context.get())); AsyncGenerator> sink_gen; auto exprs = scan_options_->projection.call()->arguments; @@ -444,14 +444,14 @@ Result AsyncScanner::ScanBatchesUnorderedAsync( ->field_names; RETURN_NOT_OK( - compute::Declaration::Sequence( + acero::Declaration::Sequence( { {"scan", ScanNodeOptions{dataset_, scan_options_, sequence_fragments}}, - {"filter", compute::FilterNodeOptions{scan_options_->filter}}, + {"filter", acero::FilterNodeOptions{scan_options_->filter}}, {"augmented_project", - compute::ProjectNodeOptions{std::move(exprs), std::move(names)}}, - {"sink", compute::SinkNodeOptions{&sink_gen, /*schema=*/nullptr, - scan_options_->backpressure}}, + acero::ProjectNodeOptions{std::move(exprs), std::move(names)}}, + {"sink", acero::SinkNodeOptions{&sink_gen, /*schema=*/nullptr, + scan_options_->backpressure}}, }) .AddToPlan(plan.get())); @@ -725,7 +725,7 @@ Future AsyncScanner::CountRowsAsync(Executor* 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, acero::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, @@ -753,16 +753,16 @@ Future AsyncScanner::CountRowsAsync(Executor* executor) { }); }); - compute::Declaration count_plan = compute::Declaration::Sequence( + acero::Declaration count_plan = acero::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{ + {"project", acero::ProjectNodeOptions{{options->filter}, {"mask"}}}, + {"aggregate", acero::AggregateNodeOptions{{compute::Aggregate{ "sum", nullptr, "mask", "selected_count"}}}}}); - return compute::DeclarationToBatchesAsync(std::move(count_plan), exec_context) + return acero::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, @@ -964,7 +964,7 @@ Status ScannerBuilder::FragmentScanOptions( return Status::OK(); } -Status ScannerBuilder::Backpressure(compute::BackpressureOptions backpressure) { +Status ScannerBuilder::Backpressure(acero::BackpressureOptions backpressure) { scan_options_->backpressure = backpressure; return Status::OK(); } @@ -984,9 +984,9 @@ Result> ScannerBuilder::Finish() { namespace { -Result MakeScanNode(compute::ExecPlan* plan, - std::vector inputs, - const compute::ExecNodeOptions& options) { +Result MakeScanNode(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { const auto& scan_node_options = checked_cast(options); auto scan_options = scan_node_options.scan_options; auto dataset = scan_node_options.dataset; @@ -1056,15 +1056,15 @@ Result MakeScanNode(compute::ExecPlan* plan, fields.push_back(aug_field); } - return compute::MakeExecNode( + return acero::MakeExecNode( "source", plan, {}, - compute::SourceNodeOptions{schema(std::move(fields)), std::move(gen)}); + acero::SourceNodeOptions{schema(std::move(fields)), std::move(gen)}); } -Result MakeAugmentedProjectNode( - compute::ExecPlan* plan, std::vector inputs, - const compute::ExecNodeOptions& options) { - const auto& project_options = checked_cast(options); +Result MakeAugmentedProjectNode(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { + const auto& project_options = checked_cast(options); auto exprs = project_options.expressions; auto names = project_options.names; @@ -1079,14 +1079,14 @@ Result MakeAugmentedProjectNode( exprs.push_back(compute::field_ref(aug_field->name())); names.push_back(aug_field->name()); } - return compute::MakeExecNode( + return acero::MakeExecNode( "project", plan, std::move(inputs), - compute::ProjectNodeOptions{std::move(exprs), std::move(names)}); + acero::ProjectNodeOptions{std::move(exprs), std::move(names)}); } -Result MakeOrderedSinkNode(compute::ExecPlan* plan, - std::vector inputs, - const compute::ExecNodeOptions& options) { +Result MakeOrderedSinkNode(acero::ExecPlan* plan, + std::vector inputs, + const acero::ExecNodeOptions& options) { if (inputs.size() != 1) { return Status::Invalid("Ordered SinkNode requires exactly 1 input, got ", inputs.size()); @@ -1095,8 +1095,8 @@ Result MakeOrderedSinkNode(compute::ExecPlan* plan, AsyncGenerator> unordered; ARROW_ASSIGN_OR_RAISE(auto node, - compute::MakeExecNode("sink", plan, std::move(inputs), - compute::SinkNodeOptions{&unordered})); + acero::MakeExecNode("sink", plan, std::move(inputs), + acero::SinkNodeOptions{&unordered})); const Schema& schema = *input->output_schema(); ARROW_ASSIGN_OR_RAISE(FieldPath match, FieldRef("__fragment_index").FindOne(schema)); @@ -1156,7 +1156,7 @@ Result MakeOrderedSinkNode(compute::ExecPlan* plan, last_in_fragment(*prev) && batch_index(*next) == 0; }; - const auto& sink_options = checked_cast(options); + const auto& sink_options = checked_cast(options); *sink_options.generator = MakeSequencingGenerator(std::move(unordered), left_after_right, is_next, std::make_optional(std::move(before_any))); @@ -1167,7 +1167,7 @@ Result MakeOrderedSinkNode(compute::ExecPlan* plan, } // namespace namespace internal { -void InitializeScanner(arrow::compute::ExecFactoryRegistry* registry) { +void InitializeScanner(arrow::acero::ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("scan", MakeScanNode)); DCHECK_OK(registry->AddFactory("ordered_sink", MakeOrderedSinkNode)); DCHECK_OK(registry->AddFactory("augmented_project", MakeAugmentedProjectNode)); diff --git a/cpp/src/arrow/dataset/scanner.h b/cpp/src/arrow/dataset/scanner.h index 89bd3c65627..5479a0d9db4 100644 --- a/cpp/src/arrow/dataset/scanner.h +++ b/cpp/src/arrow/dataset/scanner.h @@ -25,7 +25,7 @@ #include #include -#include "arrow/compute/exec/options.h" +#include "arrow/acero/options.h" #include "arrow/compute/expression.h" #include "arrow/compute/type_fwd.h" #include "arrow/dataset/dataset.h" @@ -134,8 +134,8 @@ struct ARROW_DS_EXPORT ScanOptions { std::vector MaterializedFields() const; /// Parameters which control when the plan should pause for a slow consumer - compute::BackpressureOptions backpressure = - compute::BackpressureOptions::DefaultBackpressure(); + acero::BackpressureOptions backpressure = + acero::BackpressureOptions::DefaultBackpressure(); }; /// Scan-specific options, which can be changed between scans of the same dataset. @@ -151,7 +151,7 @@ struct ARROW_DS_EXPORT ScanOptions { /// schema. This is sometimes referred to as the physical or fragment schema. /// Conversion from the fragment schema to the dataset schema is a process /// known as evolution. -struct ARROW_DS_EXPORT ScanV2Options : public compute::ExecNodeOptions { +struct ARROW_DS_EXPORT ScanV2Options : public acero::ExecNodeOptions { explicit ScanV2Options(std::shared_ptr dataset) : dataset(std::move(dataset)) {} @@ -533,7 +533,7 @@ class ARROW_DS_EXPORT ScannerBuilder { Status FragmentScanOptions(std::shared_ptr fragment_scan_options); /// \brief Override default backpressure configuration - Status Backpressure(compute::BackpressureOptions backpressure); + Status Backpressure(acero::BackpressureOptions backpressure); /// \brief Return the current scan options for the builder. Result> GetScanOptions(); @@ -554,7 +554,7 @@ class ARROW_DS_EXPORT ScannerBuilder { /// Does not construct associated filter or project nodes. /// Yielded batches will be augmented with fragment/batch indices to enable stable /// ordering for simple ExecPlans. -class ARROW_DS_EXPORT ScanNodeOptions : public compute::ExecNodeOptions { +class ARROW_DS_EXPORT ScanNodeOptions : public acero::ExecNodeOptions { public: explicit ScanNodeOptions(std::shared_ptr dataset, std::shared_ptr scan_options, @@ -571,8 +571,8 @@ class ARROW_DS_EXPORT ScanNodeOptions : public compute::ExecNodeOptions { /// @} namespace internal { -ARROW_DS_EXPORT void InitializeScanner(arrow::compute::ExecFactoryRegistry* registry); -ARROW_DS_EXPORT void InitializeScannerV2(arrow::compute::ExecFactoryRegistry* registry); +ARROW_DS_EXPORT void InitializeScanner(arrow::acero::ExecFactoryRegistry* registry); +ARROW_DS_EXPORT void InitializeScannerV2(arrow::acero::ExecFactoryRegistry* registry); } // namespace internal } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/scanner_benchmark.cc b/cpp/src/arrow/dataset/scanner_benchmark.cc index 2ad934d9a60..be953b35559 100644 --- a/cpp/src/arrow/dataset/scanner_benchmark.cc +++ b/cpp/src/arrow/dataset/scanner_benchmark.cc @@ -17,10 +17,9 @@ #include "benchmark/benchmark.h" +#include "arrow/acero/options.h" #include "arrow/api.h" #include "arrow/compute/api.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/dataset/dataset.h" #include "arrow/dataset/plan.h" #include "arrow/dataset/scanner.h" @@ -30,6 +29,9 @@ #include "arrow/testing/random.h" namespace arrow { + +using dataset::BatchesWithSchema; + namespace compute { constexpr auto kSeed = 0x0ff1ce; @@ -98,7 +100,7 @@ size_t GetBytesForSchema() { return sizeof(int32_t) + sizeof(bool); } void MinimalEndToEndScan( size_t num_batches, size_t batch_size, const std::string& factory_name, - std::function>(size_t, size_t)> + std::function>(size_t, size_t)> options_factory) { // ensure arrow::dataset node factories are in the registry ::arrow::dataset::internal::Initialize(); @@ -112,35 +114,35 @@ void MinimalEndToEndScan( std::shared_ptr dataset = std::make_shared(GetSchema(), batches); - ASSERT_OK_AND_ASSIGN(std::shared_ptr node_options, + ASSERT_OK_AND_ASSIGN(std::shared_ptr node_options, options_factory(num_batches, batch_size)); // construct the scan node - compute::Declaration scan(factory_name, std::move(node_options)); + acero::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)); - compute::Declaration filter("filter", {std::move(scan)}, - compute::FilterNodeOptions{b_is_true}); + acero::Declaration filter("filter", {std::move(scan)}, + acero::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)}); - compute::Declaration project("project", {std::move(filter)}, - compute::ProjectNodeOptions{{a_times_2}, {"a*2"}}); + acero::Declaration project("project", {std::move(filter)}, + acero::ProjectNodeOptions{{a_times_2}, {"a*2"}}); // Consume the plan and transform into a table ASSERT_OK_AND_ASSIGN(std::shared_ptr
collected, - compute::DeclarationToTable(std::move(project))); + acero::DeclarationToTable(std::move(project))); ASSERT_GT(collected->num_rows(), 0); } void ScanOnly( size_t num_batches, size_t batch_size, const std::string& factory_name, - std::function>(size_t, size_t)> + std::function>(size_t, size_t)> options_factory) { // ensure arrow::dataset node factories are in the registry ::arrow::dataset::internal::Initialize(); @@ -150,14 +152,14 @@ void ScanOnly( std::shared_ptr dataset = std::make_shared(GetSchema(), batches); - ASSERT_OK_AND_ASSIGN(std::shared_ptr node_options, + ASSERT_OK_AND_ASSIGN(std::shared_ptr node_options, options_factory(num_batches, batch_size)); // construct the plan - compute::Declaration scan(factory_name, std::move(node_options)); + acero::Declaration scan(factory_name, std::move(node_options)); ASSERT_OK_AND_ASSIGN(std::shared_ptr
collected, - compute::DeclarationToTable(std::move(scan))); + acero::DeclarationToTable(std::move(scan))); ASSERT_GT(collected->num_rows(), 0); ASSERT_EQ(collected->num_columns(), 2); @@ -166,7 +168,7 @@ void ScanOnly( static constexpr int kScanIdx = 0; static constexpr int kScanV2Idx = 1; -const std::function>(size_t, size_t)> +const std::function>(size_t, size_t)> kScanFactory = [](size_t num_batches, size_t batch_size) { RecordBatchVector batches = GetBatches(num_batches, batch_size); std::shared_ptr dataset = @@ -182,10 +184,10 @@ const std::function>(size_t, si return std::make_shared(std::move(dataset), std::move(options)); }; -const std::function>(size_t, size_t)> +const std::function>(size_t, size_t)> kScanV2Factory = [](size_t num_batches, - size_t batch_size) -> Result> { + size_t batch_size) -> Result> { RecordBatchVector batches = GetBatches(num_batches, batch_size); std::shared_ptr sch = GetSchema(); std::shared_ptr dataset = @@ -204,7 +206,7 @@ static void MinimalEndToEndBench(benchmark::State& state) { size_t num_batches = state.range(0); size_t batch_size = state.range(1); - std::function>(size_t, size_t)> + std::function>(size_t, size_t)> options_factory; std::string scan_factory = "scan"; if (state.range(2) == kScanIdx) { @@ -227,7 +229,7 @@ static void ScanOnlyBench(benchmark::State& state) { size_t num_batches = state.range(0); size_t batch_size = state.range(1); - std::function>(size_t, size_t)> + std::function>(size_t, size_t)> options_factory; std::string scan_factory = "scan"; if (state.range(2) == kScanIdx) { diff --git a/cpp/src/arrow/dataset/scanner_test.cc b/cpp/src/arrow/dataset/scanner_test.cc index 0a15eb87fd5..b0dd9bfede3 100644 --- a/cpp/src/arrow/dataset/scanner_test.cc +++ b/cpp/src/arrow/dataset/scanner_test.cc @@ -23,12 +23,11 @@ #include +#include "arrow/acero/exec_plan.h" #include "arrow/compute/api.h" #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/test_util.h" #include "arrow/compute/expression_internal.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/plan.h" diff --git a/cpp/src/arrow/dataset/test_util_internal.cc b/cpp/src/arrow/dataset/test_util_internal.cc new file mode 100644 index 00000000000..88b514b2e5c --- /dev/null +++ b/cpp/src/arrow/dataset/test_util_internal.cc @@ -0,0 +1,218 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/dataset/test_util_internal.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" +#include "arrow/compute/api_vector.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/function_internal.h" +#include "arrow/datum.h" +#include "arrow/io/interfaces.h" +#include "arrow/record_batch.h" +#include "arrow/table.h" +#include "arrow/testing/builder.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/random.h" +#include "arrow/type.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/iterator.h" +#include "arrow/util/logging.h" +#include "arrow/util/unreachable.h" +#include "arrow/util/vector.h" + +namespace arrow { + +using acero::TableFromExecBatches; +using compute::SortKey; +using compute::SortOptions; +using compute::Take; +using internal::Executor; + +namespace dataset { +namespace { + +struct DummyNode : ExecNode { + DummyNode(ExecPlan* plan, NodeVector inputs, bool is_sink, + StartProducingFunc start_producing, StopProducingFunc stop_producing) + : ExecNode(plan, std::move(inputs), {}, (is_sink) ? nullptr : dummy_schema()), + start_producing_(std::move(start_producing)), + stop_producing_(std::move(stop_producing)) { + input_labels_.resize(inputs_.size()); + for (size_t i = 0; i < input_labels_.size(); ++i) { + input_labels_[i] = std::to_string(i); + } + } + + const char* kind_name() const override { return "Dummy"; } + + Status InputReceived(ExecNode* input, ExecBatch batch) override { return Status::OK(); } + + Status InputFinished(ExecNode* input, int total_batches) override { + return Status::OK(); + } + + Status StartProducing() override { + if (start_producing_) { + RETURN_NOT_OK(start_producing_(this)); + } + started_ = true; + return Status::OK(); + } + + void PauseProducing(ExecNode* output, int32_t counter) override { + ASSERT_NE(output_, nullptr) << "Sink nodes should not experience backpressure"; + AssertIsOutput(output); + } + + void ResumeProducing(ExecNode* output, int32_t counter) override { + ASSERT_NE(output_, nullptr) << "Sink nodes should not experience backpressure"; + AssertIsOutput(output); + } + + Status StopProducingImpl() override { + if (stop_producing_) { + stop_producing_(this); + } + return Status::OK(); + } + + private: + void AssertIsOutput(ExecNode* output) { ASSERT_EQ(output->output(), nullptr); } + + std::shared_ptr dummy_schema() const { + return schema({field("dummy", null())}); + } + + StartProducingFunc start_producing_; + StopProducingFunc stop_producing_; + std::unordered_set requested_stop_; + bool started_ = false; +}; + +} // namespace + +ExecBatch ExecBatchFromJSON(const std::vector& types, std::string_view json) { + auto fields = ::arrow::internal::MapVector( + [](const TypeHolder& th) { return field("", th.GetSharedPtr()); }, types); + + ExecBatch batch{*RecordBatchFromJSON(schema(std::move(fields)), json)}; + + return batch; +} + +ExecBatch ExecBatchFromJSON(const std::vector& types, + const std::vector& shapes, std::string_view json) { + DCHECK_EQ(types.size(), shapes.size()); + + ExecBatch batch = ExecBatchFromJSON(types, json); + + auto value_it = batch.values.begin(); + for (ArgShape shape : shapes) { + if (shape == ArgShape::SCALAR) { + if (batch.length == 0) { + *value_it = MakeNullScalar(value_it->type()); + } else { + *value_it = value_it->make_array()->GetScalar(0).ValueOrDie(); + } + } + ++value_it; + } + + return batch; +} + +Future<> StartAndFinish(ExecPlan* plan) { + RETURN_NOT_OK(plan->Validate()); + plan->StartProducing(); + return plan->finished(); +} + +Future> StartAndCollect( + ExecPlan* plan, AsyncGenerator> gen) { + RETURN_NOT_OK(plan->Validate()); + plan->StartProducing(); + + auto collected_fut = CollectAsyncGenerator(gen); + + return AllFinished({plan->finished(), Future<>(collected_fut)}) + .Then([collected_fut]() -> Result> { + ARROW_ASSIGN_OR_RAISE(auto collected, collected_fut.result()); + return ::arrow::internal::MapVector( + [](std::optional batch) { return batch.value_or(ExecBatch()); }, + std::move(collected)); + }); +} + +Result> SortTableOnAllFields(const std::shared_ptr
& tab) { + std::vector sort_keys; + 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 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()); + } else { + ASSERT_OK_AND_ASSIGN(auto exp_sorted, SortTableOnAllFields(exp)); + ASSERT_OK_AND_ASSIGN(auto act_sorted, SortTableOnAllFields(act)); + + AssertTablesEqual(*exp_sorted, *act_sorted, + /*same_chunk_layout=*/false, /*flatten=*/true); + } +} + +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)); + AssertTablesEqualIgnoringOrder(exp_tab, act_tab); +} + +void AssertExecBatchesEqual(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); +} + +} // namespace dataset +} // namespace arrow diff --git a/cpp/src/arrow/dataset/test_util_internal.h b/cpp/src/arrow/dataset/test_util_internal.h index 589c5e29c54..4488ee88be6 100644 --- a/cpp/src/arrow/dataset/test_util_internal.h +++ b/cpp/src/arrow/dataset/test_util_internal.h @@ -31,9 +31,11 @@ #include #include +#include "arrow/acero/exec_plan.h" #include "arrow/array.h" -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/compute/exec.h" #include "arrow/compute/expression.h" +#include "arrow/compute/kernel.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/discovery.h" #include "arrow/dataset/file_base.h" @@ -48,20 +50,92 @@ #include "arrow/testing/gtest_util.h" #include "arrow/testing/matchers.h" #include "arrow/testing/random.h" +#include "arrow/testing/visibility.h" #include "arrow/util/async_generator.h" #include "arrow/util/io_util.h" #include "arrow/util/iterator.h" #include "arrow/util/logging.h" +#include "arrow/util/pcg_random.h" #include "arrow/util/thread_pool.h" +#include "arrow/util/vector.h" namespace arrow { +using acero::ExecNode; +using acero::ExecPlan; +using compute::ExecBatch; using internal::checked_cast; using internal::checked_pointer_cast; using internal::TemporaryDir; namespace dataset { +using StartProducingFunc = std::function; +using StopProducingFunc = std::function; + +ExecBatch ExecBatchFromJSON(const std::vector& types, std::string_view json); + +/// \brief Shape qualifier for value types. In certain instances +/// (e.g. "map_lookup" kernel), an argument may only be a scalar, where in +/// other kernels arguments can be arrays or scalars +enum class ArgShape { ANY, ARRAY, SCALAR }; + +ExecBatch ExecBatchFromJSON(const std::vector& types, + const std::vector& shapes, std::string_view json); + +struct BatchesWithSchema { + std::vector batches; + std::shared_ptr schema; + + AsyncGenerator> gen(bool parallel, bool slow) const { + auto opt_batches = ::arrow::internal::MapVector( + [](ExecBatch batch) { return std::make_optional(std::move(batch)); }, batches); + + AsyncGenerator> gen; + + if (parallel) { + // emulate batches completing initial decode-after-scan on a cpu thread + gen = MakeBackgroundGenerator(MakeVectorIterator(std::move(opt_batches)), + ::arrow::internal::GetCpuThreadPool()) + .ValueOrDie(); + + // ensure that callbacks are not executed immediately on a background thread + gen = + MakeTransferredGenerator(std::move(gen), ::arrow::internal::GetCpuThreadPool()); + } else { + gen = MakeVectorGenerator(std::move(opt_batches)); + } + + if (slow) { + gen = + MakeMappedGenerator(std::move(gen), [](const std::optional& batch) { + SleepABit(); + return batch; + }); + } + + return gen; + } +}; + +Future<> StartAndFinish(ExecPlan* plan); + +Future> StartAndCollect( + ExecPlan* plan, AsyncGenerator> gen); + +Result> SortTableOnAllFields(const std::shared_ptr
& tab); + +void AssertTablesEqualIgnoringOrder(const std::shared_ptr
& exp, + const std::shared_ptr
& act); + +void AssertExecBatchesEqualIgnoringOrder(const std::shared_ptr& schema, + const std::vector& exp, + const std::vector& act); + +void AssertExecBatchesEqual(const std::shared_ptr& schema, + const std::vector& exp, + const std::vector& act); + using compute::call; using compute::field_ref; using compute::literal; @@ -1189,10 +1263,9 @@ class FileFormatScanNodeMixin : public FileFormatFixtureMixinV2, ARROW_RETURN_NOT_OK(ScanV2Options::AddFieldsNeededForFilter(opts_.get())); } opts_->format_options = GetFormatOptions(); - ARROW_ASSIGN_OR_RAISE( - std::unique_ptr reader, - compute::DeclarationToReader(compute::Declaration("scan2", *opts_), - GetParam().use_threads)); + ARROW_ASSIGN_OR_RAISE(std::unique_ptr reader, + acero::DeclarationToReader(acero::Declaration("scan2", *opts_), + GetParam().use_threads)); return reader; } diff --git a/cpp/src/arrow/engine/ArrowSubstraitConfig.cmake.in b/cpp/src/arrow/engine/ArrowSubstraitConfig.cmake.in index 2e96d372ad7..932c52ac65f 100644 --- a/cpp/src/arrow/engine/ArrowSubstraitConfig.cmake.in +++ b/cpp/src/arrow/engine/ArrowSubstraitConfig.cmake.in @@ -28,6 +28,7 @@ include(CMakeFindDependencyMacro) find_dependency(Arrow) +find_dependency(ArrowAcero) find_dependency(ArrowDataset) find_dependency(Parquet) diff --git a/cpp/src/arrow/engine/CMakeLists.txt b/cpp/src/arrow/engine/CMakeLists.txt index d3aa011537f..7494be8ebb1 100644 --- a/cpp/src/arrow/engine/CMakeLists.txt +++ b/cpp/src/arrow/engine/CMakeLists.txt @@ -64,7 +64,8 @@ endforeach() set(ARROW_SUBSTRAIT_TEST_LINK_LIBS ${ARROW_SUBSTRAIT_LINK_lIBS} ${ARROW_TEST_LINK_LIBS}) if(ARROW_TEST_LINKAGE STREQUAL "static") - list(APPEND ARROW_SUBSTRAIT_TEST_LINK_LIBS arrow_substrait_static) + list(APPEND ARROW_SUBSTRAIT_TEST_LINK_LIBS arrow_substrait_static + arrow_substrait_static) else() list(APPEND ARROW_SUBSTRAIT_TEST_LINK_LIBS arrow_substrait_shared) endif() @@ -75,6 +76,7 @@ add_arrow_test(substrait_test substrait/function_test.cc substrait/serde_test.cc substrait/protobuf_test_util.cc + substrait/test_util.cc EXTRA_LINK_LIBS ${ARROW_SUBSTRAIT_TEST_LINK_LIBS} PREFIX diff --git a/cpp/src/arrow/engine/arrow-substrait.pc.in b/cpp/src/arrow/engine/arrow-substrait.pc.in index 173e1c91efd..1e891995e2d 100644 --- a/cpp/src/arrow/engine/arrow-substrait.pc.in +++ b/cpp/src/arrow/engine/arrow-substrait.pc.in @@ -22,6 +22,6 @@ libdir=@ARROW_PKG_CONFIG_LIBDIR@ Name: Apache Arrow Substrait Consumer Description: Apache Arrow's Substrait Consumer. Version: @ARROW_VERSION@ -Requires: arrow +Requires: arrow-dataset Libs: -L${libdir} -larrow_substrait Cflags.private: -DARROW_ENGINE_STATIC diff --git a/cpp/src/arrow/engine/substrait/function_test.cc b/cpp/src/arrow/engine/substrait/function_test.cc index a0b02bfd76b..bb9df208469 100644 --- a/cpp/src/arrow/engine/substrait/function_test.cc +++ b/cpp/src/arrow/engine/substrait/function_test.cc @@ -28,12 +28,12 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" #include "arrow/array/builder_binary.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/util.h" #include "arrow/datum.h" #include "arrow/engine/substrait/extension_set.h" #include "arrow/engine/substrait/options.h" @@ -105,7 +105,7 @@ Result> GetOutputTable( return table; } -Result> PlanFromTestCase( +Result> PlanFromTestCase( const FunctionTestCase& test_case, std::shared_ptr
* output_table) { ARROW_ASSIGN_OR_RAISE(std::shared_ptr
input_table, GetInputTable(test_case.arguments, test_case.data_types)); @@ -114,23 +114,22 @@ Result> PlanFromTestCase( internal::CreateScanProjectSubstrait( test_case.function_id, input_table, test_case.arguments, test_case.options, test_case.data_types, *test_case.expected_output_type)); - std::shared_ptr consumer = - std::make_shared(output_table, - default_memory_pool()); + std::shared_ptr consumer = + std::make_shared(output_table, default_memory_pool()); // Mock table provider that ignores the table name and returns input_table NamedTableProvider table_provider = [input_table](const std::vector&, const Schema&) { - std::shared_ptr options = - std::make_shared(input_table); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(input_table); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); ARROW_ASSIGN_OR_RAISE( - std::shared_ptr plan, + std::shared_ptr plan, DeserializePlan(*substrait, std::move(consumer), default_extension_id_registry(), /*ext_set_out=*/nullptr, conversion_options)); return plan; @@ -139,7 +138,7 @@ Result> PlanFromTestCase( void CheckValidTestCases(const std::vector& valid_cases) { for (const FunctionTestCase& test_case : valid_cases) { std::shared_ptr
output_table; - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, PlanFromTestCase(test_case, &output_table)); plan->StartProducing(); ASSERT_FINISHES_OK(plan->finished()); @@ -160,7 +159,7 @@ void CheckErrorTestCases(const std::vector& error_cases) { ARROW_SCOPED_TRACE("func=", test_case.function_id.uri, "#", test_case.function_id.name); std::shared_ptr
output_table; - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, PlanFromTestCase(test_case, &output_table)); plan->StartProducing(); ASSERT_FINISHES_AND_RAISES(Invalid, plan->finished()); @@ -599,7 +598,7 @@ std::shared_ptr
GetOutputTableForAggregateCase( return table; } -std::shared_ptr PlanFromAggregateCase( +std::shared_ptr PlanFromAggregateCase( const AggregateTestCase& test_case, std::shared_ptr
* output_table, bool with_keys) { std::shared_ptr
input_table = GetInputTableForAggregateCase(test_case); @@ -613,23 +612,22 @@ std::shared_ptr PlanFromAggregateCase( test_case.function_id, input_table, key_idxs, /*arg_idxs=*/test_case.nullary ? std::vector{} : std::vector{1}, *test_case.output_type)); - std::shared_ptr consumer = - std::make_shared(output_table, - default_memory_pool()); + std::shared_ptr consumer = + std::make_shared(output_table, default_memory_pool()); // Mock table provider that ignores the table name and returns input_table NamedTableProvider table_provider = [input_table](const std::vector&, const Schema&) { - std::shared_ptr options = - std::make_shared(input_table); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(input_table); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); EXPECT_OK_AND_ASSIGN( - std::shared_ptr plan, + std::shared_ptr plan, DeserializePlan(*substrait, std::move(consumer), default_extension_id_registry(), /*ext_set_out=*/nullptr, conversion_options)); return plan; @@ -637,7 +635,7 @@ std::shared_ptr PlanFromAggregateCase( void CheckWholeAggregateCase(const AggregateTestCase& test_case) { std::shared_ptr
output_table; - std::shared_ptr plan = + std::shared_ptr plan = PlanFromAggregateCase(test_case, &output_table, /*with_keys=*/false); plan->StartProducing(); @@ -653,7 +651,7 @@ void CheckWholeAggregateCase(const AggregateTestCase& test_case) { void CheckGroupedAggregateCase(const AggregateTestCase& test_case) { std::shared_ptr
output_table; - std::shared_ptr plan = + std::shared_ptr plan = PlanFromAggregateCase(test_case, &output_table, /*with_keys=*/true); plan->StartProducing(); diff --git a/cpp/src/arrow/engine/substrait/options.cc b/cpp/src/arrow/engine/substrait/options.cc index da07f151633..979db875df2 100644 --- a/cpp/src/arrow/engine/substrait/options.cc +++ b/cpp/src/arrow/engine/substrait/options.cc @@ -20,8 +20,8 @@ #include #include -#include "arrow/compute/exec/asof_join_node.h" -#include "arrow/compute/exec/options.h" +#include "arrow/acero/asof_join_node.h" +#include "arrow/acero/options.h" #include "arrow/engine/substrait/expression_internal.h" #include "arrow/engine/substrait/options_internal.h" #include "arrow/engine/substrait/relation_internal.h" @@ -32,9 +32,9 @@ namespace engine { namespace { -std::vector MakeDeclarationInputs( +std::vector MakeDeclarationInputs( const std::vector& inputs) { - std::vector input_decls(inputs.size()); + std::vector input_decls(inputs.size()); for (size_t i = 0; i < inputs.size(); i++) { input_decls[i] = inputs[i].declaration; } @@ -97,7 +97,7 @@ class DefaultExtensionProvider : public BaseExtensionProvider { } size_t n_input = inputs.size(), i = 0; - std::vector input_keys(n_input); + std::vector input_keys(n_input); for (const auto& keys : as_of_join_rel.keys()) { // on-key if (!keys.has_on()) { @@ -134,14 +134,14 @@ class DefaultExtensionProvider : public BaseExtensionProvider { } std::vector field_output_indices; ARROW_ASSIGN_OR_RAISE(auto schema, - compute::asofjoin::MakeOutputSchema(input_schema, input_keys, - &field_output_indices)); - compute::AsofJoinNodeOptions asofjoin_node_opts{std::move(input_keys), tolerance}; + acero::asofjoin::MakeOutputSchema(input_schema, input_keys, + &field_output_indices)); + acero::AsofJoinNodeOptions asofjoin_node_opts{std::move(input_keys), tolerance}; // declaration auto input_decls = MakeDeclarationInputs(inputs); return RelationInfo{ - {compute::Declaration("asofjoin", input_decls, std::move(asofjoin_node_opts)), + {acero::Declaration("asofjoin", input_decls, std::move(asofjoin_node_opts)), std::move(schema)}, std::move(field_output_indices)}; } @@ -271,9 +271,9 @@ default_extension_provider_singleton() { ConfigurableSingleton& default_named_tap_provider_singleton() { static ConfigurableSingleton singleton( - [](const std::string& tap_kind, std::vector inputs, + [](const std::string& tap_kind, std::vector inputs, const std::string& tap_name, - std::shared_ptr tap_schema) -> Result { + std::shared_ptr tap_schema) -> Result { return Status::NotImplemented( "Plan contained a NamedTapRel but no provider configured"); }); diff --git a/cpp/src/arrow/engine/substrait/options.h b/cpp/src/arrow/engine/substrait/options.h index 6818d0a9c3c..e501914dd37 100644 --- a/cpp/src/arrow/engine/substrait/options.h +++ b/cpp/src/arrow/engine/substrait/options.h @@ -23,8 +23,8 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/compute/type_fwd.h" #include "arrow/engine/substrait/type_fwd.h" #include "arrow/engine/substrait/visibility.h" @@ -65,12 +65,12 @@ enum class ARROW_ENGINE_EXPORT ConversionStrictness { BEST_EFFORT, }; -using NamedTableProvider = std::function( +using NamedTableProvider = std::function( const std::vector&, const Schema&)>; static NamedTableProvider kDefaultNamedTableProvider; -using NamedTapProvider = std::function( - const std::string&, std::vector, const std::string&, +using NamedTapProvider = std::function( + const std::string&, std::vector, const std::string&, std::shared_ptr)>; class ARROW_ENGINE_EXPORT ExtensionDetails { diff --git a/cpp/src/arrow/engine/substrait/plan_internal.cc b/cpp/src/arrow/engine/substrait/plan_internal.cc index 6d12c19fcd7..ecee81e25ff 100644 --- a/cpp/src/arrow/engine/substrait/plan_internal.cc +++ b/cpp/src/arrow/engine/substrait/plan_internal.cc @@ -160,7 +160,7 @@ std::unique_ptr CreateVersion() { } // namespace Result> PlanToProto( - const compute::Declaration& declr, ExtensionSet* ext_set, + const acero::Declaration& declr, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { auto subs_plan = std::make_unique(); subs_plan->set_allocated_version(CreateVersion().release()); diff --git a/cpp/src/arrow/engine/substrait/plan_internal.h b/cpp/src/arrow/engine/substrait/plan_internal.h index 235bf1a6ce1..737e65b7e2e 100644 --- a/cpp/src/arrow/engine/substrait/plan_internal.h +++ b/cpp/src/arrow/engine/substrait/plan_internal.h @@ -65,7 +65,7 @@ Result GetExtensionSetFromPlan( /// \param[in] conversion_options options to control serialization behavior /// \return the serialized plan ARROW_ENGINE_EXPORT Result> PlanToProto( - const compute::Declaration& declr, ExtensionSet* ext_set, + const acero::Declaration& declr, ExtensionSet* ext_set, const ConversionOptions& conversion_options = {}); } // namespace engine diff --git a/cpp/src/arrow/engine/substrait/relation.h b/cpp/src/arrow/engine/substrait/relation.h index d750abd4e33..a7e3605bf68 100644 --- a/cpp/src/arrow/engine/substrait/relation.h +++ b/cpp/src/arrow/engine/substrait/relation.h @@ -19,7 +19,7 @@ #include -#include "arrow/compute/exec/exec_plan.h" +#include "arrow/acero/exec_plan.h" #include "arrow/engine/substrait/visibility.h" #include "arrow/type_fwd.h" @@ -29,7 +29,7 @@ namespace engine { /// Execution information resulting from converting a Substrait relation. struct ARROW_ENGINE_EXPORT DeclarationInfo { /// The compute declaration produced thus far. - compute::Declaration declaration; + acero::Declaration declaration; std::shared_ptr output_schema; }; diff --git a/cpp/src/arrow/engine/substrait/relation_internal.cc b/cpp/src/arrow/engine/substrait/relation_internal.cc index 7f9c4c289fb..17aac62118b 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.cc +++ b/cpp/src/arrow/engine/substrait/relation_internal.cc @@ -28,9 +28,9 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/compute/api_aggregate.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" #include "arrow/compute/expression.h" #include "arrow/compute/kernel.h" #include "arrow/dataset/dataset.h" @@ -102,7 +102,7 @@ Result ProcessEmitProject( case substrait::RelCommon::EmitKindCase::kEmit: { const auto& emit = rel_common_opt->emit(); int emit_size = emit.output_mapping_size(); - const auto& proj_options = checked_cast( + const auto& proj_options = checked_cast( *project_declr.declaration.options); FieldVector emit_fields(emit_size); std::vector emit_proj_exprs(emit_size); @@ -115,9 +115,9 @@ Result ProcessEmitProject( // ProjectRel and the ProjectNodeOptions are set by only considering // what is in the emit expression in Substrait. return DeclarationInfo{ - compute::Declaration::Sequence( - {std::get(project_declr.declaration.inputs[0]), - {"project", compute::ProjectNodeOptions{std::move(emit_proj_exprs)}}}), + acero::Declaration::Sequence( + {std::get(project_declr.declaration.inputs[0]), + {"project", acero::ProjectNodeOptions{std::move(emit_proj_exprs)}}}), schema(std::move(emit_fields))}; } default: @@ -139,10 +139,9 @@ Result ProcessEmit(const RelMessage& rel, case substrait::RelCommon::EmitKindCase::kEmit: { ARROW_ASSIGN_OR_RAISE(auto emit_info, GetEmitInfo(rel, schema)); return DeclarationInfo{ - compute::Declaration::Sequence( - {no_emit_declr.declaration, - {"project", - compute::ProjectNodeOptions{std::move(emit_info.expressions)}}}), + acero::Declaration::Sequence({no_emit_declr.declaration, + {"project", acero::ProjectNodeOptions{std::move( + emit_info.expressions)}}}), std::move(emit_info.schema)}; } default: @@ -186,9 +185,9 @@ Result ProcessExtensionEmit( std::shared_ptr emit_schema = schema(std::move(emit_fields)); return DeclarationInfo{ - compute::Declaration::Sequence( + acero::Declaration::Sequence( {no_emit_declr.declaration, - {"project", compute::ProjectNodeOptions{std::move(proj_field_refs)}}}), + {"project", acero::ProjectNodeOptions{std::move(proj_field_refs)}}}), std::move(emit_schema)}; } @@ -413,7 +412,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& const substrait::ReadRel::NamedTable& named_table = read.named_table(); std::vector table_names(named_table.names().begin(), named_table.names().end()); - ARROW_ASSIGN_OR_RAISE(compute::Declaration source_decl, + ARROW_ASSIGN_OR_RAISE(acero::Declaration source_decl, named_table_provider(table_names, *base_schema)); if (!source_decl.IsValid()) { @@ -565,7 +564,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& ARROW_ASSIGN_OR_RAISE(auto ds, ds_factory->Finish(base_schema)); DeclarationInfo scan_declaration{ - compute::Declaration{"scan", dataset::ScanNodeOptions{ds, scan_options}}, + acero::Declaration{"scan", dataset::ScanNodeOptions{ds, scan_options}}, base_schema}; return ProcessEmit(std::move(read), std::move(scan_declaration), @@ -588,9 +587,9 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& ARROW_ASSIGN_OR_RAISE(auto condition, FromProto(filter.condition(), ext_set, conversion_options)); DeclarationInfo filter_declaration{ - compute::Declaration::Sequence({ + acero::Declaration::Sequence({ std::move(input.declaration), - {"filter", compute::FilterNodeOptions{std::move(condition)}}, + {"filter", acero::FilterNodeOptions{std::move(condition)}}, }), input.output_schema}; @@ -642,9 +641,9 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& } DeclarationInfo project_declaration{ - compute::Declaration::Sequence({ + acero::Declaration::Sequence({ std::move(input.declaration), - {"project", compute::ProjectNodeOptions{std::move(expressions)}}, + {"project", acero::ProjectNodeOptions{std::move(expressions)}}, }), project_schema}; @@ -664,27 +663,27 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& return Status::Invalid("substrait::JoinRel with no right relation"); } - compute::JoinType join_type; + acero::JoinType join_type; switch (join.type()) { case substrait::JoinRel::JOIN_TYPE_UNSPECIFIED: return Status::NotImplemented("Unspecified join type is not supported"); case substrait::JoinRel::JOIN_TYPE_INNER: - join_type = compute::JoinType::INNER; + join_type = acero::JoinType::INNER; break; case substrait::JoinRel::JOIN_TYPE_OUTER: - join_type = compute::JoinType::FULL_OUTER; + join_type = acero::JoinType::FULL_OUTER; break; case substrait::JoinRel::JOIN_TYPE_LEFT: - join_type = compute::JoinType::LEFT_OUTER; + join_type = acero::JoinType::LEFT_OUTER; break; case substrait::JoinRel::JOIN_TYPE_RIGHT: - join_type = compute::JoinType::RIGHT_OUTER; + join_type = acero::JoinType::RIGHT_OUTER; break; case substrait::JoinRel::JOIN_TYPE_SEMI: - join_type = compute::JoinType::LEFT_SEMI; + join_type = acero::JoinType::LEFT_SEMI; break; case substrait::JoinRel::JOIN_TYPE_ANTI: - join_type = compute::JoinType::LEFT_ANTI; + join_type = acero::JoinType::LEFT_ANTI; break; default: return Status::Invalid("Unsupported join type"); @@ -709,11 +708,11 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& expression.ToString()); } - compute::JoinKeyCmp join_key_cmp; + acero::JoinKeyCmp join_key_cmp; if (callptr->function_name == "equal") { - join_key_cmp = compute::JoinKeyCmp::EQ; + join_key_cmp = acero::JoinKeyCmp::EQ; } else if (callptr->function_name == "is_not_distinct_from") { - join_key_cmp = compute::JoinKeyCmp::IS; + join_key_cmp = acero::JoinKeyCmp::IS; } else { return Status::Invalid( "Only `equal` or `is_not_distinct_from` are supported for join key " @@ -745,11 +744,11 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& std::vector adjusted_field_indices(right_field_path->indices()); adjusted_field_indices[0] -= num_left_fields; FieldPath adjusted_right_keys(adjusted_field_indices); - compute::HashJoinNodeOptions join_options{{std::move(*left_keys)}, - {std::move(adjusted_right_keys)}}; + acero::HashJoinNodeOptions join_options{{std::move(*left_keys)}, + {std::move(adjusted_right_keys)}}; join_options.join_type = join_type; join_options.key_cmp = {join_key_cmp}; - compute::Declaration join_dec{"hashjoin", std::move(join_options)}; + acero::Declaration join_dec{"hashjoin", std::move(join_options)}; join_dec.inputs.emplace_back(std::move(left.declaration)); join_dec.inputs.emplace_back(std::move(right.declaration)); @@ -901,7 +900,7 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& return Status::Invalid("Unknown union type"); } int input_size = set.inputs_size(); - compute::Declaration union_declr{"union", compute::ExecNodeOptions{}}; + acero::Declaration union_declr{"union", acero::ExecNodeOptions{}}; std::shared_ptr union_schema; for (int input_id = 0; input_id < input_size; input_id++) { ARROW_ASSIGN_OR_RAISE( @@ -922,23 +921,22 @@ Result FromProto(const substrait::Rel& rel, const ExtensionSet& } return Status::NotImplemented( - "conversion to arrow::compute::Declaration from Substrait relation ", + "conversion to arrow::acero::Declaration from Substrait relation ", rel.DebugString()); } namespace { -Result> ExtractSchemaToBind(const compute::Declaration& declr) { +Result> ExtractSchemaToBind(const acero::Declaration& declr) { std::shared_ptr bind_schema; if (declr.factory_name == "scan") { const auto& opts = checked_cast(*(declr.options)); bind_schema = opts.dataset->schema(); } else if (declr.factory_name == "filter") { - auto input_declr = std::get(declr.inputs[0]); + auto input_declr = std::get(declr.inputs[0]); ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr)); } else if (declr.factory_name == "named_table") { - const auto& opts = - checked_cast(*declr.options); + const auto& opts = checked_cast(*declr.options); bind_schema = opts.schema; } else if (declr.factory_name == "sink") { // Note that the sink has no output_schema @@ -951,11 +949,11 @@ Result> ExtractSchemaToBind(const compute::Declaration& } Result> NamedTableRelationConverter( - const std::shared_ptr& schema, const compute::Declaration& declaration, + const std::shared_ptr& schema, const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { auto read_rel = std::make_unique(); const auto& named_table_options = - checked_cast(*declaration.options); + checked_cast(*declaration.options); // set schema ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*schema, ext_set, conversion_options)); @@ -975,7 +973,7 @@ Result> NamedTableRelationConverter( } Result> ScanRelationConverter( - const std::shared_ptr& schema, const compute::Declaration& declaration, + const std::shared_ptr& schema, const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { auto read_rel = std::make_unique(); const auto& scan_node_options = @@ -1019,11 +1017,11 @@ Result> ScanRelationConverter( } Result> FilterRelationConverter( - const std::shared_ptr& schema, const compute::Declaration& declaration, + const std::shared_ptr& schema, const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { auto filter_rel = std::make_unique(); const auto& filter_node_options = - checked_cast(*(declaration.options)); + checked_cast(*(declaration.options)); auto filter_expr = filter_node_options.filter_expression; compute::Expression bound_expression; @@ -1037,9 +1035,8 @@ Result> FilterRelationConverter( // handling input auto declr_input = declaration.inputs[0]; - ARROW_ASSIGN_OR_RAISE( - auto input_rel, - ToProto(std::get(declr_input), ext_set, conversion_options)); + ARROW_ASSIGN_OR_RAISE(auto input_rel, ToProto(std::get(declr_input), + ext_set, conversion_options)); filter_rel->set_allocated_input(input_rel.release()); ARROW_ASSIGN_OR_RAISE(auto subs_expr, @@ -1050,7 +1047,7 @@ Result> FilterRelationConverter( } // namespace -Status SerializeAndCombineRelations(const compute::Declaration& declaration, +Status SerializeAndCombineRelations(const acero::Declaration& declaration, ExtensionSet* ext_set, std::unique_ptr* rel, const ConversionOptions& conversion_options) { @@ -1078,7 +1075,7 @@ Status SerializeAndCombineRelations(const compute::Declaration& declaration, // Generally when a plan is deserialized the declaration will be a sink declaration. // Since there is no Sink relation in substrait, this function would be recursively // called on the input of the Sink declaration. - auto sink_input_decl = std::get(declaration.inputs[0]); + auto sink_input_decl = std::get(declaration.inputs[0]); RETURN_NOT_OK( SerializeAndCombineRelations(sink_input_decl, ext_set, rel, conversion_options)); } else { @@ -1090,7 +1087,7 @@ Status SerializeAndCombineRelations(const compute::Declaration& declaration, } Result> ToProto( - const compute::Declaration& declr, ExtensionSet* ext_set, + const acero::Declaration& declr, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { auto rel = std::make_unique(); RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, &rel, conversion_options)); diff --git a/cpp/src/arrow/engine/substrait/relation_internal.h b/cpp/src/arrow/engine/substrait/relation_internal.h index 8d560d9f41f..e9085dedf30 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.h +++ b/cpp/src/arrow/engine/substrait/relation_internal.h @@ -45,7 +45,7 @@ Result FromProto(const substrait::Rel&, const ExtensionSet&, /// is preferred in the Substrait space rather than internal components of /// Acero execution engine. ARROW_ENGINE_EXPORT Result> ToProto( - const compute::Declaration&, ExtensionSet*, const ConversionOptions&); + const acero::Declaration&, ExtensionSet*, const ConversionOptions&); namespace internal { diff --git a/cpp/src/arrow/engine/substrait/serde.cc b/cpp/src/arrow/engine/substrait/serde.cc index de4aae343ac..78f0ea88925 100644 --- a/cpp/src/arrow/engine/substrait/serde.cc +++ b/cpp/src/arrow/engine/substrait/serde.cc @@ -30,9 +30,9 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/buffer.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" #include "arrow/compute/expression.h" #include "arrow/dataset/file_base.h" #include "arrow/engine/substrait/expression_internal.h" @@ -67,7 +67,7 @@ Result ParseFromBuffer(const Buffer& buf) { } Result> SerializePlan( - const compute::Declaration& declaration, ExtensionSet* ext_set, + const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { ARROW_ASSIGN_OR_RAISE(auto subs_plan, PlanToProto(declaration, ext_set, conversion_options)); @@ -76,14 +76,14 @@ Result> SerializePlan( } Result> SerializeRelation( - const compute::Declaration& declaration, ExtensionSet* ext_set, + const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options) { ARROW_ASSIGN_OR_RAISE(auto relation, ToProto(declaration, ext_set, conversion_options)); std::string serialized = relation->SerializeAsString(); return Buffer::FromString(std::move(serialized)); } -Result DeserializeRelation( +Result DeserializeRelation( const Buffer& buf, const ExtensionSet& ext_set, const ConversionOptions& conversion_options) { ARROW_ASSIGN_OR_RAISE(auto rel, ParseFromBuffer(buf)); @@ -91,38 +91,37 @@ Result DeserializeRelation( return std::move(decl_info.declaration); } -using DeclarationFactory = std::function( - compute::Declaration, std::vector names)>; +using DeclarationFactory = std::function( + acero::Declaration, std::vector names)>; namespace { DeclarationFactory MakeConsumingSinkDeclarationFactory( const ConsumerFactory& consumer_factory) { return [&consumer_factory]( - compute::Declaration input, - std::vector names) -> Result { - std::shared_ptr consumer = consumer_factory(); + acero::Declaration input, + std::vector names) -> Result { + std::shared_ptr consumer = consumer_factory(); if (consumer == nullptr) { return Status::Invalid("consumer factory is exhausted"); } - std::shared_ptr options = - std::make_shared( - compute::ConsumingSinkNodeOptions{std::move(consumer), std::move(names)}); - return compute::Declaration::Sequence( - {std::move(input), {"consuming_sink", options}}); + std::shared_ptr options = + std::make_shared( + acero::ConsumingSinkNodeOptions{std::move(consumer), std::move(names)}); + return acero::Declaration::Sequence({std::move(input), {"consuming_sink", options}}); }; } DeclarationFactory MakeWriteDeclarationFactory( const WriteOptionsFactory& write_options_factory) { return [&write_options_factory]( - compute::Declaration input, - std::vector names) -> Result { + acero::Declaration input, + std::vector names) -> Result { std::shared_ptr options = write_options_factory(); if (options == nullptr) { return Status::Invalid("write options factory is exhausted"); } - return compute::Declaration::Sequence( + return acero::Declaration::Sequence( {std::move(input), {"write", std::move(*options)}}); }; } @@ -130,7 +129,7 @@ DeclarationFactory MakeWriteDeclarationFactory( constexpr uint32_t kMinimumMajorVersion = 0; constexpr uint32_t kMinimumMinorVersion = 20; -Result> DeserializePlans( +Result> DeserializePlans( const Buffer& buf, DeclarationFactory declaration_factory, const ExtensionIdRegistry* registry, ExtensionSet* ext_set_out, const ConversionOptions& conversion_options) { @@ -145,7 +144,7 @@ Result> DeserializePlans( ARROW_ASSIGN_OR_RAISE(auto ext_set, GetExtensionSetFromPlan(plan, conversion_options, registry)); - std::vector sink_decls; + std::vector sink_decls; for (const substrait::PlanRel& plan_rel : plan.relations()) { ARROW_ASSIGN_OR_RAISE( auto decl_info, @@ -171,7 +170,7 @@ Result> DeserializePlans( } // namespace -Result> DeserializePlans( +Result> DeserializePlans( const Buffer& buf, const ConsumerFactory& consumer_factory, const ExtensionIdRegistry* registry, ExtensionSet* ext_set_out, const ConversionOptions& conversion_options) { @@ -179,7 +178,7 @@ Result> DeserializePlans( registry, ext_set_out, conversion_options); } -Result> DeserializePlans( +Result> DeserializePlans( const Buffer& buf, const WriteOptionsFactory& write_options_factory, const ExtensionIdRegistry* registry, ExtensionSet* ext_set_out, const ConversionOptions& conversion_options) { @@ -230,12 +229,12 @@ ARROW_ENGINE_EXPORT Result DeserializePlan( namespace { -Result> MakeSingleDeclarationPlan( - std::vector declarations) { +Result> MakeSingleDeclarationPlan( + std::vector declarations) { if (declarations.size() > 1) { return Status::Invalid("DeserializePlan does not support multiple root relations"); } else { - ARROW_ASSIGN_OR_RAISE(auto plan, compute::ExecPlan::Make()); + ARROW_ASSIGN_OR_RAISE(auto plan, acero::ExecPlan::Make()); ARROW_RETURN_NOT_OK(declarations[0].AddToPlan(plan.get())); return std::move(plan); } @@ -243,21 +242,21 @@ Result> MakeSingleDeclarationPlan( } // namespace -Result> DeserializePlan( - const Buffer& buf, const std::shared_ptr& consumer, +Result> DeserializePlan( + const Buffer& buf, const std::shared_ptr& consumer, const ExtensionIdRegistry* registry, ExtensionSet* ext_set_out, const ConversionOptions& conversion_options) { struct SingleConsumer { - std::shared_ptr operator()() { + std::shared_ptr operator()() { if (factory_done) { Status::Invalid("SingleConsumer invoked more than once").Warn(); - return std::shared_ptr{}; + return std::shared_ptr{}; } factory_done = true; return consumer; } bool factory_done; - std::shared_ptr consumer; + std::shared_ptr consumer; }; ARROW_ASSIGN_OR_RAISE(auto declarations, DeserializePlans(buf, SingleConsumer{false, consumer}, registry, @@ -265,7 +264,7 @@ Result> DeserializePlan( return MakeSingleDeclarationPlan(declarations); } -Result> DeserializePlan( +Result> DeserializePlan( const Buffer& buf, const std::shared_ptr& write_options, const ExtensionIdRegistry* registry, ExtensionSet* ext_set_out, const ConversionOptions& conversion_options) { diff --git a/cpp/src/arrow/engine/substrait/serde.h b/cpp/src/arrow/engine/substrait/serde.h index c13e0b90f31..6533bba88b5 100644 --- a/cpp/src/arrow/engine/substrait/serde.h +++ b/cpp/src/arrow/engine/substrait/serde.h @@ -49,12 +49,12 @@ namespace engine { /// \return a buffer containing the protobuf serialization of the Acero relation ARROW_ENGINE_EXPORT Result> SerializePlan( - const compute::Declaration& declaration, ExtensionSet* ext_set, + const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options = {}); /// Factory function type for generating the node that consumes the batches produced by /// each toplevel Substrait relation when deserializing a Substrait Plan. -using ConsumerFactory = std::function()>; +using ConsumerFactory = std::function()>; /// \brief Deserializes a Substrait Plan message to a list of ExecNode declarations /// @@ -71,7 +71,7 @@ using ConsumerFactory = std::function /// \param[in] conversion_options options to control how the conversion is to be done. /// \return a vector of ExecNode declarations, one for each toplevel relation in the /// Substrait Plan -ARROW_ENGINE_EXPORT Result> DeserializePlans( +ARROW_ENGINE_EXPORT Result> DeserializePlans( const Buffer& buf, const ConsumerFactory& consumer_factory, const ExtensionIdRegistry* registry = NULLPTR, ExtensionSet* ext_set_out = NULLPTR, const ConversionOptions& conversion_options = {}); @@ -91,8 +91,8 @@ ARROW_ENGINE_EXPORT Result> DeserializePlans( /// Plan is returned here. /// \return an ExecNode corresponding to the single toplevel relation in the Substrait /// Plan -ARROW_ENGINE_EXPORT Result> DeserializePlan( - const Buffer& buf, const std::shared_ptr& consumer, +ARROW_ENGINE_EXPORT Result> DeserializePlan( + const Buffer& buf, const std::shared_ptr& consumer, const ExtensionIdRegistry* registry = NULLPTR, ExtensionSet* ext_set_out = NULLPTR, const ConversionOptions& conversion_options = {}); @@ -115,7 +115,7 @@ using WriteOptionsFactory = std::function> DeserializePlans( +ARROW_ENGINE_EXPORT Result> DeserializePlans( const Buffer& buf, const WriteOptionsFactory& write_options_factory, const ExtensionIdRegistry* registry = NULLPTR, ExtensionSet* ext_set_out = NULLPTR, const ConversionOptions& conversion_options = {}); @@ -135,7 +135,7 @@ ARROW_ENGINE_EXPORT Result> DeserializePlans( /// \param[in] conversion_options options to control how the conversion is to be done. /// \return a vector of ExecNode declarations, one for each toplevel relation in the /// Substrait Plan -ARROW_ENGINE_EXPORT Result> DeserializePlan( +ARROW_ENGINE_EXPORT Result> DeserializePlan( const Buffer& buf, const std::shared_ptr& write_options, const ExtensionIdRegistry* registry = NULLPTR, ExtensionSet* ext_set_out = NULLPTR, const ConversionOptions& conversion_options = {}); @@ -243,7 +243,7 @@ Result> SerializeExpression( /// /// \return a buffer containing the protobuf serialization of the Acero relation ARROW_ENGINE_EXPORT Result> SerializeRelation( - const compute::Declaration& declaration, ExtensionSet* ext_set, + const acero::Declaration& declaration, ExtensionSet* ext_set, const ConversionOptions& conversion_options = {}); /// \brief Deserializes a Substrait Rel (relation) message to an ExecNode declaration @@ -254,7 +254,7 @@ ARROW_ENGINE_EXPORT Result> SerializeRelation( /// surrounding Plan message /// \param[in] conversion_options options to control how the conversion is to be done. /// \return the corresponding ExecNode declaration -ARROW_ENGINE_EXPORT Result DeserializeRelation( +ARROW_ENGINE_EXPORT Result DeserializeRelation( const Buffer& buf, const ExtensionSet& ext_set, const ConversionOptions& conversion_options = {}); diff --git a/cpp/src/arrow/engine/substrait/serde_test.cc b/cpp/src/arrow/engine/substrait/serde_test.cc index 0f605cd7ce4..8568a496944 100644 --- a/cpp/src/arrow/engine/substrait/serde_test.cc +++ b/cpp/src/arrow/engine/substrait/serde_test.cc @@ -28,16 +28,15 @@ #include #include +#include "arrow/acero/asof_join_node.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/map_node.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" #include "arrow/buffer.h" #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/asof_join_node.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/map_node.h" -#include "arrow/compute/exec/options.h" -#include "arrow/compute/exec/test_util.h" -#include "arrow/compute/exec/util.h" #include "arrow/compute/expression.h" #include "arrow/compute/expression_internal.h" #include "arrow/compute/registry.h" @@ -54,6 +53,7 @@ #include "arrow/engine/substrait/extension_types.h" #include "arrow/engine/substrait/options.h" #include "arrow/engine/substrait/serde.h" +#include "arrow/engine/substrait/test_util.h" #include "arrow/engine/substrait/util.h" #include "arrow/filesystem/filesystem.h" #include "arrow/filesystem/localfs.h" @@ -91,14 +91,14 @@ namespace engine { Status AddPassFactory( const std::string& factory_name, - compute::ExecFactoryRegistry* registry = compute::default_exec_factory_registry()) { + acero::ExecFactoryRegistry* registry = acero::default_exec_factory_registry()) { + using acero::ExecNode; + using acero::ExecNodeOptions; + using acero::ExecPlan; using compute::ExecBatch; - using compute::ExecNode; - using compute::ExecNodeOptions; - using compute::ExecPlan; - struct PassNode : public compute::MapNode { + struct PassNode : public acero::MapNode { static Result Make(ExecPlan* plan, std::vector inputs, - const compute::ExecNodeOptions& options) { + const acero::ExecNodeOptions& options) { RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "PassNode")); return plan->EmplaceNode(plan, inputs, inputs[0]->output_schema()); } @@ -113,7 +113,7 @@ Status AddPassFactory( return registry->AddFactory(factory_name, PassNode::Make); } -const auto kNullConsumer = std::make_shared(); +const auto kNullConsumer = std::make_shared(); void WriteIpcData(const std::string& path, const std::shared_ptr file_system, @@ -137,10 +137,10 @@ void CheckRoundTripResult(const std::shared_ptr
expected_table, ASSERT_OK_AND_ASSIGN(auto sink_decls, DeserializePlans( *buf, [] { return kNullConsumer; }, ext_id_reg, &ext_set, conversion_options)); - auto& other_declrs = std::get(sink_decls[0].inputs[0]); + auto& other_declrs = std::get(sink_decls[0].inputs[0]); ASSERT_OK_AND_ASSIGN(auto output_table, - compute::DeclarationToTable(other_declrs, /*use_threads=*/false)); + acero::DeclarationToTable(other_declrs, /*use_threads=*/false)); if (!include_columns.empty()) { ASSERT_OK_AND_ASSIGN(output_table, output_table->SelectColumns(include_columns)); @@ -155,7 +155,7 @@ void CheckRoundTripResult(const std::shared_ptr
expected_table, } ASSERT_OK_AND_ASSIGN(output_table, output_table->CombineChunks()); ASSERT_OK_AND_ASSIGN(auto merged_expected, expected_table->CombineChunks()); - compute::AssertTablesEqualIgnoringOrder(merged_expected, output_table); + engine::AssertTablesEqualIgnoringOrder(merged_expected, output_table); } const std::shared_ptr kBoringSchema = schema({ @@ -202,14 +202,14 @@ inline compute::Expression UseBoringRefs(const compute::Expression& expr) { return compute::Expression{std::move(modified_call)}; } -int CountProjectNodeOptionsInDeclarations(const compute::Declaration& input) { +int CountProjectNodeOptionsInDeclarations(const acero::Declaration& input) { int counter = 0; if (input.factory_name == "project") { counter++; } const auto& inputs = input.inputs; for (const auto& in : inputs) { - counter += CountProjectNodeOptionsInDeclarations(std::get(in)); + counter += CountProjectNodeOptionsInDeclarations(std::get(in)); } return counter; } @@ -225,7 +225,7 @@ void ValidateNumProjectNodes(int expected_projections, const std::shared_ptr(sink_decls[0].inputs[0]); + auto& other_declrs = std::get(sink_decls[0].inputs[0]); int num_projections = CountProjectNodeOptionsInDeclarations(other_declrs); ASSERT_EQ(num_projections, expected_projections); } @@ -966,9 +966,9 @@ TEST(Substrait, ReadRel) { /// \brief Create a NamedTableProvider that provides `table` regardless of the name NamedTableProvider AlwaysProvideSameTable(std::shared_ptr
table) { return [table = std::move(table)](const std::vector&, const Schema&) { - std::shared_ptr options = - std::make_shared(table); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(table); + return acero::Declaration("table_source", {}, options, "mock_source"); }; } @@ -1145,7 +1145,7 @@ TEST(Substrait, ExtensionSetFromPlanExhaustedFactory) { ASSERT_RAISES( Invalid, DeserializePlans( - *buf, []() -> std::shared_ptr { return nullptr; }, + *buf, []() -> std::shared_ptr { return nullptr; }, ext_id_reg, &ext_set)); ASSERT_RAISES( Invalid, @@ -1244,11 +1244,11 @@ TEST(Substrait, DeserializeWithConsumerFactory) { ASSERT_OK_AND_ASSIGN(std::string substrait_json, GetSubstraitJSON()); ASSERT_OK_AND_ASSIGN(auto buf, SerializeJsonPlan(substrait_json)); ASSERT_OK_AND_ASSIGN(auto declarations, - DeserializePlans(*buf, compute::NullSinkNodeConsumer::Make)); + DeserializePlans(*buf, acero::NullSinkNodeConsumer::Make)); ASSERT_EQ(declarations.size(), 1); - compute::Declaration* decl = &declarations[0]; + acero::Declaration* decl = &declarations[0]; ASSERT_EQ(decl->factory_name, "consuming_sink"); - ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + ASSERT_OK_AND_ASSIGN(auto plan, acero::ExecPlan::Make()); ASSERT_OK_AND_ASSIGN(auto sink_node, declarations[0].AddToPlan(plan.get())); ASSERT_STREQ(sink_node->kind_name(), "ConsumingSinkNode"); ASSERT_EQ(sink_node->num_inputs(), 1); @@ -1262,10 +1262,10 @@ TEST(Substrait, DeserializeWithConsumerFactory) { TEST(Substrait, DeserializeSinglePlanWithConsumerFactory) { ASSERT_OK_AND_ASSIGN(std::string substrait_json, GetSubstraitJSON()); ASSERT_OK_AND_ASSIGN(auto buf, SerializeJsonPlan(substrait_json)); - ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, - DeserializePlan(*buf, compute::NullSinkNodeConsumer::Make())); + ASSERT_OK_AND_ASSIGN(std::shared_ptr plan, + DeserializePlan(*buf, acero::NullSinkNodeConsumer::Make())); ASSERT_EQ(2, plan->nodes().size()); - compute::ExecNode* sink_node = plan->nodes()[1]; + acero::ExecNode* sink_node = plan->nodes()[1]; ASSERT_STREQ(sink_node->kind_name(), "ConsumingSinkNode"); ASSERT_EQ(sink_node->num_inputs(), 1); auto& prev_node = sink_node->inputs()[0]; @@ -1297,13 +1297,13 @@ TEST(Substrait, DeserializeWithWriteOptionsFactory) { ASSERT_OK_AND_ASSIGN(auto buf, SerializeJsonPlan(substrait_json)); ASSERT_OK_AND_ASSIGN(auto declarations, DeserializePlans(*buf, write_options_factory)); ASSERT_EQ(declarations.size(), 1); - compute::Declaration* decl = &declarations[0]; + acero::Declaration* decl = &declarations[0]; ASSERT_EQ(decl->factory_name, "write"); ASSERT_EQ(decl->inputs.size(), 1); - decl = std::get_if(&decl->inputs[0]); + decl = std::get_if(&decl->inputs[0]); ASSERT_NE(decl, nullptr); ASSERT_EQ(decl->factory_name, "scan"); - ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make()); + ASSERT_OK_AND_ASSIGN(auto plan, acero::ExecPlan::Make()); ASSERT_OK_AND_ASSIGN(auto sink_node, declarations[0].AddToPlan(plan.get())); ASSERT_STREQ(sink_node->kind_name(), "ConsumingSinkNode"); ASSERT_EQ(sink_node->num_inputs(), 1); @@ -1495,16 +1495,16 @@ TEST(Substrait, JoinPlanBasic) { auto join_decl = sink_decls[0].inputs[0]; - const auto& join_rel = std::get(join_decl); + const auto& join_rel = std::get(join_decl); const auto& join_options = - checked_cast(*join_rel.options); + checked_cast(*join_rel.options); EXPECT_EQ(join_rel.factory_name, "hashjoin"); - EXPECT_EQ(join_options.join_type, compute::JoinType::INNER); + EXPECT_EQ(join_options.join_type, acero::JoinType::INNER); - const auto& left_rel = std::get(join_rel.inputs[0]); - const auto& right_rel = std::get(join_rel.inputs[1]); + const auto& left_rel = std::get(join_rel.inputs[0]); + const auto& right_rel = std::get(join_rel.inputs[1]); const auto& l_options = checked_cast(*left_rel.options); @@ -1518,7 +1518,7 @@ TEST(Substrait, JoinPlanBasic) { r_options.dataset->schema(), schema({field("X", int32()), field("Y", int32()), field("A", int32())})); - EXPECT_EQ(join_options.key_cmp[0], compute::JoinKeyCmp::EQ); + EXPECT_EQ(join_options.key_cmp[0], acero::JoinKeyCmp::EQ); } } @@ -1870,10 +1870,10 @@ TEST(Substrait, AggregateBasic) { DeserializePlans(*buf, [] { return kNullConsumer; })); auto agg_decl = sink_decls[0].inputs[0]; - const auto& agg_rel = std::get(agg_decl); + const auto& agg_rel = std::get(agg_decl); const auto& agg_options = - checked_cast(*agg_rel.options); + checked_cast(*agg_rel.options); EXPECT_EQ(agg_rel.factory_name, "aggregate"); EXPECT_EQ(agg_options.aggregates[0].name, ""); @@ -2250,11 +2250,10 @@ TEST(SubstraitRoundTrip, BasicPlan) { arrow::AsyncGenerator> sink_gen; - auto declarations = compute::Declaration::Sequence( - {compute::Declaration( - {"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"}), - compute::Declaration({"filter", compute::FilterNodeOptions{filter}, "f"}), - compute::Declaration({"sink", compute::SinkNodeOptions{&sink_gen}, "e"})}); + auto declarations = acero::Declaration::Sequence( + {acero::Declaration({"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"}), + acero::Declaration({"filter", acero::FilterNodeOptions{filter}, "f"}), + acero::Declaration({"sink", acero::SinkNodeOptions{&sink_gen}, "e"})}); std::shared_ptr sp_ext_id_reg = MakeExtensionIdRegistry(); ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get(); @@ -2267,10 +2266,9 @@ TEST(SubstraitRoundTrip, BasicPlan) { DeserializePlans( *serialized_plan, [] { return kNullConsumer; }, ext_id_reg, &ext_set)); // filter declaration - const auto& roundtripped_filter = - std::get(sink_decls[0].inputs[0]); + const auto& roundtripped_filter = std::get(sink_decls[0].inputs[0]); const auto& filter_opts = - checked_cast(*(roundtripped_filter.options)); + checked_cast(*(roundtripped_filter.options)); auto roundtripped_expr = filter_opts.filter_expression; if (auto* call = roundtripped_expr.call()) { @@ -2283,7 +2281,7 @@ TEST(SubstraitRoundTrip, BasicPlan) { } // scan declaration const auto& roundtripped_scan = - std::get(roundtripped_filter.inputs[0]); + std::get(roundtripped_filter.inputs[0]); const auto& dataset_opts = checked_cast(*(roundtripped_scan.options)); const auto& roundripped_ds = dataset_opts.dataset; @@ -2359,12 +2357,11 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { auto comp_right_value = compute::field_ref(filter_col_right); auto filter = compute::equal(comp_left_value, comp_right_value); - auto declarations = compute::Declaration::Sequence( - {compute::Declaration( - {"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"}), - compute::Declaration({"filter", compute::FilterNodeOptions{filter}, "f"})}); + auto declarations = acero::Declaration::Sequence( + {acero::Declaration({"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"}), + acero::Declaration({"filter", acero::FilterNodeOptions{filter}, "f"})}); - ASSERT_OK_AND_ASSIGN(auto expected_table, compute::DeclarationToTable(declarations)); + ASSERT_OK_AND_ASSIGN(auto expected_table, acero::DeclarationToTable(declarations)); std::shared_ptr sp_ext_id_reg = MakeExtensionIdRegistry(); ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get(); @@ -2377,9 +2374,9 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { DeserializePlans( *serialized_plan, [] { return kNullConsumer; }, ext_id_reg, &ext_set)); // filter declaration - auto& roundtripped_filter = std::get(sink_decls[0].inputs[0]); + auto& roundtripped_filter = std::get(sink_decls[0].inputs[0]); const auto& filter_opts = - checked_cast(*(roundtripped_filter.options)); + checked_cast(*(roundtripped_filter.options)); auto roundtripped_expr = filter_opts.filter_expression; if (auto* call = roundtripped_expr.call()) { @@ -2392,7 +2389,7 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { } // scan declaration const auto& roundtripped_scan = - std::get(roundtripped_filter.inputs[0]); + std::get(roundtripped_filter.inputs[0]); const auto& dataset_opts = checked_cast(*(roundtripped_scan.options)); const auto& roundripped_ds = dataset_opts.dataset; @@ -2411,8 +2408,8 @@ TEST(SubstraitRoundTrip, BasicPlanEndToEnd) { EXPECT_TRUE(l_frag->Equals(*r_frag)); } ASSERT_OK_AND_ASSIGN(auto rnd_trp_table, - compute::DeclarationToTable(roundtripped_filter)); - compute::AssertTablesEqualIgnoringOrder(expected_table, rnd_trp_table); + acero::DeclarationToTable(roundtripped_filter)); + engine::AssertTablesEqualIgnoringOrder(expected_table, rnd_trp_table); } TEST(SubstraitRoundTrip, FilterNamedTable) { @@ -2423,11 +2420,10 @@ TEST(SubstraitRoundTrip, FilterNamedTable) { schema({field("A", int32()), field("B", int32()), field("C", int32())}); auto filter = compute::equal(compute::field_ref("A"), compute::field_ref("B")); - auto declarations = compute::Declaration::Sequence( - {compute::Declaration({"named_table", - compute::NamedTableNodeOptions{table_names, dummy_schema}, - "n"}), - compute::Declaration({"filter", compute::FilterNodeOptions{filter}, "f"})}); + auto declarations = acero::Declaration::Sequence( + {acero::Declaration( + {"named_table", acero::NamedTableNodeOptions{table_names, dummy_schema}, "n"}), + acero::Declaration({"filter", acero::FilterNodeOptions{filter}, "f"})}); ExtensionSet ext_set{}; ASSERT_OK_AND_ASSIGN(auto serialized_plan, SerializePlan(declarations, &ext_set)); @@ -2445,7 +2441,7 @@ TEST(SubstraitRoundTrip, FilterNamedTable) { NamedTableProvider table_provider = [&input_table, &table_names, &dummy_schema]( const std::vector& names, - const Schema& schema) -> Result { + const Schema& schema) -> Result { if (table_names != names) { return Status::Invalid("Table name mismatch"); } @@ -2453,9 +2449,9 @@ TEST(SubstraitRoundTrip, FilterNamedTable) { return Status::Invalid("Table schema mismatch"); } - std::shared_ptr options = - std::make_shared(input_table); - return compute::Declaration("table_source", {}, std::move(options), "mock_source"); + std::shared_ptr options = + std::make_shared(input_table); + return acero::Declaration("table_source", {}, std::move(options), "mock_source"); }; ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); @@ -3199,9 +3195,9 @@ TEST(SubstraitRoundTrip, JoinRel) { output_table = right_table; } } - std::shared_ptr options = - std::make_shared(std::move(output_table)); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(std::move(output_table)); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; @@ -3350,9 +3346,9 @@ TEST(SubstraitRoundTrip, JoinRelWithEmit) { output_table = right_table; } } - std::shared_ptr options = - std::make_shared(std::move(output_table)); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(std::move(output_table)); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; @@ -3691,11 +3687,11 @@ TEST(Substrait, IsthmusPlan) { NamedTableProvider ProvideMadeTable( std::function>(const std::vector&)> make) { return [make](const std::vector& names, - const Schema&) -> Result { + const Schema&) -> Result { ARROW_ASSIGN_OR_RAISE(auto table, make(names)); - std::shared_ptr options = - std::make_shared(table); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(table); + return acero::Declaration("table_source", {}, options, "mock_source"); }; } @@ -4265,9 +4261,9 @@ TEST(Substrait, SetRelationBasic) { output_table = table2; } } - std::shared_ptr options = - std::make_shared(std::move(output_table)); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(std::move(output_table)); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; @@ -4530,7 +4526,7 @@ TEST(Substrait, PlanWithAsOfJoinExtension) { ValidateNumProjectNodes(1, buf, conversion_options); ASSERT_OK_AND_ASSIGN( auto out_schema, - compute::asofjoin::MakeOutputSchema( + acero::asofjoin::MakeOutputSchema( input_schema, {{FieldRef(0), {FieldRef(1)}}, {FieldRef(0), {FieldRef(1)}}})); auto expected_table = TableFromJSON( out_schema, {"[[2, 1, 1.1, 1.2], [4, 1, 2.1, 1.2], [6, 2, 3.1, 3.2]]"}); @@ -4807,9 +4803,9 @@ TEST(Substrait, CompoundEmitFilterless) { output_table = right_table; } } - std::shared_ptr options = - std::make_shared(std::move(output_table)); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(std::move(output_table)); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; @@ -5125,7 +5121,7 @@ TEST(Substrait, CompoundEmitWithFilter) { NamedTableProvider table_provider = [left_table, right_table]( const std::vector& names, - const Schema&) -> Result { + const Schema&) -> Result { std::shared_ptr
output_table; for (const auto& name : names) { if (name == "left") { @@ -5138,9 +5134,9 @@ TEST(Substrait, CompoundEmitWithFilter) { if (!output_table) { return Status::Invalid("NamedTableProvider couldn't set the table"); } - std::shared_ptr options = - std::make_shared(std::move(output_table)); - return compute::Declaration("table_source", {}, options, "mock_source"); + std::shared_ptr options = + std::make_shared(std::move(output_table)); + return acero::Declaration("table_source", {}, options, "mock_source"); }; ConversionOptions conversion_options; @@ -5330,7 +5326,7 @@ TEST(Substrait, PlanWithExtension) { ASSERT_OK_AND_ASSIGN( auto out_schema, - compute::asofjoin::MakeOutputSchema( + acero::asofjoin::MakeOutputSchema( input_schema, {{FieldRef(0), {FieldRef(1)}}, {FieldRef(0), {FieldRef(1)}}})); auto expected_table = TableFromJSON( out_schema, {"[[2, 1, 1.1, 1.2], [4, 1, 2.1, 1.2], [6, 2, 3.1, 3.2]]"}); @@ -5592,10 +5588,10 @@ TEST(Substrait, PlanWithNamedTapExtension) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); conversion_options.named_tap_provider = - [](const std::string& tap_kind, std::vector inputs, + [](const std::string& tap_kind, std::vector inputs, const std::string& tap_name, - std::shared_ptr tap_schema) -> Result { - return compute::Declaration{tap_kind, std::move(inputs), compute::ExecNodeOptions{}}; + std::shared_ptr tap_schema) -> Result { + return acero::Declaration{tap_kind, std::move(inputs), acero::ExecNodeOptions{}}; }; ASSERT_OK_AND_ASSIGN(auto buf, internal::SubstraitFromJSON("Plan", substrait_json)); diff --git a/cpp/src/arrow/engine/substrait/test_util.cc b/cpp/src/arrow/engine/substrait/test_util.cc new file mode 100644 index 00000000000..149d3c2fa9a --- /dev/null +++ b/cpp/src/arrow/engine/substrait/test_util.cc @@ -0,0 +1,88 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/engine/substrait/test_util.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" +#include "arrow/acero/util.h" +#include "arrow/compute/api_vector.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/function_internal.h" +#include "arrow/datum.h" +#include "arrow/io/interfaces.h" +#include "arrow/record_batch.h" +#include "arrow/table.h" +#include "arrow/testing/builder.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/random.h" +#include "arrow/type.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/iterator.h" +#include "arrow/util/logging.h" +#include "arrow/util/unreachable.h" +#include "arrow/util/vector.h" + +namespace arrow { + +using compute::SortKey; +using compute::SortOptions; +using compute::Take; +using internal::Executor; + +namespace engine { + +Result> SortTableOnAllFields(const std::shared_ptr
& tab) { + std::vector sort_keys; + 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 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()); + } else { + ASSERT_OK_AND_ASSIGN(auto exp_sorted, SortTableOnAllFields(exp)); + ASSERT_OK_AND_ASSIGN(auto act_sorted, SortTableOnAllFields(act)); + + AssertTablesEqual(*exp_sorted, *act_sorted, + /*same_chunk_layout=*/false, /*flatten=*/true); + } +} + +} // namespace engine +} // namespace arrow diff --git a/cpp/src/arrow/engine/substrait/test_util.h b/cpp/src/arrow/engine/substrait/test_util.h new file mode 100644 index 00000000000..a1db4b255ed --- /dev/null +++ b/cpp/src/arrow/engine/substrait/test_util.h @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "arrow/testing/gtest_util.h" +#include "arrow/util/vector.h" + +#include +#include +#include +#include +#include + +#include "arrow/acero/exec_plan.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/kernel.h" +#include "arrow/testing/visibility.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/pcg_random.h" + +namespace arrow { +namespace engine { + +Result> SortTableOnAllFields(const std::shared_ptr
& tab); + +void AssertTablesEqualIgnoringOrder(const std::shared_ptr
& exp, + const std::shared_ptr
& act); + +} // namespace engine +} // namespace arrow diff --git a/cpp/src/arrow/engine/substrait/util.cc b/cpp/src/arrow/engine/substrait/util.cc index a00a7470fc2..b74e333fd97 100644 --- a/cpp/src/arrow/engine/substrait/util.cc +++ b/cpp/src/arrow/engine/substrait/util.cc @@ -22,10 +22,10 @@ #include #include +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/buffer.h" #include "arrow/compute/exec.h" -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" #include "arrow/compute/type_fwd.h" #include "arrow/engine/substrait/extension_set.h" #include "arrow/engine/substrait/relation.h" @@ -48,13 +48,13 @@ Result> ExecuteSerializedPlan( ARROW_ASSIGN_OR_RAISE(PlanInfo plan_info, DeserializePlan(substrait_buffer, registry, /*ext_set_out=*/nullptr, conversion_options)); - compute::QueryOptions query_options; + acero::QueryOptions query_options; query_options.memory_pool = memory_pool; query_options.function_registry = func_registry; query_options.use_threads = use_threads; query_options.field_names = plan_info.names; - return compute::DeclarationToReader(std::move(plan_info.root.declaration), - std::move(query_options)); + return acero::DeclarationToReader(std::move(plan_info.root.declaration), + std::move(query_options)); } Result> SerializeJsonPlan(const std::string& substrait_json) { diff --git a/cpp/src/arrow/flight/sql/example/acero_server.cc b/cpp/src/arrow/flight/sql/example/acero_server.cc index c65ad186f84..f4b839f501e 100644 --- a/cpp/src/arrow/flight/sql/example/acero_server.cc +++ b/cpp/src/arrow/flight/sql/example/acero_server.cc @@ -22,8 +22,8 @@ #include #include -#include "arrow/compute/exec/exec_plan.h" -#include "arrow/compute/exec/options.h" +#include "arrow/acero/exec_plan.h" +#include "arrow/acero/options.h" #include "arrow/engine/substrait/serde.h" #include "arrow/flight/sql/types.h" #include "arrow/type.h" @@ -96,10 +96,10 @@ class AceroFlightSqlServer : public FlightSqlServerBase { ARROW_LOG(INFO) << "DoGetStatement: executing plan " - << compute::DeclarationToString(plan.root.declaration).ValueOr("Invalid plan"); + << acero::DeclarationToString(plan.root.declaration).ValueOr("Invalid plan"); ARROW_ASSIGN_OR_RAISE(auto reader, - compute::DeclarationToReader(plan.root.declaration)); + acero::DeclarationToReader(plan.root.declaration)); return std::make_unique(std::move(reader)); } @@ -160,7 +160,7 @@ class AceroFlightSqlServer : public FlightSqlServerBase { const std::string& serialized_plan) { std::shared_ptr plan_buf = Buffer::FromString(serialized_plan); ARROW_ASSIGN_OR_RAISE(engine::PlanInfo plan, engine::DeserializePlan(*plan_buf)); - return compute::DeclarationToSchema(plan.root.declaration); + return acero::DeclarationToSchema(plan.root.declaration); } arrow::Result> MakeFlightInfo( diff --git a/python/CMakeLists.txt b/python/CMakeLists.txt index 66be0049371..68484c45d2f 100644 --- a/python/CMakeLists.txt +++ b/python/CMakeLists.txt @@ -110,15 +110,15 @@ endif() # Top level cmake dir if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") + option(PYARROW_BUILD_ACERO "Build the PyArrow Acero integration" OFF) option(PYARROW_BUILD_CUDA "Build the PyArrow CUDA support" OFF) - option(PYARROW_BUILD_FLIGHT "Build the PyArrow Flight integration" OFF) - option(PYARROW_BUILD_SUBSTRAIT "Build the PyArrow Substrait integration" OFF) option(PYARROW_BUILD_DATASET "Build the PyArrow Dataset integration" OFF) + option(PYARROW_BUILD_FLIGHT "Build the PyArrow Flight integration" OFF) option(PYARROW_BUILD_GANDIVA "Build the PyArrow Gandiva integration" OFF) + option(PYARROW_BUILD_ORC "Build the PyArrow ORC integration" OFF) option(PYARROW_BUILD_PARQUET "Build the PyArrow Parquet integration" OFF) option(PYARROW_BUILD_PARQUET_ENCRYPTION "Build the PyArrow Parquet encryption integration" OFF) - option(PYARROW_BUILD_ORC "Build the PyArrow ORC integration" OFF) option(PYARROW_BUNDLE_ARROW_CPP "Bundle the Arrow C++ libraries" OFF) option(PYARROW_BUNDLE_CYTHON_CPP "Bundle the C++ files generated by Cython" OFF) option(PYARROW_GENERATE_COVERAGE "Build with Cython code coverage enabled" OFF) @@ -306,6 +306,18 @@ if(PYARROW_BUILD_DATASET) endif() endif() +if(PYARROW_BUILD_ACERO) + if(NOT ARROW_ACERO) + message(FATAL_ERROR "You must build Arrow C++ with ARROW_ACERO=ON") + endif() + find_package(ArrowAcero REQUIRED) + if(ARROW_BUILD_SHARED) + list(APPEND PYARROW_CPP_LINK_LIBS ArrowAcero::arrow_acero_shared) + else() + list(APPEND PYARROW_CPP_LINK_LIBS ArrowAcero::arrow_acero_static) + endif() +endif() + if(PYARROW_BUILD_PARQUET OR PYARROW_BUILD_PARQUET_ENCRYPTION) if(NOT ARROW_PARQUET) message(FATAL_ERROR "You must build Arrow C++ with ARROW_PARQUET=ON") @@ -517,7 +529,6 @@ set(CYTHON_EXTENSIONS lib _compute _csv - _acero _feather _fs _hdfsio @@ -557,6 +568,19 @@ if(PYARROW_BUILD_CUDA) set_source_files_properties(pyarrow/_cuda.pyx PROPERTIES CYTHON_API TRUE) endif() +# Acero +if(PYARROW_BUILD_ACERO) + if(PYARROW_BUNDLE_ARROW_CPP) + bundle_arrow_lib(${ARROW_ACERO_SHARED_LIB} SO_VERSION ${ARROW_SO_VERSION}) + if(MSVC) + bundle_arrow_import_lib(${ARROW_ACERO_IMPORT_LIB}) + endif() + endif() + + set(ACERO_LINK_LIBS ArrowAcero::arrow_acero_shared) + list(APPEND CYTHON_EXTENSIONS _acero) +endif() + # Dataset if(PYARROW_BUILD_DATASET) if(PYARROW_BUNDLE_ARROW_CPP) @@ -756,6 +780,10 @@ if(PYARROW_BUILD_SUBSTRAIT) target_link_libraries(_substrait PRIVATE ${SUBSTRAIT_LINK_LIBS}) endif() +if(PYARROW_BUILD_ACERO) + target_link_libraries(_acero PRIVATE ${ACERO_LINK_LIBS}) +endif() + if(PYARROW_BUILD_DATASET) target_link_libraries(_dataset PRIVATE ${DATASET_LINK_LIBS}) if(PYARROW_BUILD_ORC) diff --git a/python/pyarrow/_acero.pxd b/python/pyarrow/_acero.pxd index 84d90b24dbb..4553aee9d6f 100644 --- a/python/pyarrow/_acero.pxd +++ b/python/pyarrow/_acero.pxd @@ -20,6 +20,7 @@ from pyarrow.lib cimport * from pyarrow.includes.common cimport * from pyarrow.includes.libarrow cimport * +from pyarrow.includes.libarrow_acero cimport * cdef class ExecNodeOptions(_Weakrefable): diff --git a/python/pyarrow/_acero.pyx b/python/pyarrow/_acero.pyx index b799abb3e3a..af39ce361fc 100644 --- a/python/pyarrow/_acero.pyx +++ b/python/pyarrow/_acero.pyx @@ -24,7 +24,7 @@ from pyarrow.includes.common cimport * from pyarrow.includes.libarrow cimport * -from pyarrow.includes.libarrow_dataset cimport * +from pyarrow.includes.libarrow_acero cimport * from pyarrow.lib cimport (Table, pyarrow_unwrap_table, pyarrow_wrap_table, RecordBatchReader) from pyarrow.lib import frombytes, tobytes @@ -527,3 +527,35 @@ cdef class Declaration(_Weakrefable): GetResultValue(DeclarationToReader(self.unwrap(), use_threads)).release() ) return reader + + +def _group_by(table, aggregates, keys): + cdef: + shared_ptr[CTable] c_table + vector[CAggregate] c_aggregates + vector[CFieldRef] c_keys + CAggregate c_aggr + + c_table = (
table).sp_table + + for aggr_arg_indices, aggr_func_name, aggr_opts, aggr_name in aggregates: + c_aggr.function = tobytes(aggr_func_name) + if aggr_opts is not None: + c_aggr.options = (aggr_opts).wrapped + else: + c_aggr.options = nullptr + for field_idx in aggr_arg_indices: + c_aggr.target.push_back(CFieldRef( field_idx)) + + c_aggr.name = tobytes(aggr_name) + c_aggregates.push_back(move(c_aggr)) + + for key_idx in keys: + c_keys.push_back(CFieldRef( key_idx)) + + with nogil: + sp_table = GetResultValue( + CTableGroupBy(c_table, c_aggregates, c_keys) + ) + + return pyarrow_wrap_table(sp_table) diff --git a/python/pyarrow/_compute.pyx b/python/pyarrow/_compute.pyx index dd980abad73..a5db5be5514 100644 --- a/python/pyarrow/_compute.pyx +++ b/python/pyarrow/_compute.pyx @@ -2257,38 +2257,6 @@ class RankOptions(_RankOptions): self._set_options(sort_keys, null_placement, tiebreaker) -def _group_by(table, aggregates, keys): - cdef: - shared_ptr[CTable] c_table - vector[CAggregate] c_aggregates - vector[CFieldRef] c_keys - CAggregate c_aggr - - c_table = (
table).sp_table - - for aggr_arg_indices, aggr_func_name, aggr_opts, aggr_name in aggregates: - c_aggr.function = tobytes(aggr_func_name) - if aggr_opts is not None: - c_aggr.options = (aggr_opts).wrapped - else: - c_aggr.options = nullptr - for field_idx in aggr_arg_indices: - c_aggr.target.push_back(CFieldRef( field_idx)) - - c_aggr.name = tobytes(aggr_name) - c_aggregates.push_back(move(c_aggr)) - - for key_idx in keys: - c_keys.push_back(CFieldRef( key_idx)) - - with nogil: - sp_table = GetResultValue( - CTableGroupBy(c_table, c_aggregates, c_keys) - ) - - return pyarrow_wrap_table(sp_table) - - cdef class Expression(_Weakrefable): """ A logical expression to be evaluated against some input. diff --git a/python/pyarrow/compute.py b/python/pyarrow/compute.py index efc1e1d9562..b42e0f30a2a 100644 --- a/python/pyarrow/compute.py +++ b/python/pyarrow/compute.py @@ -80,7 +80,6 @@ function_registry, get_function, list_functions, - _group_by, # Udf call_tabular_function, register_scalar_function, diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd index bf1907c354e..798f0e8395d 100644 --- a/python/pyarrow/includes/libarrow.pxd +++ b/python/pyarrow/includes/libarrow.pxd @@ -1915,6 +1915,10 @@ cdef extern from "arrow/util/thread_pool.h" namespace "arrow::internal" nogil: cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil: + cdef cppclass CExecBatch "arrow::compute::ExecBatch": + vector[CDatum] values + int64_t length + cdef cppclass CExecContext" arrow::compute::ExecContext": CExecContext() CExecContext(CMemoryPool* pool) @@ -2518,6 +2522,7 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil: cdef c_string ToString(DatumType kind) + cdef extern from * namespace "arrow::compute": # inlined from compute/function_internal.h to avoid exposing # implementation details @@ -2537,20 +2542,6 @@ cdef extern from * namespace "arrow::compute": const CBuffer& buffer) -cdef extern from "arrow/compute/exec/groupby.h" namespace \ - "arrow::compute" nogil: - cdef cppclass CAggregate "arrow::compute::Aggregate": - c_string function - shared_ptr[CFunctionOptions] options - vector[CFieldRef] target - c_string name - - CResult[shared_ptr[CTable]] CTableGroupBy "arrow::compute::TableGroupBy"( - shared_ptr[CTable] table, - vector[CAggregate] aggregates, - vector[CFieldRef] keys) - - cdef extern from * namespace "arrow::compute": # inlined from expression_internal.h to avoid # proliferation of #include @@ -2605,131 +2596,6 @@ cdef extern from "arrow/compute/expression.h" \ const CExpression& partition_expression) -cdef extern from "arrow/compute/exec/options.h" namespace "arrow::compute" nogil: - cdef enum CJoinType "arrow::compute::JoinType": - CJoinType_LEFT_SEMI "arrow::compute::JoinType::LEFT_SEMI" - CJoinType_RIGHT_SEMI "arrow::compute::JoinType::RIGHT_SEMI" - CJoinType_LEFT_ANTI "arrow::compute::JoinType::LEFT_ANTI" - CJoinType_RIGHT_ANTI "arrow::compute::JoinType::RIGHT_ANTI" - CJoinType_INNER "arrow::compute::JoinType::INNER" - CJoinType_LEFT_OUTER "arrow::compute::JoinType::LEFT_OUTER" - CJoinType_RIGHT_OUTER "arrow::compute::JoinType::RIGHT_OUTER" - CJoinType_FULL_OUTER "arrow::compute::JoinType::FULL_OUTER" - - cdef cppclass CAsyncExecBatchGenerator "arrow::compute::AsyncExecBatchGenerator": - pass - - cdef cppclass CExecNodeOptions "arrow::compute::ExecNodeOptions": - pass - - cdef cppclass CSourceNodeOptions "arrow::compute::SourceNodeOptions"(CExecNodeOptions): - pass - - cdef cppclass CTableSourceNodeOptions "arrow::compute::TableSourceNodeOptions"(CExecNodeOptions): - CTableSourceNodeOptions(shared_ptr[CTable] table) - CTableSourceNodeOptions(shared_ptr[CTable] table, int64_t max_batch_size) - - cdef cppclass CSinkNodeOptions "arrow::compute::SinkNodeOptions"(CExecNodeOptions): - pass - - cdef cppclass CFilterNodeOptions "arrow::compute::FilterNodeOptions"(CExecNodeOptions): - CFilterNodeOptions(CExpression) - - cdef cppclass CProjectNodeOptions "arrow::compute::ProjectNodeOptions"(CExecNodeOptions): - CProjectNodeOptions(vector[CExpression] expressions) - CProjectNodeOptions(vector[CExpression] expressions, - vector[c_string] names) - - cdef cppclass CAggregateNodeOptions "arrow::compute::AggregateNodeOptions"(CExecNodeOptions): - CAggregateNodeOptions(vector[CAggregate] aggregates, vector[CFieldRef] names) - - cdef cppclass COrderBySinkNodeOptions "arrow::compute::OrderBySinkNodeOptions"(CExecNodeOptions): - COrderBySinkNodeOptions(vector[CSortOptions] options, - CAsyncExecBatchGenerator generator) - - cdef cppclass COrderByNodeOptions "arrow::compute::OrderByNodeOptions"(CExecNodeOptions): - COrderByNodeOptions(COrdering ordering) - - cdef cppclass CHashJoinNodeOptions "arrow::compute::HashJoinNodeOptions"(CExecNodeOptions): - CHashJoinNodeOptions(CJoinType, vector[CFieldRef] in_left_keys, - vector[CFieldRef] in_right_keys) - CHashJoinNodeOptions(CJoinType, vector[CFieldRef] in_left_keys, - vector[CFieldRef] in_right_keys, - CExpression filter, - c_string output_suffix_for_left, - c_string output_suffix_for_right) - CHashJoinNodeOptions(CJoinType join_type, - vector[CFieldRef] left_keys, - vector[CFieldRef] right_keys, - vector[CFieldRef] left_output, - vector[CFieldRef] right_output, - CExpression filter, - c_string output_suffix_for_left, - c_string output_suffix_for_right) - - -cdef extern from "arrow/compute/exec/exec_plan.h" namespace "arrow::compute" nogil: - cdef cppclass CDeclaration "arrow::compute::Declaration": - cppclass Input: - Input(CExecNode*) - Input(CDeclaration) - - c_string label - vector[Input] inputs - - CDeclaration() - CDeclaration(c_string factory_name, CExecNodeOptions options) - CDeclaration(c_string factory_name, vector[Input] inputs, shared_ptr[CExecNodeOptions] options) - - @staticmethod - CDeclaration Sequence(vector[CDeclaration] decls) - - CResult[CExecNode*] AddToPlan(CExecPlan* plan) const - - cdef cppclass CExecPlan "arrow::compute::ExecPlan": - @staticmethod - CResult[shared_ptr[CExecPlan]] Make(CExecContext* exec_context) - - void StartProducing() - CStatus Validate() - CStatus StopProducing() - - CFuture_Void finished() - - vector[CExecNode*] sinks() const - vector[CExecNode*] sources() const - - cdef cppclass CExecNode "arrow::compute::ExecNode": - const vector[CExecNode*]& inputs() const - const shared_ptr[CSchema]& output_schema() const - - cdef cppclass CExecBatch "arrow::compute::ExecBatch": - vector[CDatum] values - int64_t length - - shared_ptr[CRecordBatchReader] MakeGeneratorReader( - shared_ptr[CSchema] schema, - CAsyncExecBatchGenerator gen, - CMemoryPool* memory_pool - ) - CResult[CExecNode*] MakeExecNode(c_string factory_name, CExecPlan* plan, - vector[CExecNode*] inputs, - const CExecNodeOptions& options) - - CResult[shared_ptr[CTable]] DeclarationToTable( - CDeclaration declaration, c_bool use_threads - ) - CResult[shared_ptr[CTable]] DeclarationToTable( - CDeclaration declaration, c_bool use_threads, - CMemoryPool* memory_pool, CFunctionRegistry* function_registry - ) - CResult[unique_ptr[CRecordBatchReader]] DeclarationToReader( - CDeclaration declaration, c_bool use_threads - ) - - CResult[c_string] DeclarationToString(const CDeclaration& declaration) - - cdef extern from "arrow/extension_type.h" namespace "arrow": cdef cppclass CExtensionTypeRegistry" arrow::ExtensionTypeRegistry": @staticmethod diff --git a/python/pyarrow/includes/libarrow_acero.pxd b/python/pyarrow/includes/libarrow_acero.pxd new file mode 100644 index 00000000000..5ff5ad99dc1 --- /dev/null +++ b/python/pyarrow/includes/libarrow_acero.pxd @@ -0,0 +1,160 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# distutils: language = c++ + +from pyarrow.includes.common cimport * +from pyarrow.includes.libarrow cimport * + + +cdef extern from "arrow/acero/groupby.h" namespace \ + "arrow::acero" nogil: + cdef cppclass CAggregate "arrow::compute::Aggregate": + c_string function + shared_ptr[CFunctionOptions] options + vector[CFieldRef] target + c_string name + + CResult[shared_ptr[CTable]] CTableGroupBy "arrow::acero::TableGroupBy"( + shared_ptr[CTable] table, + vector[CAggregate] aggregates, + vector[CFieldRef] keys) + + +cdef extern from "arrow/acero/options.h" namespace "arrow::acero" nogil: + cdef enum CJoinType "arrow::acero::JoinType": + CJoinType_LEFT_SEMI "arrow::acero::JoinType::LEFT_SEMI" + CJoinType_RIGHT_SEMI "arrow::acero::JoinType::RIGHT_SEMI" + CJoinType_LEFT_ANTI "arrow::acero::JoinType::LEFT_ANTI" + CJoinType_RIGHT_ANTI "arrow::acero::JoinType::RIGHT_ANTI" + CJoinType_INNER "arrow::acero::JoinType::INNER" + CJoinType_LEFT_OUTER "arrow::acero::JoinType::LEFT_OUTER" + CJoinType_RIGHT_OUTER "arrow::acero::JoinType::RIGHT_OUTER" + CJoinType_FULL_OUTER "arrow::acero::JoinType::FULL_OUTER" + + cdef cppclass CAsyncExecBatchGenerator "arrow::acero::AsyncExecBatchGenerator": + pass + + cdef cppclass CExecNodeOptions "arrow::acero::ExecNodeOptions": + pass + + cdef cppclass CSourceNodeOptions "arrow::acero::SourceNodeOptions"(CExecNodeOptions): + pass + + cdef cppclass CTableSourceNodeOptions "arrow::acero::TableSourceNodeOptions"(CExecNodeOptions): + CTableSourceNodeOptions(shared_ptr[CTable] table) + CTableSourceNodeOptions(shared_ptr[CTable] table, int64_t max_batch_size) + + cdef cppclass CSinkNodeOptions "arrow::acero::SinkNodeOptions"(CExecNodeOptions): + pass + + cdef cppclass CFilterNodeOptions "arrow::acero::FilterNodeOptions"(CExecNodeOptions): + CFilterNodeOptions(CExpression) + + cdef cppclass CProjectNodeOptions "arrow::acero::ProjectNodeOptions"(CExecNodeOptions): + CProjectNodeOptions(vector[CExpression] expressions) + CProjectNodeOptions(vector[CExpression] expressions, + vector[c_string] names) + + cdef cppclass CAggregateNodeOptions "arrow::acero::AggregateNodeOptions"(CExecNodeOptions): + CAggregateNodeOptions(vector[CAggregate] aggregates, vector[CFieldRef] names) + + cdef cppclass COrderBySinkNodeOptions "arrow::acero::OrderBySinkNodeOptions"(CExecNodeOptions): + COrderBySinkNodeOptions(vector[CSortOptions] options, + CAsyncExecBatchGenerator generator) + + cdef cppclass COrderByNodeOptions "arrow::acero::OrderByNodeOptions"(CExecNodeOptions): + COrderByNodeOptions(COrdering ordering) + + cdef cppclass CHashJoinNodeOptions "arrow::acero::HashJoinNodeOptions"(CExecNodeOptions): + CHashJoinNodeOptions(CJoinType, vector[CFieldRef] in_left_keys, + vector[CFieldRef] in_right_keys) + CHashJoinNodeOptions(CJoinType, vector[CFieldRef] in_left_keys, + vector[CFieldRef] in_right_keys, + CExpression filter, + c_string output_suffix_for_left, + c_string output_suffix_for_right) + CHashJoinNodeOptions(CJoinType join_type, + vector[CFieldRef] left_keys, + vector[CFieldRef] right_keys, + vector[CFieldRef] left_output, + vector[CFieldRef] right_output, + CExpression filter, + c_string output_suffix_for_left, + c_string output_suffix_for_right) + + +cdef extern from "arrow/acero/exec_plan.h" namespace "arrow::acero" nogil: + cdef cppclass CDeclaration "arrow::acero::Declaration": + cppclass Input: + Input(CExecNode*) + Input(CDeclaration) + + c_string label + vector[Input] inputs + + CDeclaration() + CDeclaration(c_string factory_name, CExecNodeOptions options) + CDeclaration(c_string factory_name, vector[Input] inputs, shared_ptr[CExecNodeOptions] options) + + @staticmethod + CDeclaration Sequence(vector[CDeclaration] decls) + + CResult[CExecNode*] AddToPlan(CExecPlan* plan) const + + cdef cppclass CExecPlan "arrow::acero::ExecPlan": + @staticmethod + CResult[shared_ptr[CExecPlan]] Make(CExecContext* exec_context) + + void StartProducing() + CStatus Validate() + CStatus StopProducing() + + CFuture_Void finished() + + vector[CExecNode*] sinks() const + vector[CExecNode*] sources() const + + cdef cppclass CExecNode "arrow::acero::ExecNode": + const vector[CExecNode*]& inputs() const + const shared_ptr[CSchema]& output_schema() const + + cdef cppclass CExecBatch "arrow::acero::ExecBatch": + vector[CDatum] values + int64_t length + + shared_ptr[CRecordBatchReader] MakeGeneratorReader( + shared_ptr[CSchema] schema, + CAsyncExecBatchGenerator gen, + CMemoryPool* memory_pool + ) + CResult[CExecNode*] MakeExecNode(c_string factory_name, CExecPlan* plan, + vector[CExecNode*] inputs, + const CExecNodeOptions& options) + + CResult[shared_ptr[CTable]] DeclarationToTable( + CDeclaration declaration, c_bool use_threads + ) + CResult[shared_ptr[CTable]] DeclarationToTable( + CDeclaration declaration, c_bool use_threads, + CMemoryPool* memory_pool, CFunctionRegistry* function_registry + ) + CResult[unique_ptr[CRecordBatchReader]] DeclarationToReader( + CDeclaration declaration, c_bool use_threads + ) + + CResult[c_string] DeclarationToString(const CDeclaration& declaration) diff --git a/python/pyarrow/includes/libarrow_dataset.pxd b/python/pyarrow/includes/libarrow_dataset.pxd index 16037970849..b554633e4b1 100644 --- a/python/pyarrow/includes/libarrow_dataset.pxd +++ b/python/pyarrow/includes/libarrow_dataset.pxd @@ -22,6 +22,7 @@ from libcpp cimport bool as c_bool from pyarrow.includes.common cimport * from pyarrow.includes.libarrow cimport * +from pyarrow.includes.libarrow_acero cimport * from pyarrow.includes.libarrow_fs cimport * diff --git a/python/pyarrow/includes/libarrow_substrait.pxd b/python/pyarrow/includes/libarrow_substrait.pxd index a3631ba3322..eabccb2b4a3 100644 --- a/python/pyarrow/includes/libarrow_substrait.pxd +++ b/python/pyarrow/includes/libarrow_substrait.pxd @@ -21,6 +21,7 @@ from libcpp.vector cimport vector as std_vector from pyarrow.includes.common cimport * from pyarrow.includes.libarrow cimport * +from pyarrow.includes.libarrow_acero cimport * ctypedef CResult[CDeclaration] CNamedTableProvider(const std_vector[c_string]&, const CSchema&) diff --git a/python/pyarrow/table.pxi b/python/pyarrow/table.pxi index d8ab5477c76..82b71eeb9ac 100644 --- a/python/pyarrow/table.pxi +++ b/python/pyarrow/table.pxi @@ -5618,4 +5618,4 @@ list[tuple(str, str, FunctionOptions)] key_indices = [ self._table.schema.get_field_index(k) for k in self.keys] - return _pc()._group_by(self._table, group_by_aggrs, key_indices) + return _pac()._group_by(self._table, group_by_aggrs, key_indices) diff --git a/python/pyarrow/tests/test_acero.py b/python/pyarrow/tests/test_acero.py index c0f0e850761..40388f3e142 100644 --- a/python/pyarrow/tests/test_acero.py +++ b/python/pyarrow/tests/test_acero.py @@ -23,13 +23,14 @@ from pyarrow.acero import ( TableSourceNodeOptions, + Declaration, FilterNodeOptions, ProjectNodeOptions, AggregateNodeOptions, - OrderByNodeOptions, HashJoinNodeOptions, - Declaration, + OrderByNodeOptions ) +from pyarrow.compute import field try: import pyarrow.dataset as ds @@ -122,7 +123,8 @@ def test_project(table_source): # provide name decl = Declaration.from_sequence([ table_source, - Declaration("project", ProjectNodeOptions([pc.multiply(field("a"), 2)], ["a2"])) + Declaration("project", ProjectNodeOptions( + [pc.multiply(field("a"), 2)], ["a2"])) ]) result = decl.to_table() assert result.schema.names == ["a2"] @@ -144,7 +146,8 @@ def test_project(table_source): def test_aggregate_scalar(table_source): decl = Declaration.from_sequence([ table_source, - Declaration("aggregate", AggregateNodeOptions([("a", "sum", None, "a_sum")])) + Declaration("aggregate", AggregateNodeOptions( + [("a", "sum", None, "a_sum")])) ]) result = decl.to_table() assert result.schema.names == ["a_sum"] @@ -243,26 +246,30 @@ def test_order_by(): table_source = Declaration("table_source", TableSourceNodeOptions(table)) ord_opts = OrderByNodeOptions([("b", "ascending")]) - decl = Declaration.from_sequence([table_source, Declaration("order_by", ord_opts)]) + decl = Declaration.from_sequence( + [table_source, Declaration("order_by", ord_opts)]) result = decl.to_table() expected = pa.table({"a": [1, 4, 2, 3], "b": [1, 2, 3, None]}) assert result.equals(expected) ord_opts = OrderByNodeOptions([(field("b"), "descending")]) - decl = Declaration.from_sequence([table_source, Declaration("order_by", ord_opts)]) + decl = Declaration.from_sequence( + [table_source, Declaration("order_by", ord_opts)]) result = decl.to_table() expected = pa.table({"a": [2, 4, 1, 3], "b": [3, 2, 1, None]}) assert result.equals(expected) ord_opts = OrderByNodeOptions([(1, "descending")], null_placement="at_start") - decl = Declaration.from_sequence([table_source, Declaration("order_by", ord_opts)]) + decl = Declaration.from_sequence( + [table_source, Declaration("order_by", ord_opts)]) result = decl.to_table() expected = pa.table({"a": [3, 2, 4, 1], "b": [None, 3, 2, 1]}) assert result.equals(expected) # emtpy ordering ord_opts = OrderByNodeOptions([]) - decl = Declaration.from_sequence([table_source, Declaration("order_by", ord_opts)]) + decl = Declaration.from_sequence( + [table_source, Declaration("order_by", ord_opts)]) with pytest.raises( ValueError, match="`ordering` must be an explicit non-empty ordering" ): @@ -277,9 +284,11 @@ def test_order_by(): def test_hash_join(): left = pa.table({'key': [1, 2, 3], 'a': [4, 5, 6]}) - left_source = Declaration("table_source", options=TableSourceNodeOptions(left)) + left_source = Declaration( + "table_source", options=TableSourceNodeOptions(left)) right = pa.table({'key': [2, 3, 4], 'b': [4, 5, 6]}) - right_source = Declaration("table_source", options=TableSourceNodeOptions(right)) + right_source = Declaration( + "table_source", options=TableSourceNodeOptions(right)) # inner join join_opts = HashJoinNodeOptions("inner", left_keys="key", right_keys="key") diff --git a/python/setup.py b/python/setup.py index e7bbb83e297..16faf9c3c23 100755 --- a/python/setup.py +++ b/python/setup.py @@ -107,6 +107,7 @@ def run(self): ('with-cuda', None, 'build the Cuda extension'), ('with-flight', None, 'build the Flight extension'), ('with-substrait', None, 'build the Substrait extension'), + ('with-acero', None, 'build the Acero Engine extension'), ('with-dataset', None, 'build the Dataset extension'), ('with-parquet', None, 'build the Parquet extension'), ('with-parquet-encryption', None, @@ -160,6 +161,8 @@ def initialize_options(self): os.environ.get('PYARROW_WITH_SUBSTRAIT', '0')) self.with_flight = strtobool( os.environ.get('PYARROW_WITH_FLIGHT', '0')) + self.with_acero = strtobool( + os.environ.get('PYARROW_WITH_ACERO', '0')) self.with_dataset = strtobool( os.environ.get('PYARROW_WITH_DATASET', '0')) self.with_parquet = strtobool( @@ -180,6 +183,12 @@ def initialize_options(self): self.with_parquet_encryption = (self.with_parquet_encryption and self.with_parquet) + # enforce module dependencies + if self.with_substrait: + self.with_dataset = True + if self.with_dataset: + self.with_acero = True + CYTHON_MODULE_NAMES = [ 'lib', '_fs', @@ -263,6 +272,7 @@ def append_cmake_bool(value, varname): append_cmake_bool(self.with_substrait, 'PYARROW_BUILD_SUBSTRAIT') append_cmake_bool(self.with_flight, 'PYARROW_BUILD_FLIGHT') append_cmake_bool(self.with_gandiva, 'PYARROW_BUILD_GANDIVA') + append_cmake_bool(self.with_acero, 'PYARROW_BUILD_ACERO') append_cmake_bool(self.with_dataset, 'PYARROW_BUILD_DATASET') append_cmake_bool(self.with_orc, 'PYARROW_BUILD_ORC') append_cmake_bool(self.with_parquet, 'PYARROW_BUILD_PARQUET') @@ -342,6 +352,10 @@ def _failure_permitted(self, name): return True if name == '_dataset' and not self.with_dataset: return True + if name == '_acero' and not self.with_acero: + return True + if name == '_exec_plan' and not self.with_acero: + return True if name == '_dataset_orc' and not ( self.with_orc and self.with_dataset ): diff --git a/r/configure b/r/configure index dc0e8e63192..2e9f426f442 100755 --- a/r/configure +++ b/r/configure @@ -265,6 +265,12 @@ if [ $? -eq 0 ]; then # NOTE: parquet is assumed to have the same -L flag as arrow # so there is no need to add its location to PKG_DIRS fi + if arrow_built_with ARROW_ACERO; then + PKG_CFLAGS="$PKG_CFLAGS -DARROW_R_WITH_ACERO" + PKG_LIBS="-larrow_acero $PKG_LIBS" + # NOTE: arrow-dataset is assumed to have the same -L flag as arrow + # so there is no need to add its location to PKG_DIRS + fi if arrow_built_with ARROW_DATASET; then PKG_CFLAGS="$PKG_CFLAGS -DARROW_R_WITH_DATASET" PKG_LIBS="-larrow_dataset $PKG_LIBS" diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp index 31223615710..db6d39dd7b8 100644 --- a/r/src/arrowExports.cpp +++ b/r/src/arrowExports.cpp @@ -912,7 +912,7 @@ BEGIN_CPP11 END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecPlan_create(bool use_threads); +std::shared_ptr ExecPlan_create(bool use_threads); extern "C" SEXP _arrow_ExecPlan_create(SEXP use_threads_sexp){ BEGIN_CPP11 arrow::r::Input::type use_threads(use_threads_sexp); @@ -936,7 +936,7 @@ BEGIN_CPP11 END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecPlanReader__Plan(const std::shared_ptr& reader); +std::shared_ptr ExecPlanReader__Plan(const std::shared_ptr& reader); extern "C" SEXP _arrow_ExecPlanReader__Plan(SEXP reader_sexp){ BEGIN_CPP11 arrow::r::Input&>::type reader(reader_sexp); @@ -952,11 +952,11 @@ BEGIN_CPP11 END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecPlan_run(const std::shared_ptr& plan, const std::shared_ptr& final_node, cpp11::list sort_options, cpp11::strings metadata, int64_t head); +std::shared_ptr ExecPlan_run(const std::shared_ptr& plan, const std::shared_ptr& final_node, cpp11::list sort_options, cpp11::strings metadata, int64_t head); extern "C" SEXP _arrow_ExecPlan_run(SEXP plan_sexp, SEXP final_node_sexp, SEXP sort_options_sexp, SEXP metadata_sexp, SEXP head_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); - arrow::r::Input&>::type final_node(final_node_sexp); + arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type final_node(final_node_sexp); arrow::r::Input::type sort_options(sort_options_sexp); arrow::r::Input::type metadata(metadata_sexp); arrow::r::Input::type head(head_sexp); @@ -964,36 +964,36 @@ BEGIN_CPP11 END_CPP11 } // compute-exec.cpp -std::string ExecPlan_ToString(const std::shared_ptr& plan); +std::string ExecPlan_ToString(const std::shared_ptr& plan); extern "C" SEXP _arrow_ExecPlan_ToString(SEXP plan_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); return cpp11::as_sexp(ExecPlan_ToString(plan)); END_CPP11 } // compute-exec.cpp -void ExecPlan_UnsafeDelete(const std::shared_ptr& plan); +void ExecPlan_UnsafeDelete(const std::shared_ptr& plan); extern "C" SEXP _arrow_ExecPlan_UnsafeDelete(SEXP plan_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); ExecPlan_UnsafeDelete(plan); return R_NilValue; END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_output_schema(const std::shared_ptr& node); +std::shared_ptr ExecNode_output_schema(const std::shared_ptr& node); extern "C" SEXP _arrow_ExecNode_output_schema(SEXP node_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type node(node_sexp); + arrow::r::Input&>::type node(node_sexp); return cpp11::as_sexp(ExecNode_output_schema(node)); END_CPP11 } // compute-exec.cpp #if defined(ARROW_R_WITH_DATASET) -std::shared_ptr ExecNode_Scan(const std::shared_ptr& plan, const std::shared_ptr& dataset, const std::shared_ptr& filter, cpp11::list projection); +std::shared_ptr ExecNode_Scan(const std::shared_ptr& plan, const std::shared_ptr& dataset, const std::shared_ptr& filter, cpp11::list projection); extern "C" SEXP _arrow_ExecNode_Scan(SEXP plan_sexp, SEXP dataset_sexp, SEXP filter_sexp, SEXP projection_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); arrow::r::Input&>::type dataset(dataset_sexp); arrow::r::Input&>::type filter(filter_sexp); arrow::r::Input::type projection(projection_sexp); @@ -1008,11 +1008,11 @@ extern "C" SEXP _arrow_ExecNode_Scan(SEXP plan_sexp, SEXP dataset_sexp, SEXP fil // compute-exec.cpp #if defined(ARROW_R_WITH_DATASET) -void ExecPlan_Write(const std::shared_ptr& plan, const std::shared_ptr& final_node, cpp11::strings metadata, const std::shared_ptr& file_write_options, const std::shared_ptr& filesystem, std::string base_dir, const std::shared_ptr& partitioning, std::string basename_template, arrow::dataset::ExistingDataBehavior existing_data_behavior, int max_partitions, uint32_t max_open_files, uint64_t max_rows_per_file, uint64_t min_rows_per_group, uint64_t max_rows_per_group); +void ExecPlan_Write(const std::shared_ptr& plan, const std::shared_ptr& final_node, cpp11::strings metadata, const std::shared_ptr& file_write_options, const std::shared_ptr& filesystem, std::string base_dir, const std::shared_ptr& partitioning, std::string basename_template, arrow::dataset::ExistingDataBehavior existing_data_behavior, int max_partitions, uint32_t max_open_files, uint64_t max_rows_per_file, uint64_t min_rows_per_group, uint64_t max_rows_per_group); extern "C" SEXP _arrow_ExecPlan_Write(SEXP plan_sexp, SEXP final_node_sexp, SEXP metadata_sexp, SEXP file_write_options_sexp, SEXP filesystem_sexp, SEXP base_dir_sexp, SEXP partitioning_sexp, SEXP basename_template_sexp, SEXP existing_data_behavior_sexp, SEXP max_partitions_sexp, SEXP max_open_files_sexp, SEXP max_rows_per_file_sexp, SEXP min_rows_per_group_sexp, SEXP max_rows_per_group_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); - arrow::r::Input&>::type final_node(final_node_sexp); + arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type final_node(final_node_sexp); arrow::r::Input::type metadata(metadata_sexp); arrow::r::Input&>::type file_write_options(file_write_options_sexp); arrow::r::Input&>::type filesystem(filesystem_sexp); @@ -1036,41 +1036,41 @@ extern "C" SEXP _arrow_ExecPlan_Write(SEXP plan_sexp, SEXP final_node_sexp, SEXP #endif // compute-exec.cpp -std::shared_ptr ExecNode_Filter(const std::shared_ptr& input, const std::shared_ptr& filter); +std::shared_ptr ExecNode_Filter(const std::shared_ptr& input, const std::shared_ptr& filter); extern "C" SEXP _arrow_ExecNode_Filter(SEXP input_sexp, SEXP filter_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); + arrow::r::Input&>::type input(input_sexp); arrow::r::Input&>::type filter(filter_sexp); return cpp11::as_sexp(ExecNode_Filter(input, filter)); END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_Project(const std::shared_ptr& input, const std::vector>& exprs, std::vector names); +std::shared_ptr ExecNode_Project(const std::shared_ptr& input, const std::vector>& exprs, std::vector names); extern "C" SEXP _arrow_ExecNode_Project(SEXP input_sexp, SEXP exprs_sexp, SEXP names_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); + arrow::r::Input&>::type input(input_sexp); arrow::r::Input>&>::type exprs(exprs_sexp); arrow::r::Input>::type names(names_sexp); return cpp11::as_sexp(ExecNode_Project(input, exprs, names)); END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_Aggregate(const std::shared_ptr& input, cpp11::list options, std::vector key_names); +std::shared_ptr ExecNode_Aggregate(const std::shared_ptr& input, cpp11::list options, std::vector key_names); extern "C" SEXP _arrow_ExecNode_Aggregate(SEXP input_sexp, SEXP options_sexp, SEXP key_names_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); + arrow::r::Input&>::type input(input_sexp); arrow::r::Input::type options(options_sexp); arrow::r::Input>::type key_names(key_names_sexp); return cpp11::as_sexp(ExecNode_Aggregate(input, options, key_names)); END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_Join(const std::shared_ptr& input, compute::JoinType join_type, const std::shared_ptr& right_data, std::vector left_keys, std::vector right_keys, std::vector left_output, std::vector right_output, std::string output_suffix_for_left, std::string output_suffix_for_right); +std::shared_ptr ExecNode_Join(const std::shared_ptr& input, acero::JoinType join_type, const std::shared_ptr& right_data, std::vector left_keys, std::vector right_keys, std::vector left_output, std::vector right_output, std::string output_suffix_for_left, std::string output_suffix_for_right); extern "C" SEXP _arrow_ExecNode_Join(SEXP input_sexp, SEXP join_type_sexp, SEXP right_data_sexp, SEXP left_keys_sexp, SEXP right_keys_sexp, SEXP left_output_sexp, SEXP right_output_sexp, SEXP output_suffix_for_left_sexp, SEXP output_suffix_for_right_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); - arrow::r::Input::type join_type(join_type_sexp); - arrow::r::Input&>::type right_data(right_data_sexp); + arrow::r::Input&>::type input(input_sexp); + arrow::r::Input::type join_type(join_type_sexp); + arrow::r::Input&>::type right_data(right_data_sexp); arrow::r::Input>::type left_keys(left_keys_sexp); arrow::r::Input>::type right_keys(right_keys_sexp); arrow::r::Input>::type left_output(left_output_sexp); @@ -1081,28 +1081,28 @@ BEGIN_CPP11 END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_Union(const std::shared_ptr& input, const std::shared_ptr& right_data); +std::shared_ptr ExecNode_Union(const std::shared_ptr& input, const std::shared_ptr& right_data); extern "C" SEXP _arrow_ExecNode_Union(SEXP input_sexp, SEXP right_data_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type input(input_sexp); - arrow::r::Input&>::type right_data(right_data_sexp); + arrow::r::Input&>::type input(input_sexp); + arrow::r::Input&>::type right_data(right_data_sexp); return cpp11::as_sexp(ExecNode_Union(input, right_data)); END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_SourceNode(const std::shared_ptr& plan, const std::shared_ptr& reader); +std::shared_ptr ExecNode_SourceNode(const std::shared_ptr& plan, const std::shared_ptr& reader); extern "C" SEXP _arrow_ExecNode_SourceNode(SEXP plan_sexp, SEXP reader_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); arrow::r::Input&>::type reader(reader_sexp); return cpp11::as_sexp(ExecNode_SourceNode(plan, reader)); END_CPP11 } // compute-exec.cpp -std::shared_ptr ExecNode_TableSourceNode(const std::shared_ptr& plan, const std::shared_ptr& table); +std::shared_ptr ExecNode_TableSourceNode(const std::shared_ptr& plan, const std::shared_ptr& table); extern "C" SEXP _arrow_ExecNode_TableSourceNode(SEXP plan_sexp, SEXP table_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); arrow::r::Input&>::type table(table_sexp); return cpp11::as_sexp(ExecNode_TableSourceNode(plan, table)); END_CPP11 @@ -1139,10 +1139,10 @@ extern "C" SEXP _arrow_substrait__internal__SubstraitFromJSON(SEXP substrait_jso // compute-exec.cpp #if defined(ARROW_R_WITH_SUBSTRAIT) -std::shared_ptr ExecPlan_run_substrait(const std::shared_ptr& plan, const std::shared_ptr& serialized_plan); +std::shared_ptr ExecPlan_run_substrait(const std::shared_ptr& plan, const std::shared_ptr& serialized_plan); extern "C" SEXP _arrow_ExecPlan_run_substrait(SEXP plan_sexp, SEXP serialized_plan_sexp){ BEGIN_CPP11 - arrow::r::Input&>::type plan(plan_sexp); + arrow::r::Input&>::type plan(plan_sexp); arrow::r::Input&>::type serialized_plan(serialized_plan_sexp); return cpp11::as_sexp(ExecPlan_run_substrait(plan, serialized_plan)); END_CPP11 diff --git a/r/src/arrow_types.h b/r/src/arrow_types.h index 2f6597ad8f4..8f5071a707e 100644 --- a/r/src/arrow_types.h +++ b/r/src/arrow_types.h @@ -32,13 +32,14 @@ // forward declaration-only headers #include #include +#include #include #if defined(ARROW_R_WITH_DATASET) #include #endif -#include +#include #include #include @@ -51,15 +52,6 @@ #include #include -namespace arrow { -namespace compute { - -class ExecPlan; -class ExecNode; - -} // namespace compute -} // namespace arrow - class ExecPlanReader; #if defined(ARROW_R_WITH_PARQUET) @@ -71,6 +63,7 @@ namespace ds = ::arrow::dataset; #endif namespace compute = ::arrow::compute; +namespace acero = ::arrow::acero; namespace fs = ::arrow::fs; std::shared_ptr RecordBatch__from_arrays(SEXP, SEXP); diff --git a/r/src/compute-exec.cpp b/r/src/compute-exec.cpp index eddb91dc8a1..6adfbdc7262 100644 --- a/r/src/compute-exec.cpp +++ b/r/src/compute-exec.cpp @@ -18,9 +18,9 @@ #include "./arrow_types.h" #include "./safe-call-into-r.h" +#include #include #include -#include #include #include #include @@ -38,22 +38,22 @@ std::shared_ptr make_compute_options(std::string func_ std::shared_ptr strings_to_kvm(cpp11::strings metadata); // [[arrow::export]] -std::shared_ptr ExecPlan_create(bool use_threads) { +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 auto plan = ValueOrStop( - compute::ExecPlan::Make(use_threads ? &threaded_context : gc_context())); + acero::ExecPlan::Make(use_threads ? &threaded_context : gc_context())); return plan; } -std::shared_ptr MakeExecNodeOrStop( - const std::string& factory_name, compute::ExecPlan* plan, - std::vector inputs, const compute::ExecNodeOptions& options) { - return std::shared_ptr( - ValueOrStop(compute::MakeExecNode(factory_name, plan, std::move(inputs), options)), +std::shared_ptr MakeExecNodeOrStop( + const std::string& factory_name, acero::ExecPlan* plan, + std::vector inputs, const acero::ExecNodeOptions& options) { + return std::shared_ptr( + ValueOrStop(acero::MakeExecNode(factory_name, plan, std::move(inputs), options)), [](...) { // empty destructor: ExecNode lifetime is managed by an ExecPlan }); @@ -72,7 +72,7 @@ class ExecPlanReader : public arrow::RecordBatchReader { public: enum ExecPlanReaderStatus { PLAN_NOT_STARTED, PLAN_RUNNING, PLAN_FINISHED }; - ExecPlanReader(const std::shared_ptr& plan, + ExecPlanReader(const std::shared_ptr& plan, const std::shared_ptr& schema, arrow::AsyncGenerator> sink_gen) : schema_(schema), @@ -146,13 +146,13 @@ class ExecPlanReader : public arrow::RecordBatchReader { return arrow::Status::OK(); } - const std::shared_ptr& Plan() const { return plan_; } + const std::shared_ptr& Plan() const { return plan_; } ~ExecPlanReader() { StopProducing(); } private: std::shared_ptr schema_; - std::shared_ptr plan_; + std::shared_ptr plan_; arrow::AsyncGenerator> sink_gen_; ExecPlanReaderStatus plan_status_; arrow::StopToken stop_token_; @@ -168,7 +168,7 @@ class ExecPlanReader : public arrow::RecordBatchReader { // to finish and clean up after itself. To do this, we give a // callable with its own copy of the shared_ptr so // that it can delete itself when it is safe to do so. - std::shared_ptr plan(plan_); + std::shared_ptr plan(plan_); bool not_finished_yet = plan_->finished().TryAddCallback( [&plan] { return [plan](const arrow::Status&) {}; }); @@ -202,7 +202,7 @@ std::shared_ptr Table__from_ExecPlanReader( } // [[arrow::export]] -std::shared_ptr ExecPlanReader__Plan( +std::shared_ptr ExecPlanReader__Plan( const std::shared_ptr& reader) { if (reader->PlanStatus() == "PLAN_FINISHED") { cpp11::stop("Can't extract ExecPlan from a finished ExecPlanReader"); @@ -218,8 +218,8 @@ std::string ExecPlanReader__PlanStatus(const std::shared_ptr& re // [[arrow::export]] std::shared_ptr ExecPlan_run( - const std::shared_ptr& plan, - const std::shared_ptr& final_node, cpp11::list sort_options, + const std::shared_ptr& plan, + const std::shared_ptr& final_node, cpp11::list sort_options, cpp11::strings metadata, int64_t head = -1) { // For now, don't require R to construct SinkNodes. // Instead, just pass the node we should collect as an argument. @@ -231,7 +231,7 @@ std::shared_ptr ExecPlan_run( // Use the SelectK node to take only what we need MakeExecNodeOrStop( "select_k_sink", plan.get(), {final_node.get()}, - compute::SelectKSinkNodeOptions{ + acero::SelectKSinkNodeOptions{ arrow::compute::SelectKOptions( head, std::dynamic_pointer_cast( make_compute_options("sort_indices", sort_options)) @@ -239,14 +239,14 @@ std::shared_ptr ExecPlan_run( &sink_gen}); } else { MakeExecNodeOrStop("order_by_sink", plan.get(), {final_node.get()}, - compute::OrderBySinkNodeOptions{ + acero::OrderBySinkNodeOptions{ *std::dynamic_pointer_cast( make_compute_options("sort_indices", sort_options)), &sink_gen}); } } else { MakeExecNodeOrStop("sink", plan.get(), {final_node.get()}, - compute::SinkNodeOptions{&sink_gen}); + acero::SinkNodeOptions{&sink_gen}); } StopIfNotOk(plan->Validate()); @@ -262,19 +262,19 @@ std::shared_ptr ExecPlan_run( } // [[arrow::export]] -std::string ExecPlan_ToString(const std::shared_ptr& plan) { +std::string ExecPlan_ToString(const std::shared_ptr& plan) { return plan->ToString(); } // [[arrow::export]] -void ExecPlan_UnsafeDelete(const std::shared_ptr& plan) { - auto& plan_unsafe = const_cast&>(plan); +void ExecPlan_UnsafeDelete(const std::shared_ptr& plan) { + auto& plan_unsafe = const_cast&>(plan); plan_unsafe.reset(); } // [[arrow::export]] std::shared_ptr ExecNode_output_schema( - const std::shared_ptr& node) { + const std::shared_ptr& node) { return node->output_schema(); } @@ -285,8 +285,8 @@ std::shared_ptr ExecNode_output_schema( #include // [[dataset::export]] -std::shared_ptr ExecNode_Scan( - const std::shared_ptr& plan, +std::shared_ptr ExecNode_Scan( + const std::shared_ptr& plan, const std::shared_ptr& dataset, const std::shared_ptr& filter, cpp11::list projection) { arrow::dataset::internal::Initialize(); @@ -320,8 +320,8 @@ std::shared_ptr ExecNode_Scan( // [[dataset::export]] void ExecPlan_Write( - const std::shared_ptr& plan, - const std::shared_ptr& final_node, cpp11::strings metadata, + const std::shared_ptr& plan, + const std::shared_ptr& final_node, cpp11::strings metadata, const std::shared_ptr& file_write_options, const std::shared_ptr& filesystem, std::string base_dir, const std::shared_ptr& partitioning, std::string basename_template, @@ -362,16 +362,16 @@ void ExecPlan_Write( #endif // [[arrow::export]] -std::shared_ptr ExecNode_Filter( - const std::shared_ptr& input, +std::shared_ptr ExecNode_Filter( + const std::shared_ptr& input, const std::shared_ptr& filter) { return MakeExecNodeOrStop("filter", input->plan(), {input.get()}, - compute::FilterNodeOptions{*filter}); + acero::FilterNodeOptions{*filter}); } // [[arrow::export]] -std::shared_ptr ExecNode_Project( - const std::shared_ptr& input, +std::shared_ptr ExecNode_Project( + const std::shared_ptr& input, const std::vector>& exprs, std::vector names) { // We have shared_ptrs of expressions but need the Expressions @@ -381,12 +381,12 @@ std::shared_ptr ExecNode_Project( } return MakeExecNodeOrStop( "project", input->plan(), {input.get()}, - compute::ProjectNodeOptions{std::move(expressions), std::move(names)}); + acero::ProjectNodeOptions{std::move(expressions), std::move(names)}); } // [[arrow::export]] -std::shared_ptr ExecNode_Aggregate( - const std::shared_ptr& input, cpp11::list options, +std::shared_ptr ExecNode_Aggregate( + const std::shared_ptr& input, cpp11::list options, std::vector key_names) { std::vector aggregates; @@ -410,13 +410,13 @@ std::shared_ptr ExecNode_Aggregate( } return MakeExecNodeOrStop( "aggregate", input->plan(), {input.get()}, - compute::AggregateNodeOptions{std::move(aggregates), std::move(keys)}); + acero::AggregateNodeOptions{std::move(aggregates), std::move(keys)}); } // [[arrow::export]] -std::shared_ptr ExecNode_Join( - const std::shared_ptr& input, compute::JoinType join_type, - const std::shared_ptr& right_data, +std::shared_ptr ExecNode_Join( + const std::shared_ptr& input, acero::JoinType join_type, + const std::shared_ptr& right_data, std::vector left_keys, std::vector right_keys, std::vector left_output, std::vector right_output, std::string output_suffix_for_left, std::string output_suffix_for_right) { @@ -432,8 +432,8 @@ std::shared_ptr ExecNode_Join( } // dplyr::semi_join => LEFT_SEMI; dplyr::anti_join => LEFT_ANTI // So ignoring RIGHT_SEMI and RIGHT_ANTI here because dplyr doesn't implement them. - if (join_type != compute::JoinType::LEFT_SEMI && - join_type != compute::JoinType::LEFT_ANTI) { + if (join_type != acero::JoinType::LEFT_SEMI && + join_type != acero::JoinType::LEFT_ANTI) { // Don't include out_refs in semi/anti join for (auto&& name : right_output) { right_out_refs.emplace_back(std::move(name)); @@ -442,32 +442,32 @@ std::shared_ptr ExecNode_Join( return MakeExecNodeOrStop( "hashjoin", input->plan(), {input.get(), right_data.get()}, - compute::HashJoinNodeOptions{ + acero::HashJoinNodeOptions{ join_type, std::move(left_refs), std::move(right_refs), std::move(left_out_refs), std::move(right_out_refs), compute::literal(true), std::move(output_suffix_for_left), std::move(output_suffix_for_right)}); } // [[arrow::export]] -std::shared_ptr ExecNode_Union( - const std::shared_ptr& input, - const std::shared_ptr& right_data) { +std::shared_ptr ExecNode_Union( + const std::shared_ptr& input, + const std::shared_ptr& right_data) { return MakeExecNodeOrStop("union", input->plan(), {input.get(), right_data.get()}, {}); } // [[arrow::export]] -std::shared_ptr ExecNode_SourceNode( - const std::shared_ptr& plan, +std::shared_ptr ExecNode_SourceNode( + const std::shared_ptr& plan, const std::shared_ptr& reader) { - arrow::compute::RecordBatchReaderSourceNodeOptions options{reader}; + arrow::acero::RecordBatchReaderSourceNodeOptions options{reader}; return MakeExecNodeOrStop("record_batch_reader_source", plan.get(), {}, options); } // [[arrow::export]] -std::shared_ptr ExecNode_TableSourceNode( - const std::shared_ptr& plan, +std::shared_ptr ExecNode_TableSourceNode( + const std::shared_ptr& plan, const std::shared_ptr& table) { - arrow::compute::TableSourceNodeOptions options{/*table=*/table, + arrow::acero::TableSourceNodeOptions options{/*table=*/table, // TODO: make batch_size configurable /*batch_size=*/1048576}; @@ -480,18 +480,18 @@ std::shared_ptr ExecNode_TableSourceNode( // Just for example usage until a C++ method is available that implements // a RecordBatchReader output (ARROW-15849) -class AccumulatingConsumer : public compute::SinkNodeConsumer { +class AccumulatingConsumer : public acero::SinkNodeConsumer { public: const std::vector>& batches() { return batches_; } arrow::Status Init(const std::shared_ptr& schema, - compute::BackpressureControl* backpressure_control, - compute::ExecPlan* exec_plan) override { + acero::BackpressureControl* backpressure_control, + acero::ExecPlan* exec_plan) override { schema_ = schema; return arrow::Status::OK(); } - arrow::Status Consume(compute::ExecBatch batch) override { + arrow::Status Consume(acero::ExecBatch batch) override { auto record_batch = batch.ToRecordBatch(schema_); ARROW_RETURN_NOT_OK(record_batch); batches_.push_back(record_batch.ValueUnsafe()); @@ -522,22 +522,22 @@ std::shared_ptr substrait__internal__SubstraitFromJSON( // [[substrait::export]] std::shared_ptr ExecPlan_run_substrait( - const std::shared_ptr& plan, + const std::shared_ptr& plan, const std::shared_ptr& serialized_plan) { std::vector> consumers; - std::function()> consumer_factory = [&] { + std::function()> consumer_factory = [&] { consumers.emplace_back(new AccumulatingConsumer()); return consumers.back(); }; - arrow::Result> maybe_decls = + arrow::Result> maybe_decls = ValueOrStop(arrow::engine::DeserializePlans(*serialized_plan, consumer_factory)); - std::vector decls = std::move(ValueOrStop(maybe_decls)); + std::vector decls = std::move(ValueOrStop(maybe_decls)); // For now, the Substrait plan must include a 'read' that points to // a Parquet file (instead of using a source node create in Arrow) - for (const compute::Declaration& decl : decls) { + for (const acero::Declaration& decl : decls) { auto node = decl.AddToPlan(plan.get()); StopIfNotOk(node.status()); } From 7cda1fede222ba0fe1cec7838aa30dda52fc563b Mon Sep 17 00:00:00 2001 From: Davide Pasetto Date: Wed, 29 Mar 2023 17:04:29 -0400 Subject: [PATCH 2/8] lint and build fixes --- cpp/src/arrow/acero/aggregate_node.cc | 14 +++++++------- cpp/src/arrow/acero/asof_join_node.cc | 4 ++-- cpp/src/arrow/acero/exec_plan.cc | 6 +++--- cpp/src/arrow/acero/filter_node.cc | 2 +- cpp/src/arrow/acero/project_node.cc | 2 +- cpp/src/arrow/acero/source_node.cc | 2 +- cpp/src/arrow/acero/util.cc | 6 +++--- cpp/src/arrow/compute/key_map.h | 2 +- cpp/src/arrow/flight/sql/example/acero_server.cc | 3 +-- python/pyarrow/acero.py | 1 + 10 files changed, 21 insertions(+), 21 deletions(-) diff --git a/cpp/src/arrow/acero/aggregate_node.cc b/cpp/src/arrow/acero/aggregate_node.cc index 23ede24fdff..81a6239a512 100644 --- a/cpp/src/arrow/acero/aggregate_node.cc +++ b/cpp/src/arrow/acero/aggregate_node.cc @@ -383,7 +383,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { Status DoConsume(const ExecSpan& batch, size_t thread_index) { for (size_t i = 0; i < kernels_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, {{"function.name", aggs_[i].function}, {"function.options", @@ -486,7 +486,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { batch.values.resize(kernels_.size() + segment_field_ids_.size()); for (size_t i = 0; i < kernels_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, {{"function.name", aggs_[i].function}, {"function.options", @@ -695,7 +695,7 @@ class GroupByNode : public ExecNode, public TracedNode { // Execute aggregate kernels for (size_t i = 0; i < agg_kernels_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, {{"function.name", aggs_[i].function}, {"function.options", @@ -719,7 +719,7 @@ class GroupByNode : public ExecNode, public TracedNode { } Status Merge() { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, "Merge", {{"group_by", ToStringExtra()}, {"node.label", label()}}); ThreadLocalState* state0 = &local_states_[0]; @@ -735,7 +735,7 @@ class GroupByNode : public ExecNode, public TracedNode { state->grouper.reset(); for (size_t i = 0; i < agg_kernels_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN( span, aggs_[i].function, {{"function.name", aggs_[i].function}, @@ -758,7 +758,7 @@ class GroupByNode : public ExecNode, public TracedNode { } Result Finalize() { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, "Finalize", {{"group_by", ToStringExtra()}, {"node.label", label()}}); @@ -772,7 +772,7 @@ class GroupByNode : public ExecNode, public TracedNode { // Aggregate fields come before key fields to match the behavior of GroupBy function for (size_t i = 0; i < agg_kernels_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, {{"function.name", aggs_[i].function}, {"function.options", diff --git a/cpp/src/arrow/acero/asof_join_node.cc b/cpp/src/arrow/acero/asof_join_node.cc index 1cff9159430..f4d0ee9443f 100644 --- a/cpp/src/arrow/acero/asof_join_node.cc +++ b/cpp/src/arrow/acero/asof_join_node.cc @@ -329,7 +329,7 @@ struct MemoStore { // a specialized higher-performance variation of Hashing64 logic from hash_join_node // the code here avoids recreating objects that are independent of each batch processed class KeyHasher { - static constexpr int kMiniBatchLength = util::MiniBatch::kMiniBatchLength; + static constexpr int kMiniBatchLength = arrow::util::MiniBatch::kMiniBatchLength; public: explicit KeyHasher(const std::vector& indices) @@ -383,7 +383,7 @@ class KeyHasher { std::vector hashes_; LightContext ctx_; std::vector column_arrays_; - util::TempVectorStack stack_; + arrow::util::TempVectorStack stack_; }; class BackpressureController : public BackpressureControl { diff --git a/cpp/src/arrow/acero/exec_plan.cc b/cpp/src/arrow/acero/exec_plan.cc index 59cc32d000b..886a659d5c0 100644 --- a/cpp/src/arrow/acero/exec_plan.cc +++ b/cpp/src/arrow/acero/exec_plan.cc @@ -124,8 +124,8 @@ struct ExecPlanImpl : public ExecPlan { // part of StartProducing) then the plan may be finished before we return from this // call. auto scope = START_SCOPED_SPAN(span_, "ExecPlan", {{"plan", ToString()}}); - Future<> scheduler_finished = util::AsyncTaskScheduler::Make( - [this](util::AsyncTaskScheduler* async_scheduler) { + Future<> scheduler_finished = arrow::util::AsyncTaskScheduler::Make( + [this](arrow::util::AsyncTaskScheduler* async_scheduler) { QueryContext* ctx = query_context(); RETURN_NOT_OK(ctx->Init(ctx->max_concurrency(), async_scheduler)); @@ -336,7 +336,7 @@ struct ExecPlanImpl : public ExecPlan { NodeVector node_ptrs_; NodeVector sorted_nodes_; uint32_t auto_label_counter_ = 0; - util::tracing::Span span_; + arrow::util::tracing::Span span_; std::shared_ptr metadata_; QueryContext query_context_; // This field only exists for backwards compatibility. Remove once the deprecated diff --git a/cpp/src/arrow/acero/filter_node.cc b/cpp/src/arrow/acero/filter_node.cc index c5d8b181b5f..28b23e4dd6f 100644 --- a/cpp/src/arrow/acero/filter_node.cc +++ b/cpp/src/arrow/acero/filter_node.cc @@ -74,7 +74,7 @@ class FilterNode : public MapNode { ARROW_ASSIGN_OR_RAISE(Expression simplified_filter, SimplifyWithGuarantee(filter_, batch.guarantee)); - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, "Filter", {{"filter.expression", ToStringExtra()}, {"filter.expression.simplified", simplified_filter.ToString()}, diff --git a/cpp/src/arrow/acero/project_node.cc b/cpp/src/arrow/acero/project_node.cc index 321c4434ff3..562542fb1f5 100644 --- a/cpp/src/arrow/acero/project_node.cc +++ b/cpp/src/arrow/acero/project_node.cc @@ -80,7 +80,7 @@ class ProjectNode : public MapNode { Result ProcessBatch(ExecBatch batch) override { std::vector values{exprs_.size()}; for (size_t i = 0; i < exprs_.size(); ++i) { - util::tracing::Span span; + arrow::util::tracing::Span span; START_COMPUTE_SPAN(span, "Project", {{"project.type", exprs_[i].type()->ToString()}, {"project.length", batch.length}, diff --git a/cpp/src/arrow/acero/source_node.cc b/cpp/src/arrow/acero/source_node.cc index 24898d686bb..8ec6e378233 100644 --- a/cpp/src/arrow/acero/source_node.cc +++ b/cpp/src/arrow/acero/source_node.cc @@ -157,7 +157,7 @@ struct SourceNode : ExecNode, public TracedNode { } lock.unlock(); - util::tracing::Span fetch_batch_span; + arrow::util::tracing::Span fetch_batch_span; auto fetch_batch_scope = START_SCOPED_SPAN(fetch_batch_span, "SourceNode::ReadBatch"); return generator_().Then( diff --git a/cpp/src/arrow/acero/util.cc b/cpp/src/arrow/acero/util.cc index d0b448ad6c3..17d82e15ae3 100644 --- a/cpp/src/arrow/acero/util.cc +++ b/cpp/src/arrow/acero/util.cc @@ -99,7 +99,7 @@ Future<> TableSinkNodeConsumer::Finish() { [[nodiscard]] ::arrow::internal::tracing::Scope TracedNode::TraceStartProducing( std::string extra_details) const { std::string node_kind(node_->kind_name()); - util::tracing::Span span; + arrow::util::tracing::Span span; return START_SCOPED_SPAN( span, node_kind + "::StartProducing", {{"node.details", extra_details}, {"node.label", node_->label()}}); @@ -114,7 +114,7 @@ void TracedNode::NoteStartProducing(std::string extra_details) const { [[nodiscard]] ::arrow::internal::tracing::Scope TracedNode::TraceInputReceived( const ExecBatch& batch) const { std::string node_kind(node_->kind_name()); - util::tracing::Span span; + arrow::util::tracing::Span span; return START_SCOPED_SPAN( span, node_kind + "::InputReceived", {{"node.label", node_->label()}, {"node.batch_length", batch.length}}); @@ -129,7 +129,7 @@ void TracedNode::NoteInputReceived(const ExecBatch& batch) const { [[nodiscard]] ::arrow::internal::tracing::Scope TracedNode::TraceFinish() const { std::string node_kind(node_->kind_name()); - util::tracing::Span span; + arrow::util::tracing::Span span; return START_SCOPED_SPAN(span, node_kind + "::Finish", {{"node.label", node_->label()}}); } diff --git a/cpp/src/arrow/compute/key_map.h b/cpp/src/arrow/compute/key_map.h index c34e198e543..5e40b3d6fcf 100644 --- a/cpp/src/arrow/compute/key_map.h +++ b/cpp/src/arrow/compute/key_map.h @@ -70,7 +70,7 @@ class ARROW_EXPORT SwissTable { int minibatch_size() const { return 1 << log_minibatch_; } - int64_t num_inserted() const { return num_inserted_; } + uint32_t num_inserted() const { return num_inserted_; } int64_t hardware_flags() const { return hardware_flags_; } diff --git a/cpp/src/arrow/flight/sql/example/acero_server.cc b/cpp/src/arrow/flight/sql/example/acero_server.cc index f4b839f501e..2f1e48b0bee 100644 --- a/cpp/src/arrow/flight/sql/example/acero_server.cc +++ b/cpp/src/arrow/flight/sql/example/acero_server.cc @@ -98,8 +98,7 @@ class AceroFlightSqlServer : public FlightSqlServerBase { << "DoGetStatement: executing plan " << acero::DeclarationToString(plan.root.declaration).ValueOr("Invalid plan"); - ARROW_ASSIGN_OR_RAISE(auto reader, - acero::DeclarationToReader(plan.root.declaration)); + ARROW_ASSIGN_OR_RAISE(auto reader, acero::DeclarationToReader(plan.root.declaration)); return std::make_unique(std::move(reader)); } diff --git a/python/pyarrow/acero.py b/python/pyarrow/acero.py index 65ffc248d5e..e0b7826b449 100644 --- a/python/pyarrow/acero.py +++ b/python/pyarrow/acero.py @@ -26,6 +26,7 @@ from pyarrow.compute import Expression from pyarrow._acero import ( # noqa + _group_by, Declaration, ExecNodeOptions, TableSourceNodeOptions, From accace3bc49dee6d0d73996be9c097e7ccf22b70 Mon Sep 17 00:00:00 2001 From: Davide Pasetto Date: Wed, 29 Mar 2023 17:56:44 -0400 Subject: [PATCH 3/8] lint --- python/pyarrow/tests/test_acero.py | 1 - r/src/arrow_types.h | 2 +- r/src/compute-exec.cpp | 9 +++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyarrow/tests/test_acero.py b/python/pyarrow/tests/test_acero.py index 40388f3e142..8dbe139e7a0 100644 --- a/python/pyarrow/tests/test_acero.py +++ b/python/pyarrow/tests/test_acero.py @@ -30,7 +30,6 @@ HashJoinNodeOptions, OrderByNodeOptions ) -from pyarrow.compute import field try: import pyarrow.dataset as ds diff --git a/r/src/arrow_types.h b/r/src/arrow_types.h index 8f5071a707e..819346e2053 100644 --- a/r/src/arrow_types.h +++ b/r/src/arrow_types.h @@ -30,9 +30,9 @@ #include // forward declaration-only headers +#include #include #include -#include #include #if defined(ARROW_R_WITH_DATASET) diff --git a/r/src/compute-exec.cpp b/r/src/compute-exec.cpp index 6adfbdc7262..43b544c58ce 100644 --- a/r/src/compute-exec.cpp +++ b/r/src/compute-exec.cpp @@ -44,8 +44,9 @@ std::shared_ptr ExecPlan_create(bool use_threads) { // 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 - auto plan = ValueOrStop( - acero::ExecPlan::Make(use_threads ? &threaded_context : gc_context())); + auto plan = + ValueOrStop(acero::ExecPlan::Make(use_threads ? &threaded_context : gc_context())); + return plan; } @@ -468,8 +469,8 @@ std::shared_ptr ExecNode_TableSourceNode( const std::shared_ptr& plan, const std::shared_ptr& table) { arrow::acero::TableSourceNodeOptions options{/*table=*/table, - // TODO: make batch_size configurable - /*batch_size=*/1048576}; + // TODO: make batch_size configurable + /*batch_size=*/1048576}; return MakeExecNodeOrStop("table_source", plan.get(), {}, options); } From dc657b8abfb69ce52f0024b8ba06db93f4bcdc2c Mon Sep 17 00:00:00 2001 From: Li Jin Date: Thu, 30 Mar 2023 12:28:41 -0400 Subject: [PATCH 4/8] Resolve recent merge conflicts --- cpp/src/arrow/acero/source_node.cc | 4 ++-- cpp/src/arrow/engine/substrait/relation_internal.cc | 6 +++--- cpp/src/arrow/engine/substrait/relation_internal.h | 3 ++- cpp/src/arrow/engine/substrait/serde_test.cc | 6 +++--- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/cpp/src/arrow/acero/source_node.cc b/cpp/src/arrow/acero/source_node.cc index 8ec6e378233..22619d5e7d5 100644 --- a/cpp/src/arrow/acero/source_node.cc +++ b/cpp/src/arrow/acero/source_node.cc @@ -106,8 +106,8 @@ struct SourceNode : ExecNode, public TracedNode { ExecBatch batch = morsel.Slice(offset, batch_size); for (auto& value : batch.values) { if (value.is_array()) { - ARROW_ASSIGN_OR_RAISE( - value, util::EnsureAlignment(value.make_array(), ipc::kArrowAlignment, + ARROW_ASSIGN_OR_RAISE(value, arrow::util::EnsureAlignment( + value.make_array(), ipc::kArrowAlignment, default_memory_pool())); } } diff --git a/cpp/src/arrow/engine/substrait/relation_internal.cc b/cpp/src/arrow/engine/substrait/relation_internal.cc index 17aac62118b..d1a81d3eaf6 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.cc +++ b/cpp/src/arrow/engine/substrait/relation_internal.cc @@ -332,7 +332,7 @@ Result ParseAggregateMeasure( } ARROW_ENGINE_EXPORT Result MakeAggregateDeclaration( - compute::Declaration input_decl, std::shared_ptr input_schema, + acero::Declaration input_decl, std::shared_ptr input_schema, const int measure_size, std::vector aggregates, std::vector> agg_src_fieldsets, std::vector keys, std::vector key_field_ids, std::vector segment_keys, @@ -359,9 +359,9 @@ ARROW_ENGINE_EXPORT Result MakeAggregateDeclaration( std::shared_ptr aggregate_schema = schema(std::move(output_fields)); return DeclarationInfo{ - compute::Declaration::Sequence( + acero::Declaration::Sequence( {std::move(input_decl), - {"aggregate", compute::AggregateNodeOptions{aggregates, keys, segment_keys}}}), + {"aggregate", acero::AggregateNodeOptions{aggregates, keys, segment_keys}}}), aggregate_schema}; } diff --git a/cpp/src/arrow/engine/substrait/relation_internal.h b/cpp/src/arrow/engine/substrait/relation_internal.h index e9085dedf30..72a0c3f98af 100644 --- a/cpp/src/arrow/engine/substrait/relation_internal.h +++ b/cpp/src/arrow/engine/substrait/relation_internal.h @@ -21,6 +21,7 @@ #include +#include "arrow/acero/exec_plan.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/type_fwd.h" #include "arrow/engine/substrait/relation.h" @@ -81,7 +82,7 @@ Result ParseAggregateMeasure( /// \param[in] ext_set an extension mapping to use /// \param[in] conversion_options options to control how the conversion is done ARROW_ENGINE_EXPORT Result MakeAggregateDeclaration( - compute::Declaration input_decl, std::shared_ptr input_schema, + acero::Declaration input_decl, std::shared_ptr input_schema, const int measure_size, std::vector aggregates, std::vector> agg_src_fieldsets, std::vector keys, std::vector key_field_ids, std::vector segment_keys, diff --git a/cpp/src/arrow/engine/substrait/serde_test.cc b/cpp/src/arrow/engine/substrait/serde_test.cc index 8568a496944..02be82972d4 100644 --- a/cpp/src/arrow/engine/substrait/serde_test.cc +++ b/cpp/src/arrow/engine/substrait/serde_test.cc @@ -5715,10 +5715,10 @@ TEST(Substrait, PlanWithSegmentedAggregateExtension) { ConversionOptions conversion_options; conversion_options.named_table_provider = std::move(table_provider); conversion_options.named_tap_provider = - [](const std::string& tap_kind, std::vector inputs, + [](const std::string& tap_kind, std::vector inputs, const std::string& tap_name, - std::shared_ptr tap_schema) -> Result { - return compute::Declaration{tap_kind, std::move(inputs), compute::ExecNodeOptions{}}; + std::shared_ptr tap_schema) -> Result { + return acero::Declaration{tap_kind, std::move(inputs), acero::ExecNodeOptions{}}; }; ASSERT_OK_AND_ASSIGN(auto buf, internal::SubstraitFromJSON("Plan", substrait_json)); From 53e0ad56f2e1d9ea073d8005d6ad4f143867ef7b Mon Sep 17 00:00:00 2001 From: Li Jin Date: Thu, 30 Mar 2023 14:06:47 -0400 Subject: [PATCH 5/8] Try fixing R error --- r/src/compute-exec.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/r/src/compute-exec.cpp b/r/src/compute-exec.cpp index 43b544c58ce..7fb1b1394dc 100644 --- a/r/src/compute-exec.cpp +++ b/r/src/compute-exec.cpp @@ -492,7 +492,7 @@ class AccumulatingConsumer : public acero::SinkNodeConsumer { return arrow::Status::OK(); } - arrow::Status Consume(acero::ExecBatch batch) override { + arrow::Status Consume(compute::ExecBatch batch) override { auto record_batch = batch.ToRecordBatch(schema_); ARROW_RETURN_NOT_OK(record_batch); batches_.push_back(record_batch.ValueUnsafe()); From 9e6f0b32c6ce369e07d6c5fa003c6f33c3834724 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Fri, 31 Mar 2023 07:09:10 -0700 Subject: [PATCH 6/8] Minor fixes to repair AVX2 build --- cpp/src/arrow/acero/CMakeLists.txt | 15 ++++++++--- .../compute/row/compare_internal_avx2.cc | 25 ++++++++++--------- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/cpp/src/arrow/acero/CMakeLists.txt b/cpp/src/arrow/acero/CMakeLists.txt index 431cb53703f..c6fad5f1e42 100644 --- a/cpp/src/arrow/acero/CMakeLists.txt +++ b/cpp/src/arrow/acero/CMakeLists.txt @@ -19,6 +19,15 @@ add_custom_target(arrow_acero) arrow_install_all_headers("arrow/acero") +macro(append_acero_avx2_src SRC) + if(ARROW_HAVE_RUNTIME_AVX2) + list(APPEND ARROW_ACERO_SRCS ${SRC}) + set_source_files_properties(${SRC} PROPERTIES SKIP_PRECOMPILE_HEADERS ON) + set_source_files_properties(${SRC} PROPERTIES COMPILE_FLAGS ${ARROW_AVX2_FLAG}) + endif() +endmacro() + + set(ARROW_ACERO_SRCS groupby.cc accumulation_queue.cc @@ -47,9 +56,9 @@ set(ARROW_ACERO_SRCS union_node.cc util.cc) -append_avx2_src(bloom_filter_avx2.cc) -append_avx2_src(swiss_join_avx2.cc) -append_avx2_src(util_avx2.cc) +append_acero_avx2_src(bloom_filter_avx2.cc) +append_acero_avx2_src(swiss_join_avx2.cc) +append_acero_avx2_src(util_avx2.cc) set(ARROW_ACERO_PKG_CONFIG_REQUIRES arrow) diff --git a/cpp/src/arrow/compute/row/compare_internal_avx2.cc b/cpp/src/arrow/compute/row/compare_internal_avx2.cc index 96dacab6797..e387d4ccd9a 100644 --- a/cpp/src/arrow/compute/row/compare_internal_avx2.cc +++ b/cpp/src/arrow/compute/row/compare_internal_avx2.cc @@ -17,6 +17,7 @@ #include +#include "arrow/acero/util.h" #include "arrow/compute/row/compare_internal.h" #include "arrow/util/bit_util.h" @@ -571,10 +572,10 @@ uint32_t KeyCompare::NullUpdateColumnToRow_avx2( const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { - int64_t num_rows_safe = - TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(0)); + int64_t num_rows_safe = acero::TailSkipForSIMD::FixBitAccess( + sizeof(uint32_t), col.length(), col.bit_offset(0)); if (sel_left_maybe_null) { - num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast(num_rows_safe); @@ -600,19 +601,19 @@ uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( int64_t num_rows_safe = col.length(); if (col_width == 0) { // In this case we will access left column memory 4B at a time - num_rows_safe = - TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(1)); + num_rows_safe = acero::TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), + col.bit_offset(1)); } else if (col_width == 1 || col_width == 2) { // In this case we will access left column memory 4B at a time - num_rows_safe = - TailSkipForSIMD::FixBinaryAccess(sizeof(uint32_t), col.length(), col_width); + num_rows_safe = acero::TailSkipForSIMD::FixBinaryAccess(sizeof(uint32_t), + col.length(), col_width); } else if (col_width != 4 && col_width != 8) { // In this case we will access left column memory 32B at a time num_rows_safe = - TailSkipForSIMD::FixBinaryAccess(sizeof(__m256i), col.length(), col_width); + acero::TailSkipForSIMD::FixBinaryAccess(sizeof(__m256i), col.length(), col_width); } if (sel_left_maybe_null) { - num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast( @@ -635,10 +636,10 @@ uint32_t KeyCompare::CompareVarBinaryColumnToRow_avx2( uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { - int64_t num_rows_safe = - TailSkipForSIMD::FixVarBinaryAccess(sizeof(__m256i), col.length(), col.offsets()); + int64_t num_rows_safe = acero::TailSkipForSIMD::FixVarBinaryAccess( + sizeof(__m256i), col.length(), col.offsets()); if (use_selection) { - num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast(num_rows_safe); From 4791a74370c3752b3b2d44bac36e4cd690990171 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 31 Mar 2023 10:40:36 -0400 Subject: [PATCH 7/8] Move subtree and forest to dataset; Move TailSkipForSIMD; lint change --- cpp/src/arrow/acero/CMakeLists.txt | 1 - cpp/src/arrow/acero/util.h | 46 ------------------- .../compute/row/compare_internal_avx2.cc | 26 +++++------ cpp/src/arrow/compute/util.h | 46 +++++++++++++++++++ cpp/src/arrow/dataset/CMakeLists.txt | 1 + cpp/src/arrow/dataset/file_base.cc | 18 ++++---- .../{acero => dataset}/forest_internal.h | 4 +- .../{acero => dataset}/subtree_internal.h | 4 +- .../arrow/{acero => dataset}/subtree_test.cc | 6 +-- 9 files changed, 76 insertions(+), 76 deletions(-) rename cpp/src/arrow/{acero => dataset}/forest_internal.h (98%) rename cpp/src/arrow/{acero => dataset}/subtree_internal.h (99%) rename cpp/src/arrow/{acero => dataset}/subtree_test.cc (99%) diff --git a/cpp/src/arrow/acero/CMakeLists.txt b/cpp/src/arrow/acero/CMakeLists.txt index c6fad5f1e42..f542a32dbb1 100644 --- a/cpp/src/arrow/acero/CMakeLists.txt +++ b/cpp/src/arrow/acero/CMakeLists.txt @@ -156,7 +156,6 @@ function(add_arrow_acero_test REL_TEST_NAME) ${ARG_UNPARSED_ARGUMENTS}) endfunction() -add_arrow_acero_test(subtree_test SOURCES subtree_test.cc) add_arrow_acero_test(plan_test SOURCES plan_test.cc diff --git a/cpp/src/arrow/acero/util.h b/cpp/src/arrow/acero/util.h index b83b8712618..0eb9f4c87e1 100644 --- a/cpp/src/arrow/acero/util.h +++ b/cpp/src/arrow/acero/util.h @@ -108,52 +108,6 @@ class ARROW_ACERO_EXPORT ThreadIndexer { std::unordered_map id_to_index_; }; -// Helper class to calculate the modified number of rows to process using SIMD. -// -// Some array elements at the end will be skipped in order to avoid buffer -// overrun, when doing memory loads and stores using larger word size than a -// single array element. -// -class TailSkipForSIMD { - public: - static int64_t FixBitAccess(int num_bytes_accessed_together, int64_t num_rows, - int bit_offset) { - int64_t num_bytes = bit_util::BytesForBits(num_rows + bit_offset); - int64_t num_bytes_safe = - std::max(static_cast(0LL), num_bytes - num_bytes_accessed_together + 1); - int64_t num_rows_safe = - std::max(static_cast(0LL), 8 * num_bytes_safe - bit_offset); - return std::min(num_rows_safe, num_rows); - } - static int64_t FixBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, - int64_t length) { - int64_t num_rows_to_skip = bit_util::CeilDiv(length, num_bytes_accessed_together); - int64_t num_rows_safe = - std::max(static_cast(0LL), num_rows - num_rows_to_skip); - return num_rows_safe; - } - static int64_t FixVarBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, - const uint32_t* offsets) { - // Do not process rows that could read past the end of the buffer using N - // byte loads/stores. - // - int64_t num_rows_safe = num_rows; - while (num_rows_safe > 0 && - offsets[num_rows_safe] + num_bytes_accessed_together > offsets[num_rows]) { - --num_rows_safe; - } - return num_rows_safe; - } - static int FixSelection(int64_t num_rows_safe, int num_selected, - const uint16_t* selection) { - int num_selected_safe = num_selected; - while (num_selected_safe > 0 && selection[num_selected_safe - 1] >= num_rows_safe) { - --num_selected_safe; - } - return num_selected_safe; - } -}; - /// \brief A consumer that collects results into an in-memory table struct ARROW_ACERO_EXPORT TableSinkNodeConsumer : public SinkNodeConsumer { public: diff --git a/cpp/src/arrow/compute/row/compare_internal_avx2.cc b/cpp/src/arrow/compute/row/compare_internal_avx2.cc index e387d4ccd9a..95f37ab617d 100644 --- a/cpp/src/arrow/compute/row/compare_internal_avx2.cc +++ b/cpp/src/arrow/compute/row/compare_internal_avx2.cc @@ -17,8 +17,8 @@ #include -#include "arrow/acero/util.h" #include "arrow/compute/row/compare_internal.h" +#include "arrow/compute/util.h" #include "arrow/util/bit_util.h" namespace arrow { @@ -572,10 +572,10 @@ uint32_t KeyCompare::NullUpdateColumnToRow_avx2( const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { - int64_t num_rows_safe = acero::TailSkipForSIMD::FixBitAccess( - sizeof(uint32_t), col.length(), col.bit_offset(0)); + int64_t num_rows_safe = + TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(0)); if (sel_left_maybe_null) { - num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast(num_rows_safe); @@ -601,19 +601,19 @@ uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( int64_t num_rows_safe = col.length(); if (col_width == 0) { // In this case we will access left column memory 4B at a time - num_rows_safe = acero::TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), - col.bit_offset(1)); + num_rows_safe = + TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(1)); } else if (col_width == 1 || col_width == 2) { // In this case we will access left column memory 4B at a time - num_rows_safe = acero::TailSkipForSIMD::FixBinaryAccess(sizeof(uint32_t), - col.length(), col_width); + num_rows_safe = + TailSkipForSIMD::FixBinaryAccess(sizeof(uint32_t), col.length(), col_width); } else if (col_width != 4 && col_width != 8) { // In this case we will access left column memory 32B at a time num_rows_safe = - acero::TailSkipForSIMD::FixBinaryAccess(sizeof(__m256i), col.length(), col_width); + TailSkipForSIMD::FixBinaryAccess(sizeof(__m256i), col.length(), col_width); } if (sel_left_maybe_null) { - num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast( @@ -636,10 +636,10 @@ uint32_t KeyCompare::CompareVarBinaryColumnToRow_avx2( uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { - int64_t num_rows_safe = acero::TailSkipForSIMD::FixVarBinaryAccess( - sizeof(__m256i), col.length(), col.offsets()); + int64_t num_rows_safe = + TailSkipForSIMD::FixVarBinaryAccess(sizeof(__m256i), col.length(), col.offsets()); if (use_selection) { - num_rows_to_compare = static_cast(acero::TailSkipForSIMD::FixSelection( + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); } else { num_rows_to_compare = static_cast(num_rows_safe); diff --git a/cpp/src/arrow/compute/util.h b/cpp/src/arrow/compute/util.h index d10320e61c2..6e1bb79674c 100644 --- a/cpp/src/arrow/compute/util.h +++ b/cpp/src/arrow/compute/util.h @@ -253,5 +253,51 @@ Result ModifyExpression(Expression expr, const PreVisit& pre, return post_call(std::move(expr), NULLPTR); } +// Helper class to calculate the modified number of rows to process using SIMD. +// +// Some array elements at the end will be skipped in order to avoid buffer +// overrun, when doing memory loads and stores using larger word size than a +// single array element. +// +class TailSkipForSIMD { + public: + static int64_t FixBitAccess(int num_bytes_accessed_together, int64_t num_rows, + int bit_offset) { + int64_t num_bytes = bit_util::BytesForBits(num_rows + bit_offset); + int64_t num_bytes_safe = + std::max(static_cast(0LL), num_bytes - num_bytes_accessed_together + 1); + int64_t num_rows_safe = + std::max(static_cast(0LL), 8 * num_bytes_safe - bit_offset); + return std::min(num_rows_safe, num_rows); + } + static int64_t FixBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, + int64_t length) { + int64_t num_rows_to_skip = bit_util::CeilDiv(length, num_bytes_accessed_together); + int64_t num_rows_safe = + std::max(static_cast(0LL), num_rows - num_rows_to_skip); + return num_rows_safe; + } + static int64_t FixVarBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, + const uint32_t* offsets) { + // Do not process rows that could read past the end of the buffer using N + // byte loads/stores. + // + int64_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && + offsets[num_rows_safe] + num_bytes_accessed_together > offsets[num_rows]) { + --num_rows_safe; + } + return num_rows_safe; + } + static int FixSelection(int64_t num_rows_safe, int num_selected, + const uint16_t* selection) { + int num_selected_safe = num_selected; + while (num_selected_safe > 0 && selection[num_selected_safe - 1] >= num_rows_safe) { + --num_selected_safe; + } + return num_selected_safe; + } +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/dataset/CMakeLists.txt b/cpp/src/arrow/dataset/CMakeLists.txt index fa6e1b1deeb..13083e4bdd1 100644 --- a/cpp/src/arrow/dataset/CMakeLists.txt +++ b/cpp/src/arrow/dataset/CMakeLists.txt @@ -153,6 +153,7 @@ add_arrow_dataset_test(file_ipc_test) add_arrow_dataset_test(file_test) add_arrow_dataset_test(partition_test) add_arrow_dataset_test(scanner_test) +add_arrow_dataset_test(subtree_test) if(ARROW_CSV) add_arrow_dataset_test(file_csv_test) diff --git a/cpp/src/arrow/dataset/file_base.cc b/cpp/src/arrow/dataset/file_base.cc index aada6ad8657..b300bd67cee 100644 --- a/cpp/src/arrow/dataset/file_base.cc +++ b/cpp/src/arrow/dataset/file_base.cc @@ -26,15 +26,15 @@ #include #include -#include "arrow/acero/forest_internal.h" #include "arrow/acero/map_node.h" #include "arrow/acero/query_context.h" -#include "arrow/acero/subtree_internal.h" #include "arrow/acero/util.h" #include "arrow/compute/api_scalar.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/dataset/dataset_writer.h" +#include "arrow/dataset/forest_internal.h" #include "arrow/dataset/scanner.h" +#include "arrow/dataset/subtree_internal.h" #include "arrow/filesystem/filesystem.h" #include "arrow/filesystem/path_util.h" #include "arrow/io/compressed.h" @@ -211,7 +211,7 @@ bool FileFragment::Equals(const FileFragment& other) const { struct FileSystemDataset::FragmentSubtrees { // Forest for skipping fragments based on extracted subtree expressions - acero::Forest forest; + Forest forest; // fragment indices and subtree expressions in forest order std::vector> fragments_and_subtrees; }; @@ -268,13 +268,13 @@ std::string FileSystemDataset::ToString() const { void FileSystemDataset::SetupSubtreePruning() { subtrees_ = std::make_shared(); - acero::SubtreeImpl impl; + SubtreeImpl impl; auto encoded = impl.EncodeGuarantees( [&](int index) { return fragments_[index]->partition_expression(); }, static_cast(fragments_.size())); - std::sort(encoded.begin(), encoded.end(), acero::SubtreeImpl::ByGuarantee()); + std::sort(encoded.begin(), encoded.end(), SubtreeImpl::ByGuarantee()); for (const auto& e : encoded) { if (e.index) { @@ -284,8 +284,8 @@ void FileSystemDataset::SetupSubtreePruning() { } } - subtrees_->forest = acero::Forest(static_cast(encoded.size()), - acero::SubtreeImpl::IsAncestor{encoded}); + subtrees_->forest = + Forest(static_cast(encoded.size()), SubtreeImpl::IsAncestor{encoded}); } Result FileSystemDataset::GetFragmentsImpl( @@ -299,7 +299,7 @@ Result FileSystemDataset::GetFragmentsImpl( std::vector predicates{predicate}; RETURN_NOT_OK(subtrees_->forest.Visit( - [&](acero::Forest::Ref ref) -> Result { + [&](Forest::Ref ref) -> Result { if (auto fragment_index = std::get_if(&subtrees_->fragments_and_subtrees[ref.i])) { fragment_indices.push_back(*fragment_index); @@ -318,7 +318,7 @@ Result FileSystemDataset::GetFragmentsImpl( predicates.push_back(std::move(simplified)); return true; }, - [&](acero::Forest::Ref ref) { predicates.pop_back(); })); + [&](Forest::Ref ref) { predicates.pop_back(); })); std::sort(fragment_indices.begin(), fragment_indices.end()); diff --git a/cpp/src/arrow/acero/forest_internal.h b/cpp/src/arrow/dataset/forest_internal.h similarity index 98% rename from cpp/src/arrow/acero/forest_internal.h rename to cpp/src/arrow/dataset/forest_internal.h index 0cedd6ab9ee..1c14a6bfb49 100644 --- a/cpp/src/arrow/acero/forest_internal.h +++ b/cpp/src/arrow/dataset/forest_internal.h @@ -25,7 +25,7 @@ #include "arrow/status.h" namespace arrow { -namespace acero { +namespace dataset { /// A Forest is a view of a sorted range which carries an ancestry relation in addition /// to an ordering relation: each element's descendants appear directly after it. @@ -121,5 +121,5 @@ class Forest { std::shared_ptr> descendant_counts_; }; -} // namespace acero +} // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/acero/subtree_internal.h b/cpp/src/arrow/dataset/subtree_internal.h similarity index 99% rename from cpp/src/arrow/acero/subtree_internal.h rename to cpp/src/arrow/dataset/subtree_internal.h index a6595e77937..f16df211b97 100644 --- a/cpp/src/arrow/acero/subtree_internal.h +++ b/cpp/src/arrow/dataset/subtree_internal.h @@ -30,7 +30,7 @@ namespace arrow { using compute::Expression; -namespace acero { +namespace dataset { // Helper class for efficiently detecting subtrees given expressions. // // Using fragment partition expressions as an example: @@ -177,5 +177,5 @@ inline bool operator==(const SubtreeImpl::Encoded& l, const SubtreeImpl::Encoded return l.index == r.index && l.guarantee == r.guarantee; } -} // namespace acero +} // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/acero/subtree_test.cc b/cpp/src/arrow/dataset/subtree_test.cc similarity index 99% rename from cpp/src/arrow/acero/subtree_test.cc rename to cpp/src/arrow/dataset/subtree_test.cc index 4d526ee0e74..3535d4771a6 100644 --- a/cpp/src/arrow/acero/subtree_test.cc +++ b/cpp/src/arrow/dataset/subtree_test.cc @@ -25,7 +25,7 @@ #include #include "arrow/acero/forest_internal.h" -#include "arrow/acero/subtree_internal.h" +#include "arrow/dataset/subtree_internal.h" #include "arrow/testing/gtest_util.h" #include "arrow/util/string.h" @@ -36,7 +36,7 @@ using internal::StartsWith; using compute::field_ref; using compute::literal; -namespace acero { +namespace dataset { using testing::ContainerEq; @@ -380,5 +380,5 @@ TEST(Subtree, EncodeFragments) { SubtreeImpl::Encoded{std::nullopt, SubtreeImpl::expression_codes({2, 3})}, })); } -} // namespace acero +} // namespace dataset } // namespace arrow From 769da298bb3d94cca49f2ad3373f230c9ea07243 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 31 Mar 2023 15:02:28 -0400 Subject: [PATCH 8/8] Remove acerointernal namespace; uncomment Acero test/benchmark; lint --- cpp/src/arrow/acero/CMakeLists.txt | 41 +++++++++++----------- cpp/src/arrow/acero/aggregate_node.cc | 4 +-- cpp/src/arrow/acero/asof_join_node.cc | 4 +-- cpp/src/arrow/acero/exec_plan.cc | 26 +++++++------- cpp/src/arrow/acero/fetch_node.cc | 4 +-- cpp/src/arrow/acero/filter_node.cc | 4 +-- cpp/src/arrow/acero/hash_aggregate_test.cc | 10 +++--- cpp/src/arrow/acero/hash_join_node.cc | 4 +-- cpp/src/arrow/acero/order_by_node.cc | 4 +-- cpp/src/arrow/acero/pivot_longer_node.cc | 4 +-- cpp/src/arrow/acero/project_node.cc | 4 +-- cpp/src/arrow/acero/sink_node.cc | 4 +-- cpp/src/arrow/acero/source_node.cc | 4 +-- cpp/src/arrow/acero/tpch_benchmark.cc | 4 +-- cpp/src/arrow/acero/tpch_node.cc | 4 +-- cpp/src/arrow/acero/tpch_node.h | 4 +-- cpp/src/arrow/acero/tpch_node_test.cc | 4 +-- cpp/src/arrow/acero/union_node.cc | 4 +-- r/configure | 2 +- 19 files changed, 69 insertions(+), 70 deletions(-) diff --git a/cpp/src/arrow/acero/CMakeLists.txt b/cpp/src/arrow/acero/CMakeLists.txt index f542a32dbb1..d7a98786952 100644 --- a/cpp/src/arrow/acero/CMakeLists.txt +++ b/cpp/src/arrow/acero/CMakeLists.txt @@ -27,7 +27,6 @@ macro(append_acero_avx2_src SRC) endif() endmacro() - set(ARROW_ACERO_SRCS groupby.cc accumulation_queue.cc @@ -172,7 +171,7 @@ add_arrow_acero_test(tpch_node_test SOURCES tpch_node_test.cc) add_arrow_acero_test(union_node_test SOURCES union_node_test.cc) add_arrow_acero_test(groupby_test SOURCES groupby_test.cc) add_arrow_acero_test(util_test SOURCES util_test.cc task_util_test.cc) -#add_arrow_acero_test(hash_aggregate_test SOURCES hash_aggregate_test.cc) +add_arrow_acero_test(hash_aggregate_test SOURCES hash_aggregate_test.cc) if(ARROW_BUILD_BENCHMARKS) function(add_arrow_acero_benchmark REL_BENCHMARK_NAME) @@ -236,23 +235,23 @@ if(ARROW_BUILD_BENCHMARKS) endif() endif() - # if(ARROW_BUILD_STATIC) - # target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_static) - # target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_static) - # target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_static) - # target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_static) - # target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_static) - # if(ARROW_BUILD_OPENMP_BENCHMARKS) - # target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_static) - # endif() - # else() - # target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_shared) - # target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_shared) - # target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_shared) - # target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_shared) - # target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_shared) - # if(ARROW_BUILD_OPENMP_BENCHMARKS) - # target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_shared) - # endif() - # endif() + if(ARROW_BUILD_STATIC) + target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_static) + target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_static) + target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_static) + target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_static) + target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_static) + if(ARROW_BUILD_OPENMP_BENCHMARKS) + target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_static) + endif() + else() + target_link_libraries(arrow-acero-expression-benchmark PUBLIC arrow_acero_shared) + target_link_libraries(arrow-acero-filter-benchmark PUBLIC arrow_acero_shared) + target_link_libraries(arrow-acero-project-benchmark PUBLIC arrow_acero_shared) + target_link_libraries(arrow-acero-asof-join-benchmark PUBLIC arrow_acero_shared) + target_link_libraries(arrow-acero-tpch-benchmark PUBLIC arrow_acero_shared) + if(ARROW_BUILD_OPENMP_BENCHMARKS) + target_link_libraries(arrow-acero-hash-join-benchmark PUBLIC arrow_acero_shared) + endif() + endif() endif() diff --git a/cpp/src/arrow/acero/aggregate_node.cc b/cpp/src/arrow/acero/aggregate_node.cc index 81a6239a512..6669d30bcc0 100644 --- a/cpp/src/arrow/acero/aggregate_node.cc +++ b/cpp/src/arrow/acero/aggregate_node.cc @@ -968,7 +968,7 @@ class GroupByNode : public ExecNode, public TracedNode { } // namespace -namespace acerointernal { +namespace internal { void RegisterAggregateNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory( @@ -986,6 +986,6 @@ void RegisterAggregateNode(ExecFactoryRegistry* registry) { })); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/asof_join_node.cc b/cpp/src/arrow/acero/asof_join_node.cc index f4d0ee9443f..4d0f69fca59 100644 --- a/cpp/src/arrow/acero/asof_join_node.cc +++ b/cpp/src/arrow/acero/asof_join_node.cc @@ -1594,11 +1594,11 @@ AsofJoinNode::AsofJoinNode(ExecPlan* plan, NodeVector inputs, process_(), process_thread_() {} -namespace acerointernal { +namespace internal { void RegisterAsofJoinNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("asofjoin", AsofJoinNode::Make)); } -} // namespace acerointernal +} // namespace internal namespace asofjoin { diff --git a/cpp/src/arrow/acero/exec_plan.cc b/cpp/src/arrow/acero/exec_plan.cc index 886a659d5c0..432a859576d 100644 --- a/cpp/src/arrow/acero/exec_plan.cc +++ b/cpp/src/arrow/acero/exec_plan.cc @@ -1048,7 +1048,7 @@ Result> DeclarationToReader( return DeclarationToReader(std::move(declaration), std::move(options)); } -namespace acerointernal { +namespace internal { void RegisterSourceNode(ExecFactoryRegistry*); void RegisterFetchNode(ExecFactoryRegistry*); @@ -1062,23 +1062,23 @@ void RegisterSinkNode(ExecFactoryRegistry*); void RegisterHashJoinNode(ExecFactoryRegistry*); void RegisterAsofJoinNode(ExecFactoryRegistry*); -} // namespace acerointernal +} // namespace internal ExecFactoryRegistry* default_exec_factory_registry() { class DefaultRegistry : public ExecFactoryRegistry { public: DefaultRegistry() { - acerointernal::RegisterSourceNode(this); - acerointernal::RegisterFetchNode(this); - acerointernal::RegisterFilterNode(this); - acerointernal::RegisterOrderByNode(this); - acerointernal::RegisterPivotLongerNode(this); - acerointernal::RegisterProjectNode(this); - acerointernal::RegisterUnionNode(this); - acerointernal::RegisterAggregateNode(this); - acerointernal::RegisterSinkNode(this); - acerointernal::RegisterHashJoinNode(this); - acerointernal::RegisterAsofJoinNode(this); + internal::RegisterSourceNode(this); + internal::RegisterFetchNode(this); + internal::RegisterFilterNode(this); + internal::RegisterOrderByNode(this); + internal::RegisterPivotLongerNode(this); + internal::RegisterProjectNode(this); + internal::RegisterUnionNode(this); + internal::RegisterAggregateNode(this); + internal::RegisterSinkNode(this); + internal::RegisterHashJoinNode(this); + internal::RegisterAsofJoinNode(this); } Result GetFactory(const std::string& factory_name) override { diff --git a/cpp/src/arrow/acero/fetch_node.cc b/cpp/src/arrow/acero/fetch_node.cc index 92a9efde96b..f64d1e35f9e 100644 --- a/cpp/src/arrow/acero/fetch_node.cc +++ b/cpp/src/arrow/acero/fetch_node.cc @@ -209,12 +209,12 @@ class FetchNode : public ExecNode, public TracedNode, util::SequencingQueue::Pro } // namespace -namespace acerointernal { +namespace internal { void RegisterFetchNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory(std::string(FetchNodeOptions::kName), FetchNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/filter_node.cc b/cpp/src/arrow/acero/filter_node.cc index 28b23e4dd6f..e5435487a00 100644 --- a/cpp/src/arrow/acero/filter_node.cc +++ b/cpp/src/arrow/acero/filter_node.cc @@ -114,11 +114,11 @@ class FilterNode : public MapNode { }; } // namespace -namespace acerointernal { +namespace internal { void RegisterFilterNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("filter", FilterNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/hash_aggregate_test.cc b/cpp/src/arrow/acero/hash_aggregate_test.cc index 226b8a633a3..68059dacbfa 100644 --- a/cpp/src/arrow/acero/hash_aggregate_test.cc +++ b/cpp/src/arrow/acero/hash_aggregate_test.cc @@ -1186,7 +1186,7 @@ class GroupBy : public ::testing::TestWithParam { void ValidateGroupBy(const std::vector& aggregates, std::vector arguments, std::vector keys, bool naive = true) { - ValidateGroupBy(GetParam(), aggregates, arguments, keys, naive); + acero::ValidateGroupBy(GetParam(), aggregates, arguments, keys, naive); } Result GroupByTest(const std::vector& arguments, @@ -1194,15 +1194,15 @@ class GroupBy : public ::testing::TestWithParam { const std::vector& segment_keys, const std::vector& aggregates, bool use_threads) { - return GroupByTest(GetParam(), arguments, keys, segment_keys, aggregates, - use_threads); + return acero::GroupByTest(GetParam(), arguments, keys, segment_keys, aggregates, + use_threads); } Result GroupByTest(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, bool use_threads) { - return GroupByTest(GetParam(), arguments, keys, aggregates, use_threads); + return acero::GroupByTest(GetParam(), arguments, keys, aggregates, use_threads); } Result AltGroupBy(const std::vector& arguments, @@ -1216,7 +1216,7 @@ class GroupBy : public ::testing::TestWithParam { void TestSegmentKey(const std::shared_ptr
& table, Datum output, const std::vector& segment_keys) { - return TestSegmentKey(GetParam(), table, output, segment_keys); + return acero::TestSegmentKey(GetParam(), table, output, segment_keys); } }; diff --git a/cpp/src/arrow/acero/hash_join_node.cc b/cpp/src/arrow/acero/hash_join_node.cc index 5fc950c0144..a6179141649 100644 --- a/cpp/src/arrow/acero/hash_join_node.cc +++ b/cpp/src/arrow/acero/hash_join_node.cc @@ -1250,11 +1250,11 @@ std::pair> BloomFilterPushdownContext::GetPushdo #endif // ARROW_LITTLE_ENDIAN } -namespace acerointernal { +namespace internal { void RegisterHashJoinNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("hashjoin", HashJoinNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/order_by_node.cc b/cpp/src/arrow/acero/order_by_node.cc index bfcfccee585..1811fa9f4c7 100644 --- a/cpp/src/arrow/acero/order_by_node.cc +++ b/cpp/src/arrow/acero/order_by_node.cc @@ -156,13 +156,13 @@ class OrderByNode : public ExecNode, public TracedNode { } // namespace -namespace acerointernal { +namespace internal { void RegisterOrderByNode(ExecFactoryRegistry* registry) { DCHECK_OK( registry->AddFactory(std::string(OrderByNodeOptions::kName), OrderByNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/pivot_longer_node.cc b/cpp/src/arrow/acero/pivot_longer_node.cc index 32f08922884..e54f00a20be 100644 --- a/cpp/src/arrow/acero/pivot_longer_node.cc +++ b/cpp/src/arrow/acero/pivot_longer_node.cc @@ -270,13 +270,13 @@ class PivotLongerNode : public ExecNode, public TracedNode { } // namespace -namespace acerointernal { +namespace internal { void RegisterPivotLongerNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory(std::string(PivotLongerNodeOptions::kName), PivotLongerNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/project_node.cc b/cpp/src/arrow/acero/project_node.cc index 562542fb1f5..bcabe585b29 100644 --- a/cpp/src/arrow/acero/project_node.cc +++ b/cpp/src/arrow/acero/project_node.cc @@ -117,12 +117,12 @@ class ProjectNode : public MapNode { } // namespace -namespace acerointernal { +namespace internal { void RegisterProjectNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("project", ProjectNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/sink_node.cc b/cpp/src/arrow/acero/sink_node.cc index 4bd55b0b3ea..4ab6b4537de 100644 --- a/cpp/src/arrow/acero/sink_node.cc +++ b/cpp/src/arrow/acero/sink_node.cc @@ -554,7 +554,7 @@ struct OrderBySinkNode final : public SinkNode { } // namespace -namespace acerointernal { +namespace internal { void RegisterSinkNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("select_k_sink", OrderBySinkNode::MakeSelectK)); @@ -564,6 +564,6 @@ void RegisterSinkNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("table_sink", MakeTableConsumingSinkNode)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/source_node.cc b/cpp/src/arrow/acero/source_node.cc index 22619d5e7d5..6c138d8dccd 100644 --- a/cpp/src/arrow/acero/source_node.cc +++ b/cpp/src/arrow/acero/source_node.cc @@ -515,7 +515,7 @@ Result MakeNamedTableNode(ExecPlan* plan, std::vector inpu } // namespace -namespace acerointernal { +namespace internal { void RegisterSourceNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("source", SourceNode::Make)); @@ -528,6 +528,6 @@ void RegisterSourceNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("named_table", MakeNamedTableNode)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/tpch_benchmark.cc b/cpp/src/arrow/acero/tpch_benchmark.cc index feb16ea4bda..aa621758b35 100644 --- a/cpp/src/arrow/acero/tpch_benchmark.cc +++ b/cpp/src/arrow/acero/tpch_benchmark.cc @@ -34,7 +34,7 @@ using compute::ScalarAggregateOptions; using compute::SortKey; namespace acero { -namespace acerointernal { +namespace internal { std::shared_ptr Plan_Q1(AsyncGenerator>* sink_gen, int scale_factor) { @@ -124,6 +124,6 @@ static void BM_Tpch_Q1(benchmark::State& st) { } BENCHMARK(BM_Tpch_Q1)->Args({1})->ArgNames({"ScaleFactor"}); -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/tpch_node.cc b/cpp/src/arrow/acero/tpch_node.cc index 36f894c19bc..9797a082b49 100644 --- a/cpp/src/arrow/acero/tpch_node.cc +++ b/cpp/src/arrow/acero/tpch_node.cc @@ -47,7 +47,7 @@ using arrow::internal::checked_cast; using arrow::internal::GetRandomSeed; namespace acero { -namespace acerointernal { +namespace internal { /* Architecture of the generator: This is a multithreaded implementation of TPC-H's DBGen data generator. For each table @@ -3538,6 +3538,6 @@ Result> TpchGen::Make(ExecPlan* plan, double scale_fact return std::make_unique(plan, scale_factor, batch_size, *seed); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/tpch_node.h b/cpp/src/arrow/acero/tpch_node.h index 280e05fe5ed..e6476b57ad6 100644 --- a/cpp/src/arrow/acero/tpch_node.h +++ b/cpp/src/arrow/acero/tpch_node.h @@ -29,7 +29,7 @@ namespace arrow { namespace acero { -namespace acerointernal { +namespace internal { class ARROW_ACERO_EXPORT TpchGen { public: @@ -60,6 +60,6 @@ class ARROW_ACERO_EXPORT TpchGen { virtual Result Region(std::vector columns = {}) = 0; }; -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/tpch_node_test.cc b/cpp/src/arrow/acero/tpch_node_test.cc index 015066f9740..076bcf634a6 100644 --- a/cpp/src/arrow/acero/tpch_node_test.cc +++ b/cpp/src/arrow/acero/tpch_node_test.cc @@ -42,7 +42,7 @@ namespace arrow { using arrow::internal::StartsWith; namespace acero { -namespace acerointernal { +namespace internal { static constexpr uint32_t kStartDate = 8035; // January 1, 1992 is 8035 days after January 1, 1970 @@ -642,6 +642,6 @@ TEST(TpchNode, AllTables) { } } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/cpp/src/arrow/acero/union_node.cc b/cpp/src/arrow/acero/union_node.cc index f65027e02e6..054fcdaba24 100644 --- a/cpp/src/arrow/acero/union_node.cc +++ b/cpp/src/arrow/acero/union_node.cc @@ -119,12 +119,12 @@ class UnionNode : public ExecNode, public TracedNode { std::atomic total_batches_{0}; }; -namespace acerointernal { +namespace internal { void RegisterUnionNode(ExecFactoryRegistry* registry) { DCHECK_OK(registry->AddFactory("union", UnionNode::Make)); } -} // namespace acerointernal +} // namespace internal } // namespace acero } // namespace arrow diff --git a/r/configure b/r/configure index 2e9f426f442..3099e84b537 100755 --- a/r/configure +++ b/r/configure @@ -268,7 +268,7 @@ if [ $? -eq 0 ]; then if arrow_built_with ARROW_ACERO; then PKG_CFLAGS="$PKG_CFLAGS -DARROW_R_WITH_ACERO" PKG_LIBS="-larrow_acero $PKG_LIBS" - # NOTE: arrow-dataset is assumed to have the same -L flag as arrow + # NOTE: arrow-acero is assumed to have the same -L flag as arrow # so there is no need to add its location to PKG_DIRS fi if arrow_built_with ARROW_DATASET; then