From b2b10cfa428dd864aa3629a46c0b07c690f166e1 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Thu, 23 Feb 2023 05:17:23 -0500 Subject: [PATCH 01/27] GH-32884: [C++] Add ordered aggregation --- cpp/src/arrow/compute/exec.cc | 12 + cpp/src/arrow/compute/exec.h | 8 + cpp/src/arrow/compute/exec/aggregate.cc | 542 ++++++++--- cpp/src/arrow/compute/exec/aggregate.h | 22 +- cpp/src/arrow/compute/exec/aggregate_node.cc | 296 +++++- cpp/src/arrow/compute/exec/exec_plan.h | 3 +- cpp/src/arrow/compute/exec/options.h | 17 +- .../compute/kernels/hash_aggregate_test.cc | 918 +++++++++++++++--- cpp/src/arrow/compute/row/grouper.cc | 540 ++++++++++- cpp/src/arrow/compute/row/grouper.h | 56 +- cpp/src/arrow/scalar.h | 13 + 11 files changed, 2112 insertions(+), 315 deletions(-) diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index 15f8b263ed8..c18dfa09522 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -147,6 +147,18 @@ ExecBatch ExecBatch::Slice(int64_t offset, int64_t length) const { return out; } +Result ExecBatch::SelectValues(const std::vector& ids) const { + std::vector selected_values; + selected_values.reserve(ids.size()); + for (int id : ids) { + if (id < 0 || static_cast(id) >= values.size()) { + return Status::Invalid("ExecBatch invalid value selection: ", id); + } + selected_values.push_back(values[id]); + } + return ExecBatch(std::move(selected_values), length); +} + namespace { enum LengthInferenceError { diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index 8128d84a12b..338740f066e 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -181,6 +181,12 @@ struct ARROW_EXPORT ExecBatch { /// \brief Infer the ExecBatch length from values. static Result InferLength(const std::vector& values); + /// Creates an ExecBatch with length-validation. + /// + /// If any value is given, then all values must have a common length. If the given + /// length is negative, then the length of the ExecBatch is set to this common length, + /// or to 1 if no values are given. Otherwise, the given length must equal the common + /// length, if any value is given. static Result Make(std::vector values, int64_t length = -1); Result> ToRecordBatch( @@ -240,6 +246,8 @@ struct ARROW_EXPORT ExecBatch { ExecBatch Slice(int64_t offset, int64_t length) const; + Result SelectValues(const std::vector& ids) const; + /// \brief A convenience for returning the types from the batch. std::vector GetTypes() const { std::vector result; diff --git a/cpp/src/arrow/compute/exec/aggregate.cc b/cpp/src/arrow/compute/exec/aggregate.cc index 5e99bbba92b..278e33dd3d2 100644 --- a/cpp/src/arrow/compute/exec/aggregate.cc +++ b/cpp/src/arrow/compute/exec/aggregate.cc @@ -18,14 +18,19 @@ #include "arrow/compute/exec/aggregate.h" #include +#include +#include #include #include +#include "arrow/array/concatenate.h" +#include "arrow/compute/exec/util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/registry.h" #include "arrow/compute/row/grouper.h" #include "arrow/util/checked_cast.h" #include "arrow/util/logging.h" +#include "arrow/util/macros.h" #include "arrow/util/string.h" #include "arrow/util/task_group.h" @@ -36,6 +41,8 @@ using internal::ToChars; namespace compute { namespace internal { +using arrow::compute::detail::ExecSpanIterator; + namespace { std::vector ExtendWithGroupIdType(const std::vector& in_types) { @@ -126,173 +133,456 @@ Result ResolveKernels( return fields; } -Result GroupBy(const std::vector& arguments, const std::vector& keys, - const std::vector& aggregates, bool use_threads, - ExecContext* ctx) { - auto task_group = - use_threads - ? arrow::internal::TaskGroup::MakeThreaded(arrow::internal::GetCpuThreadPool()) - : arrow::internal::TaskGroup::MakeSerial(); - - std::vector kernels; - std::vector>> states; - FieldVector out_fields; - - using arrow::compute::detail::ExecSpanIterator; - ExecSpanIterator argument_iterator; - - ExecBatch args_batch; - Result inferred_length = ExecBatch::InferLength(arguments); - if (!inferred_length.ok()) { - inferred_length = ExecBatch::InferLength(keys); +namespace { + +template +std::ostream& operator<<(std::ostream& s, const std::vector& v) { + s << '['; + for (size_t i = 0; i < v.size(); i++) { + if (i != 0) s << ','; + s << v[i]; } - ARROW_ASSIGN_OR_RAISE(const int64_t length, std::move(inferred_length)); - if (!aggregates.empty()) { - ARROW_ASSIGN_OR_RAISE(args_batch, ExecBatch::Make(arguments, length)); + s << ']'; + return s; +} - // Construct and initialize HashAggregateKernels +template +std::string ToString(const std::vector& v) { + std::stringstream s; + s << v; + return s.str(); +} + +Result InferLength(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys) { + std::vector values; + values.reserve(arguments.size() + keys.size() + segment_keys.size()); + for (const auto& datums : {arguments, keys, segment_keys}) { + values.insert(values.end(), datums.begin(), datums.end()); + } + return ExecBatch::InferLength(values); +} + +class GroupByProcess { + public: + struct BatchInfo { + ExecBatch args_batch; std::vector> aggs_argument_types; - aggs_argument_types.reserve(aggregates.size()); - size_t i = 0; - for (const auto& aggregate : aggregates) { - auto& agg_types = aggs_argument_types.emplace_back(); - const size_t num_needed = aggregate.target.size(); - for (size_t j = 0; j < num_needed && i < arguments.size(); j++, i++) { - agg_types.emplace_back(arguments[i].type()); + ExecBatch keys_batch; + std::vector key_types; + ExecBatch segment_keys_batch; + std::vector segment_key_types; + + static Result Make(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates) { + ARROW_ASSIGN_OR_RAISE(int64_t batch_length, + InferLength(arguments, keys, segment_keys)); + + ARROW_ASSIGN_OR_RAISE(auto args_batch, ExecBatch::Make(arguments, batch_length)); + std::vector> aggs_argument_types; + aggs_argument_types.reserve(aggregates.size()); + size_t i = 0; + for (const auto& aggregate : aggregates) { + auto& agg_types = aggs_argument_types.emplace_back(); + const size_t num_needed = aggregate.target.size(); + for (size_t j = 0; j < num_needed && i < arguments.size(); j++, i++) { + agg_types.emplace_back(arguments[i].type()); + } + if (agg_types.size() != num_needed) { + return Status::Invalid( + "Not enough arguments specified to aggregate functions."); + } } - if (agg_types.size() != num_needed) { - return Status::Invalid("Not enough arguments specified to aggregate functions."); + DCHECK_EQ(aggs_argument_types.size(), aggregates.size()); + if (i != arguments.size()) { + return Status::Invalid("Aggregate functions expect exactly ", i, + " arguments, but ", arguments.size(), " were specified."); } + + ARROW_ASSIGN_OR_RAISE(auto keys_batch, ExecBatch::Make(keys, batch_length)); + auto key_types = keys_batch.GetTypes(); + + ARROW_ASSIGN_OR_RAISE(auto segment_keys_batch, + ExecBatch::Make(segment_keys, batch_length)); + auto segment_key_types = segment_keys_batch.GetTypes(); + + return BatchInfo{std::move(args_batch), std::move(aggs_argument_types), + std::move(keys_batch), std::move(key_types), + std::move(segment_keys_batch), std::move(segment_key_types)}; } - DCHECK_EQ(aggs_argument_types.size(), aggregates.size()); - if (i != arguments.size()) { - return Status::Invalid("Aggregate functions expect exactly ", i, " arguments, but ", - arguments.size(), " were specified."); + + BatchInfo Slice(int64_t offset, int64_t length) const { + return BatchInfo{args_batch.Slice(offset, length), aggs_argument_types, + keys_batch.Slice(offset, length), key_types, + segment_keys_batch.Slice(offset, length), segment_key_types}; } + }; + + struct StateInfo { + GroupByProcess& process; + std::vector> groupers; + std::vector kernels; + std::vector>> states; + FieldVector out_fields; + ExecSpanIterator batch_iterator; + ScalarVector segment_keys; - ARROW_ASSIGN_OR_RAISE(kernels, GetKernels(ctx, aggregates, aggs_argument_types)); + explicit StateInfo(GroupByProcess& process) : process(process) {} - states.resize(task_group->parallelism()); - for (auto& state : states) { - ARROW_ASSIGN_OR_RAISE(state, - InitKernels(kernels, ctx, aggregates, aggs_argument_types)); + int parallelism() { + return process.use_threads ? arrow::internal::GetCpuThreadPool()->GetCapacity() : 1; } - ARROW_ASSIGN_OR_RAISE(out_fields, ResolveKernels(aggregates, kernels, states[0], ctx, - aggs_argument_types)); + Status Init() { + const std::vector>& aggs_argument_types = + process.aggs_argument_types; + const std::vector& key_types = process.key_types; + const std::vector& aggregates = process.aggregates; + ExecContext* ctx = process.ctx; + const FieldVector& key_fields = process.key_fields; + + groupers.clear(); + groupers.resize(parallelism()); + for (auto& grouper : groupers) { + ARROW_ASSIGN_OR_RAISE(grouper, Grouper::Make(key_types, ctx)); + } - RETURN_NOT_OK(argument_iterator.Init(args_batch, ctx->exec_chunksize())); - } + if (!aggs_argument_types.empty()) { + // Construct and initialize HashAggregateKernels + ARROW_ASSIGN_OR_RAISE(kernels, GetKernels(ctx, aggregates, aggs_argument_types)); - // Construct Groupers - ARROW_ASSIGN_OR_RAISE(ExecBatch keys_batch, ExecBatch::Make(keys, length)); - auto key_types = keys_batch.GetTypes(); + states.resize(parallelism()); + for (auto& state : states) { + ARROW_ASSIGN_OR_RAISE( + state, InitKernels(kernels, ctx, aggregates, aggs_argument_types)); + } - std::vector> groupers(task_group->parallelism()); - for (auto& grouper : groupers) { - ARROW_ASSIGN_OR_RAISE(grouper, Grouper::Make(key_types, ctx)); - } + ARROW_ASSIGN_OR_RAISE(out_fields, ResolveKernels(aggregates, kernels, states[0], + ctx, aggs_argument_types)); + } else { + out_fields = {}; + } + out_fields.insert(out_fields.end(), key_fields.begin(), key_fields.end()); - std::mutex mutex; - std::unordered_map thread_ids; + return Status::OK(); + } - int i = 0; - for (const TypeHolder& key_type : key_types) { - out_fields.push_back(field("key_" + ToChars(i++), key_type.GetSharedPtr())); - } + Status Consume(const BatchInfo& batch_info) { + ExecContext* ctx = process.ctx; - ExecSpanIterator key_iterator; - RETURN_NOT_OK(key_iterator.Init(keys_batch, ctx->exec_chunksize())); + const ExecBatch& args_batch = batch_info.args_batch; + const ExecBatch& keys_batch = batch_info.keys_batch; + const ExecBatch& segment_keys_batch = batch_info.segment_keys_batch; - // start "streaming" execution - ExecSpan key_batch, argument_batch; - while ((arguments.empty() || argument_iterator.Next(&argument_batch)) && - key_iterator.Next(&key_batch)) { - if (arguments.empty()) { - // A value-less argument_batch should still have a valid length - argument_batch.length = key_batch.length; + if (segment_keys_batch.length == 0) { + return Status::OK(); + } + segment_keys = {}; + for (auto value : segment_keys_batch.values) { + if (value.is_scalar()) { + segment_keys.push_back(value.scalar()); + } else if (value.is_array()) { + ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(0)); + segment_keys.push_back(scalar); + } else if (value.is_chunked_array()) { + ARROW_ASSIGN_OR_RAISE(auto scalar, value.chunked_array()->GetScalar(0)); + segment_keys.push_back(scalar); + } else { + return Status::Invalid("consuming an invalid segment key type ", *value.type()); + } + } + + std::vector keys_args_values; + keys_args_values.reserve(keys_batch.values.size() + args_batch.values.size()); + keys_args_values.insert(keys_args_values.end(), keys_batch.values.begin(), + keys_batch.values.end()); + keys_args_values.insert(keys_args_values.end(), args_batch.values.begin(), + args_batch.values.end()); + ExecBatch keys_args_batch{std::move(keys_args_values), args_batch.length}; + ARROW_RETURN_NOT_OK(batch_iterator.Init(keys_args_batch, ctx->exec_chunksize())); + + ThreadIndexer thread_indexer; + + auto task_group = process.use_threads ? arrow::internal::TaskGroup::MakeThreaded( + arrow::internal::GetCpuThreadPool()) + : arrow::internal::TaskGroup::MakeSerial(); + + // start "streaming" execution + ExecSpan key_arg_batch, key_batch, argument_batch; + while (batch_iterator.Next(&key_arg_batch)) { + if (key_arg_batch.length == 0) continue; + key_batch.length = argument_batch.length = key_arg_batch.length; + auto key_arg_split = key_arg_batch.values.begin() + keys_batch.values.size(); + key_batch.values.clear(); + key_batch.values.insert(key_batch.values.end(), key_arg_batch.values.begin(), + key_arg_split); + argument_batch.values.clear(); + argument_batch.values.insert(argument_batch.values.end(), key_arg_split, + key_arg_batch.values.end()); + + task_group->Append([&, key_batch, argument_batch] { + size_t thread_index = thread_indexer(); + auto grouper = groupers[thread_index].get(); + + // compute a batch of group ids + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); + + // consume group ids with HashAggregateKernels + for (size_t k = 0, arg_idx = 0; k < kernels.size(); ++k) { + const auto* kernel = kernels[k]; + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[thread_index][k].get()); + + const size_t kernel_num_args = kernel->signature->in_types().size(); + DCHECK_GT(kernel_num_args, 0); + + std::vector kernel_args; + for (size_t i = 0; i + 1 < kernel_num_args; i++, arg_idx++) { + kernel_args.push_back(argument_batch[arg_idx]); + } + kernel_args.emplace_back(*id_batch.array()); + + ExecSpan kernel_batch(std::move(kernel_args), argument_batch.length); + RETURN_NOT_OK(kernel->resize(&batch_ctx, grouper->num_groups())); + RETURN_NOT_OK(kernel->consume(&batch_ctx, kernel_batch)); + } + + return Status::OK(); + }); + } + + ARROW_RETURN_NOT_OK(task_group->Finish()); + return Status::OK(); } - if (key_batch.length == 0) continue; - - task_group->Append([&, key_batch, argument_batch] { - size_t thread_index; - { - std::unique_lock lock(mutex); - auto it = thread_ids.emplace(std::this_thread::get_id(), thread_ids.size()).first; - thread_index = it->second; - DCHECK_LT(static_cast(thread_index), task_group->parallelism()); + + Status Merge() { + ExecContext* ctx = process.ctx; + size_t num_threads = static_cast(parallelism()); + for (size_t thread_index = 1; thread_index < num_threads; ++thread_index) { + ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, groupers[thread_index]->GetUniques()); + ARROW_ASSIGN_OR_RAISE(Datum transposition, + groupers[0]->Consume(ExecSpan(other_keys))); + groupers[thread_index].reset(); + + for (size_t idx = 0; idx < kernels.size(); ++idx) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[0][idx].get()); + + ARROW_RETURN_NOT_OK( + kernels[idx]->resize(&batch_ctx, groupers[0]->num_groups())); + ARROW_RETURN_NOT_OK(kernels[idx]->merge( + &batch_ctx, std::move(*states[thread_index][idx]), *transposition.array())); + states[thread_index][idx].reset(); + } } + return Status::OK(); + } - auto grouper = groupers[thread_index].get(); + Result Finalize() { + const std::vector& key_types = process.key_types; + const std::vector& segment_key_types = process.segment_key_types; + ExecContext* ctx = process.ctx; - // compute a batch of group ids - ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); + ArrayDataVector out_data(kernels.size() + key_types.size() + + segment_key_types.size()); + auto it = out_data.begin(); - // consume group ids with HashAggregateKernels - for (size_t k = 0, arg_idx = 0; k < kernels.size(); ++k) { - const auto* kernel = kernels[k]; + for (size_t idx = 0; idx < kernels.size(); ++idx) { KernelContext batch_ctx{ctx}; - batch_ctx.SetState(states[thread_index][k].get()); + batch_ctx.SetState(states[0][idx].get()); + Datum out; + ARROW_RETURN_NOT_OK(kernels[idx]->finalize(&batch_ctx, &out)); + *it++ = out.array(); + } - const size_t kernel_num_args = kernel->signature->in_types().size(); - DCHECK_GT(kernel_num_args, 0); + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, groupers[0]->GetUniques()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } - std::vector kernel_args; - for (size_t i = 0; i + 1 < kernel_num_args; i++, arg_idx++) { - kernel_args.push_back(argument_batch[arg_idx]); - } - kernel_args.emplace_back(*id_batch.array()); + const int64_t out_length = out_data[0]->length; + for (const auto& key : segment_keys) { + ARROW_ASSIGN_OR_RAISE(auto array, MakeArrayFromScalar(*key, out_length)); + *it++ = array->data(); + } - ExecSpan kernel_batch(std::move(kernel_args), argument_batch.length); - RETURN_NOT_OK(kernel->resize(&batch_ctx, grouper->num_groups())); - RETURN_NOT_OK(kernel->consume(&batch_ctx, kernel_batch)); + return ArrayData::Make(struct_(std::move(out_fields)), out_length, + {/*null_bitmap=*/nullptr}, std::move(out_data), + /*null_count=*/0); + } + }; + + ARROW_DISALLOW_COPY_AND_ASSIGN(GroupByProcess); + + GroupByProcess(std::vector> aggs_argument_types, + std::vector key_types, + std::vector segment_key_types, + const std::vector& aggregates, + std::unique_ptr segmenter, bool use_threads, + ExecContext* ctx) + : aggs_argument_types(aggs_argument_types), + key_types(key_types), + segment_key_types(segment_key_types), + aggregates(aggregates), + segmenter(std::move(segmenter)), + use_threads(use_threads), + ctx(ctx), + key_fields(), + state_info(*this) { + int i = 0; + for (auto types : {key_types, segment_key_types}) { + for (const TypeHolder& type : types) { + key_fields.push_back(field("key_" + ToChars(i++), type.GetSharedPtr())); } + } + } - return Status::OK(); - }); + static Result> Make( + const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, const std::vector& aggregates, + bool use_threads, ExecContext* ctx) { + ARROW_RETURN_NOT_OK(CheckSizes(arguments, aggregates)); + ARROW_ASSIGN_OR_RAISE(auto batch_info, + BatchInfo::Make(arguments, keys, segment_keys, aggregates)); + std::vector segment_key_types_dup = batch_info.segment_key_types; + ARROW_ASSIGN_OR_RAISE(auto segmenter, + GroupingSegmenter::Make(std::move(segment_key_types_dup), ctx)); + return std::make_unique( + std::move(batch_info.aggs_argument_types), std::move(batch_info.key_types), + std::move(batch_info.segment_key_types), aggregates, std::move(segmenter), + use_threads, ctx); } - RETURN_NOT_OK(task_group->Finish()); + static Status CheckSizes(const std::vector& arguments, + const std::vector& aggregates) { + size_t total_target_size = 0; + for (const auto& agg : aggregates) { + total_target_size += agg.target.size(); + } + if (total_target_size != arguments.size()) { + return Status::Invalid("Aggregate functions expect exactly ", total_target_size, + " arguments, but ", arguments.size(), " were specified."); + } + return Status::OK(); + } - // Merge if necessary - for (size_t thread_index = 1; thread_index < thread_ids.size(); ++thread_index) { - ARROW_ASSIGN_OR_RAISE(ExecBatch other_keys, groupers[thread_index]->GetUniques()); - ARROW_ASSIGN_OR_RAISE(Datum transposition, - groupers[0]->Consume(ExecSpan(other_keys))); - groupers[thread_index].reset(); + template + static Status CheckTypes(const std::vector& expected_types, + const std::vector& actual_types, + const std::string& types_kind) { + if (expected_types != actual_types) { + return Status::Invalid("expected ", types_kind, " ", ToString(expected_types), + " but got ", ToString(actual_types)); + } + return Status::OK(); + } - for (size_t idx = 0; idx < kernels.size(); ++idx) { - KernelContext batch_ctx{ctx}; - batch_ctx.SetState(states[0][idx].get()); + Status CheckTypes(const BatchInfo& batch_info) { + ARROW_RETURN_NOT_OK(CheckTypes(aggs_argument_types, batch_info.aggs_argument_types, + "argument types")); + ARROW_RETURN_NOT_OK(CheckTypes(key_types, batch_info.key_types, "key types")); + ARROW_RETURN_NOT_OK( + CheckTypes(segment_key_types, batch_info.segment_key_types, "segment key types")); + return Status::OK(); + } - RETURN_NOT_OK(kernels[idx]->resize(&batch_ctx, groupers[0]->num_groups())); - RETURN_NOT_OK(kernels[idx]->merge(&batch_ctx, std::move(*states[thread_index][idx]), - *transposition.array())); - states[thread_index][idx].reset(); - } + Result Run(const BatchInfo& batch_info) { + ARROW_RETURN_NOT_OK(CheckTypes(batch_info)); + ARROW_RETURN_NOT_OK(state_info.Init()); + + // Consume batch + ARROW_RETURN_NOT_OK(state_info.Consume(batch_info)); + + // Merge if necessary + ARROW_RETURN_NOT_OK(state_info.Merge()); + + // Finalize output + return state_info.Finalize(); } - // Finalize output - ArrayDataVector out_data(kernels.size() + keys.size()); - auto it = out_data.begin(); + Status Run(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, GroupByCallback callback) { + ARROW_ASSIGN_OR_RAISE(auto batch_info, + BatchInfo::Make(arguments, keys, segment_keys, aggregates)); + ARROW_RETURN_NOT_OK(CheckTypes(batch_info)); - for (size_t idx = 0; idx < kernels.size(); ++idx) { - KernelContext batch_ctx{ctx}; - batch_ctx.SetState(states[0][idx].get()); - Datum out; - RETURN_NOT_OK(kernels[idx]->finalize(&batch_ctx, &out)); - *it++ = out.array(); + if (segment_keys.size() == 0) { + // an optimized code-path - the code works correctly without it + ARROW_ASSIGN_OR_RAISE(auto datum, Run(std::move(batch_info))); + return callback(datum); + } + int64_t offset = 0; + while (true) { + ARROW_ASSIGN_OR_RAISE( + auto segment, segmenter->GetNextSegment(batch_info.segment_keys_batch, offset)); + if (segment.offset >= batch_info.segment_keys_batch.length) break; + BatchInfo segment_batch_info = batch_info.Slice(segment.offset, segment.length); + ARROW_ASSIGN_OR_RAISE(auto datum, Run(segment_batch_info)); + ARROW_RETURN_NOT_OK(callback(datum)); + offset = segment.offset + segment.length; + } + return Status::OK(); } - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, groupers[0]->GetUniques()); - for (const auto& key : out_keys.values) { - *it++ = key.array(); + Result Run(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys) { + ArrayVector arrays; + ARROW_RETURN_NOT_OK(Run(arguments, keys, segment_keys, [&arrays](const Datum& datum) { + arrays.push_back(datum.make_array()); + return Status::OK(); + })); + if (arrays.size() == 1) { + return arrays[0]; + } else { + return ChunkedArray::Make(arrays); + } } - const int64_t out_length = out_data[0]->length; - return ArrayData::Make(struct_(std::move(out_fields)), out_length, - {/*null_bitmap=*/nullptr}, std::move(out_data), - /*null_count=*/0); + private: + const std::vector> aggs_argument_types; + const std::vector key_types; + const std::vector segment_key_types; + const std::vector& aggregates; + std::unique_ptr segmenter; + bool use_threads; + ExecContext* ctx; + FieldVector key_fields; + StateInfo state_info; +}; + +} // namespace + +Result GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& aggregates, bool use_threads, + ExecContext* ctx) { + return GroupBy(arguments, keys, {}, aggregates, use_threads, ctx); +} + +Result GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, bool use_threads, + ExecContext* ctx) { + ARROW_ASSIGN_OR_RAISE(auto gbp, GroupByProcess::Make(arguments, keys, segment_keys, + aggregates, use_threads, ctx)); + return gbp->Run(arguments, keys, segment_keys); +} + +Status GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& aggregates, GroupByCallback callback, + bool use_threads, ExecContext* ctx) { + return GroupBy(arguments, keys, {}, aggregates, callback, use_threads, ctx); +} + +Status GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, GroupByCallback callback, + bool use_threads, ExecContext* ctx) { + ARROW_ASSIGN_OR_RAISE(auto gbp, GroupByProcess::Make(arguments, keys, segment_keys, + aggregates, use_threads, ctx)); + return gbp->Run(arguments, keys, segment_keys, callback); } } // namespace internal diff --git a/cpp/src/arrow/compute/exec/aggregate.h b/cpp/src/arrow/compute/exec/aggregate.h index 027449f02a0..b875da9443c 100644 --- a/cpp/src/arrow/compute/exec/aggregate.h +++ b/cpp/src/arrow/compute/exec/aggregate.h @@ -32,14 +32,34 @@ namespace arrow { namespace compute { namespace internal { -/// Internal use only: helper function for testing HashAggregateKernels. +/// Internal use only: helpers for PyArrow and testing HashAggregateKernels. /// For public use see arrow::compute::Grouper or create an execution plan /// and use an aggregate node. + +ARROW_EXPORT +Result GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& aggregates, bool use_threads = false, + ExecContext* ctx = default_exec_context()); + ARROW_EXPORT Result GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, const std::vector& aggregates, bool use_threads = false, ExecContext* ctx = default_exec_context()); +using GroupByCallback = std::function; + +ARROW_EXPORT +Status GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& aggregates, GroupByCallback callback, + bool use_threads = false, ExecContext* ctx = default_exec_context()); + +ARROW_EXPORT +Status GroupBy(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, GroupByCallback callback, + bool use_threads = false, ExecContext* ctx = default_exec_context()); + Result> GetKernels( ExecContext* ctx, const std::vector& aggregates, const std::vector>& in_types); diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 5efb49180fa..0bec4385376 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include #include @@ -44,6 +45,46 @@ namespace compute { namespace { +/// \brief A gated shared mutex is similar to a shared mutex, in that it allows either +/// multiple shared readers or a unique writer access to the mutex, except that a waiting +/// writer gates future readers by preventing them from reacquiring shared access until it +/// has acquired and released the mutex. This is useful for ensuring a writer is never +/// starved by readers. +struct GatedSharedMutex { + std::mutex gate; + std::shared_mutex mutex; +}; + +/// \brief Acquires unique access to a gatex mutex. This is useful for a unique writer. +class GatedUniqueLock { + public: + // acquires the gate first, to ensure future readers will wait for its release + explicit GatedUniqueLock(GatedSharedMutex& gated_shared_mutex) + : lock_gate_(gated_shared_mutex.gate), lock_mutex_(gated_shared_mutex.mutex) {} + + private: + std::unique_lock lock_gate_; + std::unique_lock lock_mutex_; +}; + +/// \brief Acquires shared access to a gatex mutex. This is useful for a shared reader. +class GatedSharedLock { + struct TouchGate { + explicit TouchGate(GatedSharedMutex& gated_shared_mutex) { + std::unique_lock lock_gate(gated_shared_mutex.gate); + } + }; + + public: + // acquires and immediately releases the gate first, to ensure no writer is waiting + explicit GatedSharedLock(GatedSharedMutex& gated_shared_mutex) + : touch_gate_(gated_shared_mutex), lock_mutex_(gated_shared_mutex.mutex) {} + + private: + TouchGate touch_gate_; + std::shared_lock lock_mutex_; +}; + void AggregatesToString(std::stringstream* ss, const Schema& input_schema, const std::vector& aggs, const std::vector>& target_fieldsets, @@ -70,10 +111,54 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema, *ss << ']'; } +template +Status HandleSegments(std::unique_ptr& segmenter, + const ExecBatch& batch, const std::vector& ids, + const BatchHandler& handle_batch) { + int64_t offset = 0; + ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids)); + while (true) { + ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset)); + if (segment.offset >= segment_batch.length) break; + ARROW_RETURN_NOT_OK(handle_batch(batch, segment)); + offset = segment.offset + segment.length; + } + return Status::OK(); +} + +Status GetScalarFields(std::vector& values, const ExecBatch& input_batch, + const std::vector& field_ids) { + DCHECK_GT(input_batch.length, 0); + int64_t row = input_batch.length - 1; + values.clear(); + values.resize(field_ids.size()); + for (size_t i = 0; i < field_ids.size(); i++) { + const Datum& value = input_batch.values[field_ids[i]]; + if (value.is_scalar()) { + values[i] = value; + } else if (value.is_array()) { + ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row)); + values[i] = scalar; + } else { + DCHECK(false); + } + } + return Status::OK(); +} + +void PlaceFields(ExecBatch& batch, size_t base, std::vector& values) { + DCHECK_LE(base + values.size(), batch.values.size()); + for (size_t i = 0; i < values.size(); i++) { + batch.values[base + i] = values[i]; + } +} + class ScalarAggregateNode : public ExecNode, public TracedNode { public: ScalarAggregateNode(ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema, + std::unique_ptr segmenter, + std::vector segment_field_ids, std::vector> target_fieldsets, std::vector aggs, std::vector kernels, @@ -81,6 +166,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { : ExecNode(plan, std::move(inputs), {"target"}, /*output_schema=*/std::move(output_schema)), TracedNode(this), + segmenter_(std::move(segmenter)), + segment_field_ids_(std::move(segment_field_ids)), target_fieldsets_(std::move(target_fieldsets)), aggs_(std::move(aggs)), kernels_(std::move(kernels)), @@ -92,13 +179,38 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { const auto& aggregate_options = checked_cast(options); auto aggregates = aggregate_options.aggregates; + const auto& keys = aggregate_options.keys; + const auto& segment_keys = aggregate_options.segment_keys; + + if (keys.size() > 0) { + return Status::Invalid("Scalar aggregation with some key"); + } + if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 && + segment_keys.size() > 0) { + return Status::NotImplemented("Segmented aggregation in a multi-threaded plan"); + } const auto& input_schema = *inputs[0]->output_schema(); auto exec_ctx = plan->query_context()->exec_context(); + std::vector segment_field_ids(segment_keys.size()); + std::vector segment_key_types(segment_keys.size()); + for (size_t i = 0; i < segment_keys.size(); i++) { + ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema)); + if (match.indices().size() > 1) { + // ARROW-18369: Support nested references as segment ids + return Status::Invalid("Nested references cannot be used as segment ids"); + } + segment_field_ids[i] = match[0]; + segment_key_types[i] = input_schema.field(match[0])->type().get(); + } + + ARROW_ASSIGN_OR_RAISE( + auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx)); + std::vector kernels(aggregates.size()); std::vector>> states(kernels.size()); - FieldVector fields(kernels.size()); + FieldVector fields(kernels.size() + segment_keys.size()); std::vector> target_fieldsets(kernels.size()); for (size_t i = 0; i < kernels.size(); ++i) { @@ -144,15 +256,21 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { fields[i] = field(aggregate_options.aggregates[i].name, out_type.GetSharedPtr()); } + for (size_t i = 0; i < segment_keys.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(fields[kernels.size() + i], + segment_keys[i].GetOne(*inputs[0]->output_schema())); + } return plan->EmplaceNode( - plan, std::move(inputs), schema(std::move(fields)), std::move(target_fieldsets), - std::move(aggregates), std::move(kernels), std::move(states)); + plan, std::move(inputs), schema(std::move(fields)), std::move(segmenter), + std::move(segment_field_ids), std::move(target_fieldsets), std::move(aggregates), + std::move(kernels), std::move(states)); } const char* kind_name() const override { return "ScalarAggregateNode"; } Status DoConsume(const ExecSpan& batch, size_t thread_index) { + GatedSharedLock lock(gated_shared_mutex_); for (size_t i = 0; i < kernels_.size(); ++i) { util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, @@ -177,12 +295,19 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto scope = TraceInputReceived(batch); DCHECK_EQ(input, inputs_[0]); - auto thread_index = plan_->query_context()->GetThreadIndex(); - - ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index)); + auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) { + if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); + auto exec_batch = full_batch.Slice(segment.offset, segment.length); + auto batch = ExecSpan(exec_batch); + RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex())); + RETURN_NOT_OK(GetScalarFields(segmenter_values_, exec_batch, segment_field_ids_)); + if (!segment.is_open) RETURN_NOT_OK(OutputResult()); + return Status::OK(); + }; + RETURN_NOT_OK(HandleSegments(segmenter_, batch, segment_field_ids_, handler)); if (input_counter_.Increment()) { - return Finish(); + RETURN_NOT_OK(OutputResult(/*is_last=*/true)); } return Status::OK(); } @@ -191,15 +316,14 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { EVENT_ON_CURRENT_SPAN("InputFinished", {{"batches.length", total_batches}}); DCHECK_EQ(input, inputs_[0]); if (input_counter_.SetTotal(total_batches)) { - return Finish(); + RETURN_NOT_OK(OutputResult(/*is_last=*/true)); } return Status::OK(); } Status StartProducing() override { NoteStartProducing(ToStringExtra()); - // Scalar aggregates will only output a single batch - return output_->InputFinished(this, 1); + return Status::OK(); } void PauseProducing(ExecNode* output, int32_t counter) override { @@ -221,10 +345,31 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { } private: - Status Finish() { - auto scope = TraceFinish(); + Status ReconstructAggregates() { + const auto& input_schema = *inputs()[0]->output_schema(); + auto exec_ctx = plan()->query_context()->exec_context(); + for (size_t i = 0; i < kernels_.size(); ++i) { + std::vector in_types; + for (const auto& target : target_fieldsets_[i]) { + in_types.emplace_back(input_schema.field(target)->type().get()); + } + states_[i].resize(plan()->query_context()->max_concurrency()); + KernelContext kernel_ctx{exec_ctx}; + RETURN_NOT_OK(Kernel::InitAll( + &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()}, + &states_[i])); + } + return Status::OK(); + } + + Status OutputResult(bool is_last = false, bool traced = false) { + if (is_last && !traced) { + auto scope = TraceFinish(); + return OutputResult(is_last, /*traced=*/true); + } + GatedUniqueLock lock(gated_shared_mutex_); ExecBatch batch{{}, 1}; - batch.values.resize(kernels_.size()); + batch.values.resize(kernels_.size() + segment_field_ids_.size()); for (size_t i = 0; i < kernels_.size(); ++i) { util::tracing::Span span; @@ -232,16 +377,28 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { {{"function.name", aggs_[i].function}, {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, - {"function.kind", std::string(kind_name()) + "::Finalize"}}); + {"function.kind", std::string(kind_name()) + "::Output"}}); KernelContext ctx{plan()->query_context()->exec_context()}; ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll( kernels_[i], &ctx, std::move(states_[i]))); RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i])); } + PlaceFields(batch, kernels_.size(), segmenter_values_); - return output_->InputReceived(this, std::move(batch)); + ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch))); + total_output_batches_++; + if (is_last) { + ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); + } else { + ARROW_RETURN_NOT_OK(ReconstructAggregates()); + } + return Status::OK(); } + std::unique_ptr segmenter_; + const std::vector segment_field_ids_; + std::vector segmenter_values_; + const std::vector> target_fieldsets_; const std::vector aggs_; const std::vector kernels_; @@ -249,18 +406,25 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { std::vector>> states_; AtomicCounter input_counter_; + int64_t total_output_batches_ = 0; + GatedSharedMutex gated_shared_mutex_; }; class GroupByNode : public ExecNode, public TracedNode { public: GroupByNode(ExecNode* input, std::shared_ptr output_schema, - std::vector key_field_ids, + std::vector key_field_ids, std::vector segment_key_field_ids, + std::unique_ptr segmenter, + std::vector> agg_src_types, std::vector> agg_src_fieldsets, std::vector aggs, std::vector agg_kernels) : ExecNode(input->plan(), {input}, {"groupby"}, std::move(output_schema)), TracedNode(this), + segmenter_(std::move(segmenter)), key_field_ids_(std::move(key_field_ids)), + segment_key_field_ids_(std::move(segment_key_field_ids)), + agg_src_types_(std::move(agg_src_types)), agg_src_fieldsets_(std::move(agg_src_fieldsets)), aggs_(std::move(aggs)), agg_kernels_(std::move(agg_kernels)) {} @@ -279,9 +443,15 @@ class GroupByNode : public ExecNode, public TracedNode { auto input = inputs[0]; const auto& aggregate_options = checked_cast(options); const auto& keys = aggregate_options.keys; + const auto& segment_keys = aggregate_options.segment_keys; // Copy (need to modify options pointer below) auto aggs = aggregate_options.aggregates; + if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 && + segment_keys.size() > 0) { + return Status::NotImplemented("Segmented aggregation in a multi-threaded plan"); + } + // Get input schema auto input_schema = input->output_schema(); @@ -292,6 +462,13 @@ class GroupByNode : public ExecNode, public TracedNode { key_field_ids[i] = match[0]; } + // Find input field indices for segment key fields + std::vector segment_key_field_ids(segment_keys.size()); + for (size_t i = 0; i < segment_keys.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(*input_schema)); + segment_key_field_ids[i] = match[0]; + } + // Find input field indices for aggregates std::vector> agg_src_fieldsets(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { @@ -310,8 +487,18 @@ class GroupByNode : public ExecNode, public TracedNode { } } + // Build vector of segment key field data types + std::vector segment_key_types(segment_keys.size()); + for (size_t i = 0; i < segment_keys.size(); ++i) { + auto segment_key_field_id = segment_key_field_ids[i]; + segment_key_types[i] = input_schema->field(segment_key_field_id)->type().get(); + } + auto ctx = plan->query_context()->exec_context(); + ARROW_ASSIGN_OR_RAISE(auto segmenter, + GroupingSegmenter::Make(std::move(segment_key_types), ctx)); + // Construct aggregates ARROW_ASSIGN_OR_RAISE(auto agg_kernels, internal::GetKernels(ctx, aggs, agg_src_types)); @@ -324,7 +511,7 @@ class GroupByNode : public ExecNode, public TracedNode { internal::ResolveKernels(aggs, agg_kernels, agg_states, ctx, agg_src_types)); // Build field vector for output schema - FieldVector output_fields{keys.size() + aggs.size()}; + FieldVector output_fields{keys.size() + segment_keys.size() + aggs.size()}; // Aggregate fields come before key fields to match the behavior of GroupBy function for (size_t i = 0; i < aggs.size(); ++i) { @@ -336,15 +523,33 @@ class GroupByNode : public ExecNode, public TracedNode { int key_field_id = key_field_ids[i]; output_fields[base + i] = input_schema->field(key_field_id); } + base += keys.size(); + for (size_t i = 0; i < segment_keys.size(); ++i) { + int segment_key_field_id = segment_key_field_ids[i]; + output_fields[base + i] = input_schema->field(segment_key_field_id); + } return input->plan()->EmplaceNode( input, schema(std::move(output_fields)), std::move(key_field_ids), + std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types), std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels)); } + Status ReconstructAggregates() { + auto ctx = plan()->query_context()->exec_context(); + + ARROW_ASSIGN_OR_RAISE(agg_kernels_, internal::GetKernels(ctx, aggs_, agg_src_types_)); + + ARROW_ASSIGN_OR_RAISE( + auto agg_states, internal::InitKernels(agg_kernels_, ctx, aggs_, agg_src_types_)); + + return Status::OK(); + } + const char* kind_name() const override { return "GroupByNode"; } Status Consume(ExecSpan batch) { + GatedSharedLock lock(gated_shared_mutex_); size_t thread_index = plan_->query_context()->GetThreadIndex(); if (thread_index >= local_states_.size()) { return Status::IndexError("thread index ", thread_index, " is out of range [0, ", @@ -438,7 +643,8 @@ class GroupByNode : public ExecNode, public TracedNode { RETURN_NOT_OK(InitLocalStateIfNeeded(state)); ExecBatch out_data{{}, state->grouper->num_groups()}; - out_data.values.resize(agg_kernels_.size() + key_field_ids_.size()); + out_data.values.resize(agg_kernels_.size() + key_field_ids_.size() + + segment_key_field_ids_.size()); // Aggregate fields come before key fields to match the behavior of GroupBy function for (size_t i = 0; i < agg_kernels_.size(); ++i) { @@ -457,6 +663,7 @@ class GroupByNode : public ExecNode, public TracedNode { ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, state->grouper->GetUniques()); std::move(out_keys.values.begin(), out_keys.values.end(), out_data.values.begin() + agg_kernels_.size()); + PlaceFields(out_data, agg_kernels_.size() + key_field_ids_.size(), segmenter_values_); state->grouper.reset(); return out_data; } @@ -466,8 +673,12 @@ class GroupByNode : public ExecNode, public TracedNode { return output_->InputReceived(this, out_data_.Slice(batch_size * n, batch_size)); } - Status OutputResult() { - auto scope = TraceFinish(); + Status OutputResult(bool is_last = false, bool traced = false) { + if (is_last && !traced) { + auto scope = TraceFinish(); + return OutputResult(is_last, /*traced=*/true); + } + GatedUniqueLock lock(gated_shared_mutex_); // To simplify merging, ensure that the first grouper is nonempty for (size_t i = 0; i < local_states_.size(); i++) { if (local_states_[i].grouper) { @@ -480,9 +691,19 @@ class GroupByNode : public ExecNode, public TracedNode { ARROW_ASSIGN_OR_RAISE(out_data_, Finalize()); int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size()); - RETURN_NOT_OK(output_->InputFinished(this, static_cast(num_output_batches))); - return plan_->query_context()->StartTaskGroup(output_task_group_id_, - num_output_batches); + total_output_batches_ += num_output_batches; + if (is_last) { + ARROW_RETURN_NOT_OK( + output_->InputFinished(this, static_cast(total_output_batches_))); + RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_, + num_output_batches)); + } else { + for (int64_t i = 0; i < num_output_batches; i++) { + ARROW_RETURN_NOT_OK(OutputNthBatch(i)); + } + ARROW_RETURN_NOT_OK(ReconstructAggregates()); + } + return Status::OK(); } Status InputReceived(ExecNode* input, ExecBatch batch) override { @@ -490,10 +711,21 @@ class GroupByNode : public ExecNode, public TracedNode { DCHECK_EQ(input, inputs_[0]); - ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch))); + auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) { + if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); + auto exec_batch = full_batch.Slice(segment.offset, segment.length); + auto batch = ExecSpan(exec_batch); + RETURN_NOT_OK(Consume(batch)); + RETURN_NOT_OK( + GetScalarFields(segmenter_values_, exec_batch, segment_key_field_ids_)); + if (!segment.is_open) RETURN_NOT_OK(OutputResult()); + return Status::OK(); + }; + ARROW_RETURN_NOT_OK( + HandleSegments(segmenter_, batch, segment_key_field_ids_, handler)); if (input_counter_.Increment()) { - return OutputResult(); + ARROW_RETURN_NOT_OK(OutputResult(/*is_last=*/true)); } return Status::OK(); } @@ -502,7 +734,7 @@ class GroupByNode : public ExecNode, public TracedNode { DCHECK_EQ(input, inputs_[0]); if (input_counter_.SetTotal(total_batches)) { - return OutputResult(); + RETURN_NOT_OK(OutputResult(/*is_last=*/true)); } return Status::OK(); } @@ -515,12 +747,12 @@ class GroupByNode : public ExecNode, public TracedNode { void PauseProducing(ExecNode* output, int32_t counter) override { // TODO(ARROW-16260) - // Without spillover there is way to handle backpressure in this node + // Without spillover there is no way to handle backpressure in this node } void ResumeProducing(ExecNode* output, int32_t counter) override { // TODO(ARROW-16260) - // Without spillover there is way to handle backpressure in this node + // Without spillover there is no way to handle backpressure in this node } Status StopProducingImpl() override { return Status::OK(); } @@ -593,16 +825,22 @@ class GroupByNode : public ExecNode, public TracedNode { } int output_task_group_id_; + std::unique_ptr segmenter_; + std::vector segmenter_values_; const std::vector key_field_ids_; + const std::vector segment_key_field_ids_; + const std::vector> agg_src_types_; const std::vector> agg_src_fieldsets_; const std::vector aggs_; - const std::vector agg_kernels_; + std::vector agg_kernels_; AtomicCounter input_counter_; + int64_t total_output_batches_ = 0; std::vector local_states_; ExecBatch out_data_; + GatedSharedMutex gated_shared_mutex_; }; } // namespace diff --git a/cpp/src/arrow/compute/exec/exec_plan.h b/cpp/src/arrow/compute/exec/exec_plan.h index c2738945c27..c63e861a615 100644 --- a/cpp/src/arrow/compute/exec/exec_plan.h +++ b/cpp/src/arrow/compute/exec/exec_plan.h @@ -196,8 +196,7 @@ class ARROW_EXPORT ExecNode { /// concurrently, potentially even before the call to StartProducing /// has finished. /// - PauseProducing(), ResumeProducing(), StopProducing() may be called - /// by the downstream nodes' InputReceived(), ErrorReceived(), InputFinished() - /// methods + /// by the downstream nodes' InputReceived(), InputFinished() methods /// /// StopProducing may be called due to an error, by the user (e.g. cancel), or /// because a node has all the data it needs (e.g. limit, top-k on sorted data). diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index bd2bbcb8e64..4578ab9332d 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -199,7 +199,13 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { std::vector names; }; -/// \brief Make a node which aggregates input batches, optionally grouped by keys. +/// \brief Make a node which aggregates input batches, optionally grouped by keys and +/// optionally segmented by segment-keys. Both keys and segment-keys determine the group. +/// However segment-keys are also used for determining grouping segments, which should be +/// large, and allow streaming a partial aggregation result after processing each segment. +/// One common use-case for segment-keys is ordered aggregation, in which the segment-key +/// attribute specifies a column with non-decreasing values or a lexigographically-ordered +/// set of such columns. /// /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is /// expected to be a HashAggregate function. If the keys attribute is an empty vector, @@ -207,13 +213,18 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { public: explicit AggregateNodeOptions(std::vector aggregates, - std::vector keys = {}) - : aggregates(std::move(aggregates)), keys(std::move(keys)) {} + std::vector keys = {}, + std::vector segment_keys = {}) + : aggregates(std::move(aggregates)), + keys(std::move(keys)), + segment_keys(std::move(segment_keys)) {} // aggregations which will be applied to the targetted fields std::vector aggregates; // keys by which aggregations will be grouped std::vector keys; + // keys by which aggregations will be segmented + std::vector segment_keys; }; constexpr int32_t kDefaultBackpressureHighBytes = 1 << 30; // 1GiB diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 2fb3a28baf0..f1154b4ac1e 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -71,6 +71,42 @@ using internal::checked_pointer_cast; namespace compute { namespace { +bool kDefaultUseThreads = false; +ExecContext* kDefaultCtx = default_exec_context(); + +using GroupByFunction = std::function( + const std::vector&, const std::vector&, const std::vector&, + const std::vector&, bool, ExecContext*, bool)>; + +Result GroupByDirectImpl(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads = kDefaultUseThreads, + ExecContext* ctx = kDefaultCtx, bool naive = false) { + return internal::GroupBy(arguments, keys, segment_keys, aggregates, use_threads, ctx); +} + +Result GroupByWithArrays(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads, ExecContext* ctx, bool naive) { + ArrayVector arrays; + ARROW_RETURN_NOT_OK(internal::GroupBy( + arguments, keys, segment_keys, aggregates, + [&arrays](const Datum& datum) { + arrays.push_back(datum.make_array()); + return Status::OK(); + }, + use_threads, ctx)); + if (arrays.size() == 1) { + return arrays[0]; + } else { + return ChunkedArray::Make(arrays); + } +} + Result NaiveGroupBy(std::vector arguments, std::vector keys, const std::vector& aggregates) { ARROW_ASSIGN_OR_RAISE(auto key_batch, ExecBatch::Make(std::move(keys))); @@ -121,23 +157,96 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys return StructArray::Make(std::move(out_columns), std::move(out_names)); } +Result MakeGroupByOutput(const std::vector& output_batches, + const std::shared_ptr output_schema, + size_t num_aggregates, size_t num_keys, bool naive) { + ArrayVector out_arrays(num_aggregates + num_keys); + for (size_t i = 0; i < out_arrays.size(); ++i) { + std::vector> arrays(output_batches.size()); + for (size_t j = 0; j < output_batches.size(); ++j) { + arrays[j] = output_batches[j].values[i].make_array(); + } + if (arrays.empty()) { + ARROW_ASSIGN_OR_RAISE( + out_arrays[i], + MakeArrayOfNull(output_schema->field(static_cast(i))->type(), + /*length=*/0)); + } else { + ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays)); + } + } + + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr struct_arr, + StructArray::Make(std::move(out_arrays), output_schema->fields())); + + bool need_sort = !naive; + for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) { + if (output_schema->field(i)->type()->id() == Type::DICTIONARY) { + need_sort = false; + } + } + if (!need_sort) { + return struct_arr; + } + + // The exec plan may reorder the output rows. The tests are all setup to expect ouptut + // in ascending order of keys. So we need to sort the result by the key columns. To do + // that we create a table using the key columns, calculate the sort indices from that + // table (sorting on all fields) and then use those indices to calculate our result. + std::vector> key_fields; + std::vector> key_columns; + std::vector sort_keys; + for (std::size_t i = 0; i < num_keys; i++) { + const std::shared_ptr& arr = out_arrays[i + num_aggregates]; + key_columns.push_back(arr); + key_fields.push_back(field("name_does_not_matter", arr->type())); + sort_keys.emplace_back(static_cast(i)); + } + std::shared_ptr key_schema = schema(std::move(key_fields)); + std::shared_ptr key_table = Table::Make(std::move(key_schema), key_columns); + SortOptions sort_options(std::move(sort_keys)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr sort_indices, + SortIndices(key_table, sort_options)); + + return Take(struct_arr, sort_indices); +} + Result GroupByUsingExecPlan(const BatchesWithSchema& input, const std::vector& key_names, + const std::vector& segment_key_names, const std::vector& aggregates, - bool use_threads) { + bool use_threads, bool segmented = false, + bool naive = false, ExecContext* ctx = nullptr) { + if (ctx == nullptr) { + if (segment_key_names.size() > 0) { + ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1)); + ExecContext seq_ctx(default_memory_pool(), thread_pool.get()); + return GroupByUsingExecPlan(input, key_names, segment_key_names, aggregates, + use_threads, segmented, naive, &seq_ctx); + } else { + return GroupByUsingExecPlan(input, key_names, segment_key_names, aggregates, + use_threads, segmented, naive, threaded_exec_context()); + } + } std::vector keys(key_names.size()); for (size_t i = 0; i < key_names.size(); ++i) { keys[i] = FieldRef(key_names[i]); } + std::vector segment_keys(segment_key_names.size()); + for (size_t i = 0; i < segment_key_names.size(); ++i) { + segment_keys[i] = FieldRef(segment_key_names[i]); + } - ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(*threaded_exec_context())); + ARROW_ASSIGN_OR_RAISE(auto plan, ExecPlan::Make(*ctx)); AsyncGenerator> sink_gen; RETURN_NOT_OK( Declaration::Sequence( { {"source", SourceNodeOptions{input.schema, input.gen(use_threads, /*slow=*/false)}}, - {"aggregate", AggregateNodeOptions{std::move(aggregates), std::move(keys)}}, + {"aggregate", AggregateNodeOptions{std::move(aggregates), std::move(keys), + std::move(segment_keys)}}, {"sink", SinkNodeOptions{&sink_gen}}, }) .AddToPlan(plan.get())); @@ -161,71 +270,103 @@ Result GroupByUsingExecPlan(const BatchesWithSchema& input, ARROW_ASSIGN_OR_RAISE(std::vector output_batches, start_and_collect.MoveResult()); - ArrayVector out_arrays(aggregates.size() + key_names.size()); const auto& output_schema = plan->nodes()[0]->output()->output_schema(); + if (!segmented) { + return MakeGroupByOutput(output_batches, output_schema, aggregates.size(), + key_names.size(), naive); + } + + std::vector out_arrays(aggregates.size() + key_names.size() + + segment_key_names.size()); for (size_t i = 0; i < out_arrays.size(); ++i) { std::vector> arrays(output_batches.size()); for (size_t j = 0; j < output_batches.size(); ++j) { - arrays[j] = output_batches[j].values[i].make_array(); + auto& value = output_batches[j].values[i]; + if (value.is_scalar()) { + ARROW_ASSIGN_OR_RAISE( + arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length)); + } else if (value.is_array()) { + arrays[j] = value.make_array(); + } else { + return Status::Invalid("GroupByUsingExecPlan unsupported value kind ", + ToString(value.kind())); + } } if (arrays.empty()) { + arrays.resize(1); ARROW_ASSIGN_OR_RAISE( - out_arrays[i], - MakeArrayOfNull(output_schema->field(static_cast(i))->type(), - /*length=*/0)); - } else { - ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays)); + arrays[0], MakeArrayOfNull(output_schema->field(static_cast(i))->type(), + /*length=*/0)); } + out_arrays[i] = {std::move(arrays)}; } - // The exec plan may reorder the output rows. The tests are all setup to expect ouptut - // in ascending order of keys. So we need to sort the result by the key columns. To do - // that we create a table using the key columns, calculate the sort indices from that - // table (sorting on all fields) and then use those indices to calculate our result. - std::vector> key_fields; - std::vector> key_columns; - std::vector sort_keys; - for (std::size_t i = 0; i < key_names.size(); i++) { - const std::shared_ptr& arr = out_arrays[i + aggregates.size()]; - key_columns.push_back(arr); - key_fields.push_back(field("name_does_not_matter", arr->type())); - sort_keys.emplace_back(static_cast(i)); + if (segmented && segment_key_names.size() > 0) { + ArrayVector struct_arrays; + struct_arrays.reserve(output_batches.size()); + for (size_t j = 0; j < output_batches.size(); ++j) { + ArrayVector struct_fields; + struct_fields.reserve(out_arrays.size()); + for (auto out_array : out_arrays) { + struct_fields.push_back(out_array[j]); + } + ARROW_ASSIGN_OR_RAISE(auto struct_array, + StructArray::Make(struct_fields, output_schema->fields())); + struct_arrays.push_back(struct_array); + } + return ChunkedArray::Make(struct_arrays); + } else { + ArrayVector struct_fields(out_arrays.size()); + for (size_t i = 0; i < out_arrays.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i])); + } + return StructArray::Make(std::move(struct_fields), output_schema->fields()); } - std::shared_ptr key_schema = schema(std::move(key_fields)); - std::shared_ptr
key_table = Table::Make(std::move(key_schema), key_columns); - SortOptions sort_options(std::move(sort_keys)); - ARROW_ASSIGN_OR_RAISE(std::shared_ptr sort_indices, - SortIndices(key_table, sort_options)); - - ARROW_ASSIGN_OR_RAISE( - std::shared_ptr struct_arr, - StructArray::Make(std::move(out_arrays), output_schema->fields())); +} - return Take(struct_arr, sort_indices); +Result GroupByUsingExecPlan(const BatchesWithSchema& input, + const std::vector& key_names, + const std::vector& aggregates, + bool use_threads, bool segmented = false, + bool naive = false) { + return GroupByUsingExecPlan(input, key_names, {}, aggregates, use_threads, segmented); } /// Simpler overload where you can give the columns as datums Result GroupByUsingExecPlan(const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, const std::vector& aggregates, - bool use_threads) { + bool use_threads, bool segmented = false, + bool naive = false) { using arrow::compute::detail::ExecSpanIterator; - FieldVector scan_fields(arguments.size() + keys.size()); + FieldVector scan_fields(arguments.size() + keys.size() + segment_keys.size()); std::vector key_names(keys.size()); + std::vector segment_key_names(segment_keys.size()); for (size_t i = 0; i < arguments.size(); ++i) { auto name = std::string("agg_") + std::to_string(i); scan_fields[i] = field(name, arguments[i].type()); } + size_t base = arguments.size(); for (size_t i = 0; i < keys.size(); ++i) { auto name = std::string("key_") + std::to_string(i); - scan_fields[arguments.size() + i] = field(name, keys[i].type()); + scan_fields[base + i] = field(name, keys[i].type()); key_names[i] = std::move(name); } + base += keys.size(); + size_t j = segmented ? keys.size() : keys.size(); + std::string prefix(segmented ? "key_" : "key_"); + for (size_t i = 0; i < segment_keys.size(); ++i) { + auto name = prefix + std::to_string(j++); + scan_fields[base + i] = field(name, segment_keys[i].type()); + segment_key_names[i] = std::move(name); + } std::vector inputs = arguments; - inputs.reserve(inputs.size() + keys.size()); + inputs.reserve(inputs.size() + keys.size() + segment_keys.size()); inputs.insert(inputs.end(), keys.begin(), keys.end()); + inputs.insert(inputs.end(), segment_keys.begin(), segment_keys.end()); ExecSpanIterator span_iterator; ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make(inputs)); @@ -238,14 +379,37 @@ Result GroupByUsingExecPlan(const std::vector& arguments, input.batches.push_back(span.ToExecBatch()); } - return GroupByUsingExecPlan(input, key_names, aggregates, use_threads); + return GroupByUsingExecPlan(input, key_names, segment_key_names, aggregates, + use_threads, segmented, naive); } -void ValidateGroupBy(const std::vector& aggregates, +Result GroupByUsingExecPlanImpl(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads, ExecContext* ctx, + bool naive = false) { + return GroupByUsingExecPlan(arguments, keys, segment_keys, aggregates, use_threads, + ctx); +} + +Result SegmentedGroupByUsingExecPlan(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads, ExecContext* ctx, + bool naive = false) { + return GroupByUsingExecPlan(arguments, keys, segment_keys, aggregates, use_threads, + /*segmented=*/true, naive); +} + +void ValidateGroupBy(GroupByFunction group_by, const std::vector& aggregates, std::vector arguments, std::vector keys) { ASSERT_OK_AND_ASSIGN(Datum expected, NaiveGroupBy(arguments, keys, aggregates)); - ASSERT_OK_AND_ASSIGN(Datum actual, internal::GroupBy(arguments, keys, aggregates)); + ASSERT_OK_AND_ASSIGN(Datum actual, group_by(arguments, keys, {}, aggregates, + kDefaultUseThreads, kDefaultCtx, + /*naive=*/true)); ASSERT_OK(expected.make_array()->ValidateFull()); ValidateOutput(actual); @@ -266,8 +430,9 @@ struct TestAggregate { std::shared_ptr options; }; -Result GroupByTest(const std::vector& arguments, +Result GroupByTest(GroupByFunction group_by, const std::vector& arguments, const std::vector& keys, + const std::vector& segment_keys, const std::vector& aggregates, bool use_threads, bool use_exec_plan = false) { std::vector internal_aggregates; @@ -278,31 +443,39 @@ Result GroupByTest(const std::vector& arguments, idx = idx + 1; } if (use_exec_plan) { - return GroupByUsingExecPlan(arguments, keys, internal_aggregates, use_threads); + return GroupByUsingExecPlan(arguments, keys, segment_keys, internal_aggregates, + use_threads); } else { - return internal::GroupBy(arguments, keys, internal_aggregates, use_threads, - default_exec_context()); + return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads, + default_exec_context(), /*naive=*/false); } } -} // namespace +Result GroupByTest(GroupByFunction group_by, const std::vector& arguments, + const std::vector& keys, + const std::vector& aggregates, bool use_threads, + bool use_exec_plan = false) { + return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads, + use_exec_plan); +} -TEST(Grouper, SupportedKeys) { - ASSERT_OK(Grouper::Make({boolean()})); +template +void TestGroupClassSupportedKeys() { + ASSERT_OK(GroupClass::Make({boolean()})); - ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()})); + ASSERT_OK(GroupClass::Make({int8(), uint16(), int32(), uint64()})); - ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())})); + ASSERT_OK(GroupClass::Make({dictionary(int64(), utf8())})); - ASSERT_OK(Grouper::Make({float16(), float32(), float64()})); + ASSERT_OK(GroupClass::Make({float16(), float32(), float64()})); - ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()})); + ASSERT_OK(GroupClass::Make({utf8(), binary(), large_utf8(), large_binary()})); - ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)})); + ASSERT_OK(GroupClass::Make({fixed_size_binary(16), fixed_size_binary(32)})); - ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)})); + ASSERT_OK(GroupClass::Make({decimal128(32, 10), decimal256(76, 20)})); - ASSERT_OK(Grouper::Make({date32(), date64()})); + ASSERT_OK(GroupClass::Make({date32(), date64()})); for (auto unit : { TimeUnit::SECOND, @@ -310,25 +483,149 @@ TEST(Grouper, SupportedKeys) { TimeUnit::MICRO, TimeUnit::NANO, }) { - ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)})); + ASSERT_OK(GroupClass::Make({timestamp(unit), duration(unit)})); + } + + ASSERT_OK(GroupClass::Make( + {day_time_interval(), month_interval(), month_day_nano_interval()})); + + ASSERT_OK(GroupClass::Make({null()})); + + ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({field("", int64())})})); + + ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({})})); + + ASSERT_RAISES(NotImplemented, GroupClass::Make({list(int32())})); + + ASSERT_RAISES(NotImplemented, GroupClass::Make({fixed_size_list(int32(), 5)})); + + ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})})); +} + +template +void TestSegments(std::unique_ptr& segmenter, const Batch& batch, + std::vector expected_segments) { + int64_t offset = 0; + for (auto expected_segment : expected_segments) { + ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset)); + ASSERT_EQ(expected_segment, segment); + offset = segment.offset + segment.length; + } +} + +} // namespace + +TEST(GroupingSegmenter, SupportedKeys) { + TestGroupClassSupportedKeys(); +} + +namespace { + +template +void test_grouping_segmenter_basics(SetupBatch setup, ConvertBatch convert) { + std::vector bad_types2 = {int32(), float32()}; + std::vector types2 = {int32(), int32()}; + std::vector bad_types1 = {float32()}; + std::vector types1 = {int32()}; + std::vector types0 = {}; + ASSERT_OK_AND_ASSIGN(auto batch2, + setup(ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]"))); + ASSERT_OK_AND_ASSIGN(auto batch1, setup(ExecBatchFromJSON(types1, "[[1], [1], [2]]"))); + ExecBatch batch0({}, 3); + { + SCOPED_TRACE("offset"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto converted0, convert(batch0)); + for (int64_t offset : {-1, 4}) { + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + HasSubstr("invalid grouping segmenter offset"), + segmenter->GetNextSegment(converted0, offset)); + } + } + { + SCOPED_TRACE("types0 segmenting of batch2"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "), + segmenter->GetNextSegment(converted2, 0)); + ASSERT_OK_AND_ASSIGN(auto converted0, convert(batch0)); + TestSegments(segmenter, converted0, {{0, 3, true, true}, {3, 0, true, true}}); + } + { + SCOPED_TRACE("bad_types1 segmenting of batch1"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types1)); + ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "), + segmenter->GetNextSegment(converted1, 0)); } + { + SCOPED_TRACE("types1 segmenting of batch2"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types1)); + ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "), + segmenter->GetNextSegment(converted2, 0)); + ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); + TestSegments(segmenter, converted1, + {{0, 2, false, true}, {2, 1, true, false}, {3, 0, true, true}}); + } + { + SCOPED_TRACE("bad_types2 segmenting of batch2"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types2)); + ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "), + segmenter->GetNextSegment(converted2, 0)); + } + { + SCOPED_TRACE("types2 segmenting of batch1"); + ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types2)); + ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "), + segmenter->GetNextSegment(converted1, 0)); + ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + TestSegments(segmenter, converted2, + {{0, 1, false, true}, + {1, 1, false, false}, + {2, 1, true, false}, + {3, 0, true, true}}); + } +} - ASSERT_OK( - Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()})); +auto batch_identity = [](const ExecBatch& batch) -> Result { return batch; }; - ASSERT_OK(Grouper::Make({null()})); +auto batch_to_span = [](const ExecBatch& batch) -> Result { + return ExecSpan(batch); +}; - ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})})); +auto batch_make_chunked = [](const ExecBatch& batch) -> Result { + int64_t length = batch.length; + DCHECK_GT(length, 1); + std::vector values; + for (auto value : batch.values) { + DCHECK(value.is_array()); + auto array = value.make_array(); + ARROW_ASSIGN_OR_RAISE( + auto chunked, ChunkedArray::Make({array->Slice(0, 1), array->Slice(1, length)})); + values.emplace_back(chunked); + } + return ExecBatch(values, length); +}; - ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})})); +} // namespace - ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())})); +TEST(GroupingSegmenter, Basics) { + test_grouping_segmenter_basics(batch_identity, batch_identity); +} - ASSERT_RAISES(NotImplemented, Grouper::Make({fixed_size_list(int32(), 5)})); +TEST(GroupingSegmenter, SpanBasics) { + test_grouping_segmenter_basics(batch_identity, batch_to_span); +} - ASSERT_RAISES(NotImplemented, Grouper::Make({dense_union({field("", int32())})})); +TEST(GroupingSegmenter, ChunkedBasics) { + test_grouping_segmenter_basics(batch_make_chunked, batch_identity); } +TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(); } + struct TestGrouper { explicit TestGrouper(std::vector types, std::vector shapes = {}) : types_(std::move(types)), shapes_(std::move(shapes)) { @@ -764,7 +1061,50 @@ TEST(Grouper, ScalarValues) { } } -TEST(GroupBy, Errors) { +void TestSegmentKey(GroupByFunction group_by, const std::shared_ptr
& table, + Datum output, const std::vector& segment_keys); + +class GroupBy : public ::testing::TestWithParam { + public: + void ValidateGroupBy(const std::vector& aggregates, + std::vector arguments, std::vector keys) { + compute::ValidateGroupBy(GetParam(), aggregates, arguments, keys); + } + + Result GroupByTest(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads, bool use_exec_plan = false) { + return compute::GroupByTest(GetParam(), arguments, keys, segment_keys, aggregates, + use_threads, use_exec_plan); + } + + Result GroupByTest(const std::vector& arguments, + const std::vector& keys, + const std::vector& aggregates, + bool use_threads, bool use_exec_plan = false) { + return compute::GroupByTest(GetParam(), arguments, keys, aggregates, use_threads, + use_exec_plan); + } + + Result AlternatorGroupBy(const std::vector& arguments, + const std::vector& keys, + const std::vector& segment_keys, + const std::vector& aggregates, + bool use_threads = false, + ExecContext* ctx = default_exec_context()) { + return GetParam()(arguments, keys, segment_keys, aggregates, use_threads, ctx, + /*naive=*/false); + } + + void TestSegmentKey(const std::shared_ptr
& table, Datum output, + const std::vector& segment_keys) { + return compute::TestSegmentKey(GetParam(), table, output, segment_keys); + } +}; + +TEST_P(GroupBy, Errors) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("group_id", uint32())}), R"([ [1.0, 1], @@ -785,18 +1125,18 @@ TEST(GroupBy, Errors) { HasSubstr("Direct execution of HASH_AGGREGATE functions"))); } -TEST(GroupBy, NoBatches) { +TEST_P(GroupBy, NoBatches) { // Regression test for ARROW-14583: handle when no batches are // passed to the group by node before finalizing auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {}); - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - GroupByTest({table->GetColumnByName("argument")}, {table->GetColumnByName("key")}, - { - {"hash_count", nullptr}, - }, - /*use_threads=*/true, /*use_exec_plan=*/true)); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupByTest({table->GetColumnByName("argument")}, + {table->GetColumnByName("key")}, {}, + { + {"hash_count", nullptr}, + }, + /*use_threads=*/true, /*use_exec_plan=*/true)); AssertDatumsEqual(ArrayFromJSON(struct_({ field("hash_count", int64()), field("key_0", int64()), @@ -832,7 +1172,7 @@ void SortBy(std::vector names, Datum* aggregated_and_grouped) { } } // namespace -TEST(GroupBy, CountOnly) { +TEST_P(GroupBy, CountOnly) { for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -880,7 +1220,7 @@ TEST(GroupBy, CountOnly) { } } -TEST(GroupBy, CountScalar) { +TEST_P(GroupBy, CountScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({int32(), int64()}, {ArgShape::SCALAR, ArgShape::ARRAY}, @@ -920,7 +1260,7 @@ TEST(GroupBy, CountScalar) { } } -TEST(GroupBy, SumOnly) { +TEST_P(GroupBy, SumOnly) { for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -968,7 +1308,7 @@ TEST(GroupBy, SumOnly) { } } -TEST(GroupBy, SumMeanProductDecimal) { +TEST_P(GroupBy, SumMeanProductDecimal) { auto in_schema = schema({ field("argument0", decimal128(3, 2)), field("argument1", decimal256(3, 2)), @@ -1044,7 +1384,7 @@ TEST(GroupBy, SumMeanProductDecimal) { } } -TEST(GroupBy, MeanOnly) { +TEST_P(GroupBy, MeanOnly) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -1095,7 +1435,7 @@ TEST(GroupBy, MeanOnly) { } } -TEST(GroupBy, SumMeanProductScalar) { +TEST_P(GroupBy, SumMeanProductScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({int32(), int64()}, {ArgShape::SCALAR, ArgShape::ARRAY}, @@ -1133,7 +1473,7 @@ TEST(GroupBy, SumMeanProductScalar) { } } -TEST(GroupBy, VarianceAndStddev) { +TEST_P(GroupBy, VarianceAndStddev) { auto batch = RecordBatchFromJSON( schema({field("argument", int32()), field("key", int64())}), R"([ [1, 1], @@ -1157,6 +1497,7 @@ TEST(GroupBy, VarianceAndStddev) { { batch->GetColumnByName("key"), }, + {}, { {"hash_variance", nullptr}, {"hash_stddev", nullptr}, @@ -1199,6 +1540,7 @@ TEST(GroupBy, VarianceAndStddev) { { batch->GetColumnByName("key"), }, + {}, { {"hash_variance", nullptr}, {"hash_stddev", nullptr}, @@ -1230,6 +1572,7 @@ TEST(GroupBy, VarianceAndStddev) { { batch->GetColumnByName("key"), }, + {}, { {"hash_variance", variance_options}, {"hash_stddev", variance_options}, @@ -1251,7 +1594,7 @@ TEST(GroupBy, VarianceAndStddev) { /*verbose=*/true); } -TEST(GroupBy, VarianceAndStddevDecimal) { +TEST_P(GroupBy, VarianceAndStddevDecimal) { auto batch = RecordBatchFromJSON( schema({field("argument0", decimal128(3, 2)), field("argument1", decimal128(3, 2)), field("key", int64())}), @@ -1277,6 +1620,7 @@ TEST(GroupBy, VarianceAndStddevDecimal) { { batch->GetColumnByName("key"), }, + {}, { {"hash_variance", nullptr}, {"hash_stddev", nullptr}, @@ -1301,7 +1645,7 @@ TEST(GroupBy, VarianceAndStddevDecimal) { /*verbose=*/true); } -TEST(GroupBy, TDigest) { +TEST_P(GroupBy, TDigest) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ [1, 1], @@ -1346,6 +1690,7 @@ TEST(GroupBy, TDigest) { { batch->GetColumnByName("key"), }, + {}, { {"hash_tdigest", nullptr}, {"hash_tdigest", options1}, @@ -1377,7 +1722,7 @@ TEST(GroupBy, TDigest) { /*verbose=*/true); } -TEST(GroupBy, TDigestDecimal) { +TEST_P(GroupBy, TDigestDecimal) { auto batch = RecordBatchFromJSON( schema({field("argument0", decimal128(3, 2)), field("argument1", decimal256(3, 2)), field("key", int64())}), @@ -1420,7 +1765,7 @@ TEST(GroupBy, TDigestDecimal) { /*verbose=*/true); } -TEST(GroupBy, ApproximateMedian) { +TEST_P(GroupBy, ApproximateMedian) { for (const auto& type : {float64(), int8()}) { auto batch = RecordBatchFromJSON(schema({field("argument", type), field("key", int64())}), R"([ @@ -1458,6 +1803,7 @@ TEST(GroupBy, ApproximateMedian) { { batch->GetColumnByName("key"), }, + {}, { {"hash_approximate_median", options}, {"hash_approximate_median", keep_nulls}, @@ -1485,7 +1831,7 @@ TEST(GroupBy, ApproximateMedian) { } } -TEST(GroupBy, StddevVarianceTDigestScalar) { +TEST_P(GroupBy, StddevVarianceTDigestScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({int32(), float32(), int64()}, @@ -1534,7 +1880,7 @@ TEST(GroupBy, StddevVarianceTDigestScalar) { } } -TEST(GroupBy, VarianceOptions) { +TEST_P(GroupBy, VarianceOptions) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON( @@ -1628,7 +1974,7 @@ TEST(GroupBy, VarianceOptions) { } } -TEST(GroupBy, MinMaxOnly) { +TEST_P(GroupBy, MinMaxOnly) { auto in_schema = schema({ field("argument", float64()), field("argument1", null()), @@ -1701,7 +2047,7 @@ TEST(GroupBy, MinMaxOnly) { } } -TEST(GroupBy, MinMaxTypes) { +TEST_P(GroupBy, MinMaxTypes) { std::vector> types; types.insert(types.end(), NumericTypes().begin(), NumericTypes().end()); types.insert(types.end(), TemporalTypes().begin(), TemporalTypes().end()); @@ -1789,7 +2135,7 @@ TEST(GroupBy, MinMaxTypes) { } } -TEST(GroupBy, MinMaxDecimal) { +TEST_P(GroupBy, MinMaxDecimal) { auto in_schema = schema({ field("argument0", decimal128(3, 2)), field("argument1", decimal256(3, 2)), @@ -1858,7 +2204,7 @@ TEST(GroupBy, MinMaxDecimal) { } } -TEST(GroupBy, MinMaxBinary) { +TEST_P(GroupBy, MinMaxBinary) { for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { for (const auto& ty : BaseBinaryTypes()) { @@ -1912,7 +2258,7 @@ TEST(GroupBy, MinMaxBinary) { } } -TEST(GroupBy, MinMaxFixedSizeBinary) { +TEST_P(GroupBy, MinMaxFixedSizeBinary) { const auto ty = fixed_size_binary(3); for (bool use_exec_plan : {false, true}) { for (bool use_threads : {true, false}) { @@ -1965,7 +2311,7 @@ TEST(GroupBy, MinMaxFixedSizeBinary) { } } -TEST(GroupBy, MinOrMax) { +TEST_P(GroupBy, MinOrMax) { auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ [1.0, 1], @@ -2018,7 +2364,7 @@ TEST(GroupBy, MinOrMax) { /*verbose=*/true); } -TEST(GroupBy, MinMaxScalar) { +TEST_P(GroupBy, MinMaxScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({int32(), int64()}, {ArgShape::SCALAR, ArgShape::ARRAY}, @@ -2052,7 +2398,7 @@ TEST(GroupBy, MinMaxScalar) { } } -TEST(GroupBy, AnyAndAll) { +TEST_P(GroupBy, AnyAndAll) { for (bool use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -2086,7 +2432,7 @@ TEST(GroupBy, AnyAndAll) { auto keep_nulls_min_count = std::make_shared(/*skip_nulls=*/false, /*min_count=*/3); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2097,7 +2443,7 @@ TEST(GroupBy, AnyAndAll) { table->GetColumnByName("argument"), table->GetColumnByName("argument"), }, - {table->GetColumnByName("key")}, + {table->GetColumnByName("key")}, {}, { {"hash_any", no_min, "agg_0", "hash_any"}, {"hash_any", min_count, "agg_1", "hash_any"}, @@ -2141,7 +2487,7 @@ TEST(GroupBy, AnyAndAll) { } } -TEST(GroupBy, AnyAllScalar) { +TEST_P(GroupBy, AnyAllScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({boolean(), int64()}, {ArgShape::SCALAR, ArgShape::ARRAY}, @@ -2183,7 +2529,7 @@ TEST(GroupBy, AnyAllScalar) { } } -TEST(GroupBy, CountDistinct) { +TEST_P(GroupBy, CountDistinct) { auto all = std::make_shared(CountOptions::ALL); auto only_valid = std::make_shared(CountOptions::ONLY_VALID); auto only_null = std::make_shared(CountOptions::ONLY_NULL); @@ -2223,7 +2569,7 @@ TEST(GroupBy, CountDistinct) { ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2232,6 +2578,7 @@ TEST(GroupBy, CountDistinct) { { table->GetColumnByName("key"), }, + {}, { {"hash_count_distinct", all, "agg_0", "hash_count_distinct"}, {"hash_count_distinct", only_valid, "agg_1", "hash_count_distinct"}, @@ -2290,7 +2637,7 @@ TEST(GroupBy, CountDistinct) { ASSERT_OK_AND_ASSIGN( aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2299,6 +2646,7 @@ TEST(GroupBy, CountDistinct) { { table->GetColumnByName("key"), }, + {}, { {"hash_count_distinct", all, "agg_0", "hash_count_distinct"}, {"hash_count_distinct", only_valid, "agg_1", "hash_count_distinct"}, @@ -2337,7 +2685,7 @@ TEST(GroupBy, CountDistinct) { ASSERT_OK_AND_ASSIGN( aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2346,6 +2694,7 @@ TEST(GroupBy, CountDistinct) { { table->GetColumnByName("key"), }, + {}, { {"hash_count_distinct", all, "agg_0", "hash_count_distinct"}, {"hash_count_distinct", only_valid, "agg_1", "hash_count_distinct"}, @@ -2370,7 +2719,7 @@ TEST(GroupBy, CountDistinct) { } } -TEST(GroupBy, Distinct) { +TEST_P(GroupBy, Distinct) { auto all = std::make_shared(CountOptions::ALL); auto only_valid = std::make_shared(CountOptions::ONLY_VALID); auto only_null = std::make_shared(CountOptions::ONLY_NULL); @@ -2409,7 +2758,7 @@ TEST(GroupBy, Distinct) { ])"}); ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2418,6 +2767,7 @@ TEST(GroupBy, Distinct) { { table->GetColumnByName("key"), }, + {}, { {"hash_distinct", all, "agg_0", "hash_distinct"}, {"hash_distinct", only_valid, "agg_1", "hash_distinct"}, @@ -2482,7 +2832,7 @@ TEST(GroupBy, Distinct) { ])", }); ASSERT_OK_AND_ASSIGN(aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -2491,6 +2841,7 @@ TEST(GroupBy, Distinct) { { table->GetColumnByName("key"), }, + {}, { {"hash_distinct", all, "agg_0", "hash_distinct"}, {"hash_distinct", only_valid, "agg_1", "hash_distinct"}, @@ -2513,7 +2864,7 @@ TEST(GroupBy, Distinct) { } } -TEST(GroupBy, OneMiscTypes) { +TEST_P(GroupBy, OneMiscTypes) { auto in_schema = schema({ field("floats", float64()), field("nulls", null()), @@ -2632,7 +2983,7 @@ TEST(GroupBy, OneMiscTypes) { } } -TEST(GroupBy, OneNumericTypes) { +TEST_P(GroupBy, OneNumericTypes) { std::vector> types; types.insert(types.end(), NumericTypes().begin(), NumericTypes().end()); types.insert(types.end(), TemporalTypes().begin(), TemporalTypes().end()); @@ -2720,7 +3071,7 @@ TEST(GroupBy, OneNumericTypes) { } } -TEST(GroupBy, OneBinaryTypes) { +TEST_P(GroupBy, OneBinaryTypes) { for (bool use_exec_plan : {true, false}) { for (bool use_threads : {true, false}) { for (const auto& type : BaseBinaryTypes()) { @@ -2771,7 +3122,7 @@ TEST(GroupBy, OneBinaryTypes) { } } -TEST(GroupBy, OneScalar) { +TEST_P(GroupBy, OneScalar) { BatchesWithSchema input; input.batches = { ExecBatchFromJSON({int32(), int64()}, {ArgShape::SCALAR, ArgShape::ARRAY}, @@ -2801,7 +3152,7 @@ TEST(GroupBy, OneScalar) { } } -TEST(GroupBy, ListNumeric) { +TEST_P(GroupBy, ListNumeric) { for (const auto& type : NumericTypes()) { for (auto use_threads : {true, false}) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -2839,13 +3190,14 @@ TEST(GroupBy, ListNumeric) { ])"}); ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), }, { table->GetColumnByName("key"), }, + {}, { {"hash_list", nullptr, "agg_0", "hash_list"}, }, @@ -2910,13 +3262,14 @@ TEST(GroupBy, ListNumeric) { ])"}); ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), }, { table->GetColumnByName("key"), }, + {}, { {"hash_list", nullptr, "agg_0", "hash_list"}, }, @@ -2951,7 +3304,7 @@ TEST(GroupBy, ListNumeric) { } } -TEST(GroupBy, ListBinaryTypes) { +TEST_P(GroupBy, ListBinaryTypes) { for (bool use_threads : {true, false}) { for (const auto& type : BaseBinaryTypes()) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -2979,13 +3332,14 @@ TEST(GroupBy, ListBinaryTypes) { ])"}); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument0"), }, { table->GetColumnByName("key"), }, + {}, { {"hash_list", nullptr, "agg_0", "hash_list"}, }, @@ -3041,13 +3395,14 @@ TEST(GroupBy, ListBinaryTypes) { ])"}); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument0"), }, { table->GetColumnByName("key"), }, + {}, { {"hash_list", nullptr, "agg_0", "hash_list"}, }, @@ -3083,7 +3438,7 @@ TEST(GroupBy, ListBinaryTypes) { } } -TEST(GroupBy, ListMiscTypes) { +TEST_P(GroupBy, ListMiscTypes) { auto in_schema = schema({ field("floats", float64()), field("nulls", null()), @@ -3244,7 +3599,7 @@ TEST(GroupBy, ListMiscTypes) { } } -TEST(GroupBy, CountAndSum) { +TEST_P(GroupBy, CountAndSum) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ [1.0, 1], @@ -3266,7 +3621,7 @@ TEST(GroupBy, CountAndSum) { std::make_shared(/*skip_nulls=*/true, /*min_count=*/3); ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { // NB: passing an argument twice or also using it as a key is legal batch->GetColumnByName("argument"), @@ -3279,6 +3634,7 @@ TEST(GroupBy, CountAndSum) { { batch->GetColumnByName("key"), }, + {}, { {"hash_count", count_opts, "agg_0", "hash_count"}, {"hash_count", count_nulls_opts, "agg_1", "hash_count"}, @@ -3334,6 +3690,7 @@ TEST(GroupBy, StandAloneNullaryCount) { { batch->GetColumnByName("key"), }, + {}, { {"hash_count_all", "hash_count_all"}, })); @@ -3352,7 +3709,7 @@ TEST(GroupBy, StandAloneNullaryCount) { /*verbose=*/true); } -TEST(GroupBy, Product) { +TEST_P(GroupBy, Product) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ [-1.0, 1], @@ -3370,7 +3727,7 @@ TEST(GroupBy, Product) { auto min_count = std::make_shared(/*skip_nulls=*/true, /*min_count=*/3); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { batch->GetColumnByName("argument"), batch->GetColumnByName("key"), @@ -3379,6 +3736,7 @@ TEST(GroupBy, Product) { { batch->GetColumnByName("key"), }, + {}, { {"hash_product", nullptr, "agg_0", "hash_product"}, {"hash_product", nullptr, "agg_1", "hash_product"}, @@ -3408,13 +3766,14 @@ TEST(GroupBy, Product) { ])"); ASSERT_OK_AND_ASSIGN(aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { batch->GetColumnByName("argument"), }, { batch->GetColumnByName("key"), }, + {}, { {"hash_product", nullptr, "agg_0", "hash_product"}, })); @@ -3428,7 +3787,7 @@ TEST(GroupBy, Product) { /*verbose=*/true); } -TEST(GroupBy, SumMeanProductKeepNulls) { +TEST_P(GroupBy, SumMeanProductKeepNulls) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ [-1.0, 1], @@ -3447,7 +3806,7 @@ TEST(GroupBy, SumMeanProductKeepNulls) { auto min_count = std::make_shared(/*skip_nulls=*/false, /*min_count=*/3); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { batch->GetColumnByName("argument"), batch->GetColumnByName("argument"), @@ -3459,6 +3818,7 @@ TEST(GroupBy, SumMeanProductKeepNulls) { { batch->GetColumnByName("key"), }, + {}, { {"hash_sum", keep_nulls, "agg_0", "hash_sum"}, {"hash_sum", min_count, "agg_1", "hash_sum"}, @@ -3487,7 +3847,7 @@ TEST(GroupBy, SumMeanProductKeepNulls) { /*verbose=*/true); } -TEST(GroupBy, SumOnlyStringAndDictKeys) { +TEST_P(GroupBy, SumOnlyStringAndDictKeys) { for (auto key_type : {utf8(), dictionary(int32(), utf8())}) { SCOPED_TRACE("key type: " + key_type->ToString()); @@ -3506,8 +3866,8 @@ TEST(GroupBy, SumOnlyStringAndDictKeys) { ])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({batch->GetColumnByName("argument")}, - {batch->GetColumnByName("key")}, + AlternatorGroupBy({batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}, {}, { {"hash_sum", nullptr, "agg_0", "hash_sum"}, })); @@ -3528,7 +3888,7 @@ TEST(GroupBy, SumOnlyStringAndDictKeys) { } } -TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { +TEST_P(GroupBy, ConcreteCaseWithValidateGroupBy) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", utf8())}), R"([ [1.0, "alfa"], @@ -3552,10 +3912,10 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { for (auto agg : { Aggregate{"hash_sum", nullptr, "agg_0", "hash_sum"}, - Aggregate{"hash_count", non_null, "agg_1", "hash_count"}, - Aggregate{"hash_count", nulls, "agg_2", "hash_count"}, - Aggregate{"hash_min_max", nullptr, "agg_3", "hash_min_max"}, - Aggregate{"hash_min_max", keepna, "agg_4", "hash_min_max"}, + Aggregate{"hash_count", non_null, "agg_0", "hash_count"}, + Aggregate{"hash_count", nulls, "agg_0", "hash_count"}, + Aggregate{"hash_min_max", nullptr, "agg_0", "hash_min_max"}, + Aggregate{"hash_min_max", keepna, "agg_0", "hash_min_max"}, }) { SCOPED_TRACE(agg.function); ValidateGroupBy({agg}, {batch->GetColumnByName("argument")}, @@ -3564,7 +3924,7 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { } // Count nulls/non_nulls from record batch with no nulls -TEST(GroupBy, CountNull) { +TEST_P(GroupBy, CountNull) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", utf8())}), R"([ [1.0, "alfa"], @@ -3579,7 +3939,7 @@ TEST(GroupBy, CountNull) { for (auto agg : { Aggregate{"hash_count", keepna, "agg_0", "hash_count"}, - Aggregate{"hash_count", skipna, "agg_1", "hash_count"}, + Aggregate{"hash_count", skipna, "agg_0", "hash_count"}, }) { SCOPED_TRACE(agg.function); ValidateGroupBy({agg}, {batch->GetColumnByName("argument")}, @@ -3587,7 +3947,7 @@ TEST(GroupBy, CountNull) { } } -TEST(GroupBy, RandomArraySum) { +TEST_P(GroupBy, RandomArraySum) { std::shared_ptr options = std::make_shared(/*skip_nulls=*/true, /*min_count=*/0); for (int64_t length : {1 << 10, 1 << 12, 1 << 15}) { @@ -3610,7 +3970,7 @@ TEST(GroupBy, RandomArraySum) { } } -TEST(GroupBy, WithChunkedArray) { +TEST_P(GroupBy, WithChunkedArray) { auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([{"argument": 1.0, "key": 1}, @@ -3626,7 +3986,7 @@ TEST(GroupBy, WithChunkedArray) { {"argument": null, "key": 3} ])"}); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), table->GetColumnByName("argument"), @@ -3635,6 +3995,7 @@ TEST(GroupBy, WithChunkedArray) { { table->GetColumnByName("key"), }, + {}, { {"hash_count", nullptr, "agg_0", "hash_count"}, {"hash_sum", nullptr, "agg_1", "hash_sum"}, @@ -3660,21 +4021,22 @@ TEST(GroupBy, WithChunkedArray) { /*verbose=*/true); } -TEST(GroupBy, MinMaxWithNewGroupsInChunkedArray) { +TEST_P(GroupBy, MinMaxWithNewGroupsInChunkedArray) { auto table = TableFromJSON( schema({field("argument", int64()), field("key", int64())}), {R"([{"argument": 1, "key": 0}])", R"([{"argument": 0, "key": 1}])"}); ScalarAggregateOptions count_options; ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy( + AlternatorGroupBy( { table->GetColumnByName("argument"), }, { table->GetColumnByName("key"), }, + {}, { - {"hash_min_max", nullptr, "agg_1", "hash_min_max"}, + {"hash_min_max", nullptr, "agg_0", "hash_min_max"}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -3692,7 +4054,7 @@ TEST(GroupBy, MinMaxWithNewGroupsInChunkedArray) { /*verbose=*/true); } -TEST(GroupBy, SmallChunkSizeSumOnly) { +TEST_P(GroupBy, SmallChunkSizeSumOnly) { auto batch = RecordBatchFromJSON( schema({field("argument", float64()), field("key", int64())}), R"([ [1.0, 1], @@ -3707,8 +4069,8 @@ TEST(GroupBy, SmallChunkSizeSumOnly) { [null, 3] ])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({batch->GetColumnByName("argument")}, - {batch->GetColumnByName("key")}, + AlternatorGroupBy({batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}, {}, { {"hash_sum", nullptr, "agg_0", "hash_sum"}, }, @@ -3727,7 +4089,7 @@ TEST(GroupBy, SmallChunkSizeSumOnly) { /*verbose=*/true); } -TEST(GroupBy, CountWithNullType) { +TEST_P(GroupBy, CountWithNullType) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([ [null, 1], @@ -3787,7 +4149,7 @@ TEST(GroupBy, CountWithNullType) { } } -TEST(GroupBy, CountWithNullTypeEmptyTable) { +TEST_P(GroupBy, CountWithNullTypeEmptyTable) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([])"}); @@ -3820,7 +4182,7 @@ TEST(GroupBy, CountWithNullTypeEmptyTable) { } } -TEST(GroupBy, SingleNullTypeKey) { +TEST_P(GroupBy, SingleNullTypeKey) { auto table = TableFromJSON(schema({field("argument", int64()), field("key", null())}), {R"([ [1, null], @@ -3879,7 +4241,7 @@ TEST(GroupBy, SingleNullTypeKey) { } } -TEST(GroupBy, MultipleKeysIncludesNullType) { +TEST_P(GroupBy, MultipleKeysIncludesNullType) { auto table = TableFromJSON(schema({field("argument", float64()), field("key_0", utf8()), field("key_1", null())}), {R"([ @@ -3942,7 +4304,7 @@ TEST(GroupBy, MultipleKeysIncludesNullType) { } } -TEST(GroupBy, SumNullType) { +TEST_P(GroupBy, SumNullType) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([ [null, 1], @@ -4010,7 +4372,7 @@ TEST(GroupBy, SumNullType) { } } -TEST(GroupBy, ProductNullType) { +TEST_P(GroupBy, ProductNullType) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([ [null, 1], @@ -4078,7 +4440,7 @@ TEST(GroupBy, ProductNullType) { } } -TEST(GroupBy, MeanNullType) { +TEST_P(GroupBy, MeanNullType) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([ [null, 1], @@ -4146,7 +4508,7 @@ TEST(GroupBy, MeanNullType) { } } -TEST(GroupBy, NullTypeEmptyTable) { +TEST_P(GroupBy, NullTypeEmptyTable) { auto table = TableFromJSON(schema({field("argument", null()), field("key", int64())}), {R"([])"}); @@ -4187,7 +4549,7 @@ TEST(GroupBy, NullTypeEmptyTable) { } } -TEST(GroupBy, OnlyKeys) { +TEST_P(GroupBy, OnlyKeys) { auto table = TableFromJSON(schema({field("key_0", int64()), field("key_1", utf8())}), {R"([ [1, "a"], @@ -4234,5 +4596,265 @@ TEST(GroupBy, OnlyKeys) { } } } + +INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, + ::testing::Values(GroupByDirectImpl, GroupByWithArrays, + GroupByUsingExecPlanImpl)); + +class SegmentedScalarGroupBy : public GroupBy {}; + +class SegmentedKeyGroupBy : public GroupBy {}; + +void TestSegment(GroupByFunction group_by, const std::shared_ptr
& table, + Datum output, const std::vector& keys, + const std::vector& segment_keys, bool scalar) { + const char* names[] = { + scalar ? "count" : "hash_count", + scalar ? "sum" : "hash_sum", + scalar ? "min_max" : "hash_min_max", + }; + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + group_by( + { + table->GetColumnByName("argument"), + table->GetColumnByName("argument"), + table->GetColumnByName("argument"), + }, + keys, segment_keys, + { + {names[0], nullptr, "agg_0", names[0]}, + {names[1], nullptr, "agg_1", names[1]}, + {names[2], nullptr, "agg_2", names[2]}, + }, + kDefaultUseThreads, kDefaultCtx, /*naive=*/false)); + + AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true); +} + +void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr
& table, + Datum output, const std::vector& segment_keys) { + TestSegment(group_by, table, output, {}, segment_keys, /*scalar=*/true); +} + +void TestSegmentKey(GroupByFunction group_by, const std::shared_ptr
& table, + Datum output, const std::vector& segment_keys) { + TestSegment(group_by, table, output, {table->GetColumnByName("key")}, segment_keys, + /*scalar=*/false); +} + +Result> GetSingleSegmentInputAsChunked() { + auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64()), + field("segment_key", int64())}), + {R"([{"argument": 1.0, "key": 1, "segment_key": 1}, + {"argument": null, "key": 1, "segment_key": 1} + ])", + R"([{"argument": 0.0, "key": 2, "segment_key": 1}, + {"argument": null, "key": 3, "segment_key": 1}, + {"argument": 4.0, "key": null, "segment_key": 1}, + {"argument": 3.25, "key": 1, "segment_key": 1}, + {"argument": 0.125, "key": 2, "segment_key": 1}, + {"argument": -0.25, "key": 2, "segment_key": 1}, + {"argument": 0.75, "key": null, "segment_key": 1}, + {"argument": null, "key": 3, "segment_key": 1} + ])", + R"([{"argument": 1.0, "key": 1, "segment_key": 0}, + {"argument": null, "key": 1, "segment_key": 0} + ])", + R"([{"argument": 0.0, "key": 2, "segment_key": 0}, + {"argument": null, "key": 3, "segment_key": 0}, + {"argument": 4.0, "key": null, "segment_key": 0}, + {"argument": 3.25, "key": 1, "segment_key": 0}, + {"argument": 0.125, "key": 2, "segment_key": 0}, + {"argument": -0.25, "key": 2, "segment_key": 0}, + {"argument": 0.75, "key": null, "segment_key": 0}, + {"argument": null, "key": 3, "segment_key": 0} + ])"}); + return table; +} + +Result> GetSingleSegmentInputAsCombined() { + ARROW_ASSIGN_OR_RAISE(auto table, GetSingleSegmentInputAsChunked()); + return table->CombineChunks(); +} + +Result> GetSingleSegmentScalarOutput() { + return ChunkedArrayFromJSON(struct_({ + field("count", int64()), + field("sum", float64()), + field("min_max", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("key_0", int64()), + }), + {R"([ + [7, 8.875, {"min": -0.25, "max": 4.0}, 1] + ])", + R"([ + [7, 8.875, {"min": -0.25, "max": 4.0}, 0] + ])"}); +} + +Result> GetSingleSegmentKeyOutput() { + return ChunkedArrayFromJSON(struct_({ + field("hash_count", int64()), + field("hash_sum", float64()), + field("hash_min_max", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("key_0", int64()), + field("key_1", int64()), + }), + {R"([ + [2, 4.25, {"min": 1.0, "max": 3.25}, 1, 1], + [3, -0.125, {"min": -0.25, "max": 0.125}, 2, 1], + [0, null, {"min": null, "max": null}, 3, 1], + [2, 4.75, {"min": 0.75, "max": 4.0}, null, 1] + ])", + R"([ + [2, 4.25, {"min": 1.0, "max": 3.25}, 1, 0], + [3, -0.125, {"min": -0.25, "max": 0.125}, 2, 0], + [0, null, {"min": null, "max": null}, 3, 0], + [2, 4.75, {"min": 0.75, "max": 4.0}, null, 0] + ])"}); +} + +void TestSingleSegmentScalar(GroupByFunction group_by, + std::function>()> get_table) { + ASSERT_OK_AND_ASSIGN(auto table, get_table()); + ASSERT_OK_AND_ASSIGN(auto output, GetSingleSegmentScalarOutput()); + TestSegmentScalar(group_by, table, output, {table->GetColumnByName("segment_key")}); +} + +void TestSingleSegmentKey(GroupByFunction group_by, + std::function>()> get_table) { + ASSERT_OK_AND_ASSIGN(auto table, get_table()); + ASSERT_OK_AND_ASSIGN(auto output, GetSingleSegmentKeyOutput()); + TestSegmentKey(group_by, table, output, {table->GetColumnByName("segment_key")}); +} + +TEST_P(SegmentedScalarGroupBy, SingleSegmentScalarChunked) { + TestSingleSegmentScalar(GetParam(), GetSingleSegmentInputAsChunked); +} + +TEST_P(SegmentedScalarGroupBy, SingleSegmentScalarCombined) { + TestSingleSegmentScalar(GetParam(), GetSingleSegmentInputAsCombined); +} + +TEST_P(SegmentedKeyGroupBy, SingleSegmentKeyChunked) { + TestSingleSegmentKey(GetParam(), GetSingleSegmentInputAsChunked); +} + +TEST_P(SegmentedKeyGroupBy, SingleSegmentKeyCombined) { + TestSingleSegmentKey(GetParam(), GetSingleSegmentInputAsCombined); +} + +// extracts one segment of the obtained (single-segment-key) table +Result> GetEmptySegmentInput( + std::function>()> get_table) { + ARROW_ASSIGN_OR_RAISE(auto table, get_table()); + auto sliced = table->Slice(0, 10); + ARROW_ASSIGN_OR_RAISE(auto batch, sliced->CombineChunksToBatch()); + ARROW_ASSIGN_OR_RAISE(auto array, batch->ToStructArray()); + ARROW_ASSIGN_OR_RAISE(auto chunked, ChunkedArray::Make({array}, array->type())); + return Table::FromChunkedStructArray(chunked); +} + +Result> GetEmptySegmentInputAsChunked() { + return GetEmptySegmentInput(GetSingleSegmentInputAsChunked); +} + +Result> GetEmptySegmentInputAsCombined() { + return GetEmptySegmentInput(GetSingleSegmentInputAsCombined); +} + +// extracts the expected output for one segment +Result> GetEmptySegmentKeyOutput() { + ARROW_ASSIGN_OR_RAISE(auto chunked, GetSingleSegmentKeyOutput()); + ARROW_ASSIGN_OR_RAISE(auto table, Table::FromChunkedStructArray(chunked)); + ARROW_ASSIGN_OR_RAISE(auto removed, table->RemoveColumn(table->num_columns() - 1)); + auto sliced = removed->Slice(0, 4); + ARROW_ASSIGN_OR_RAISE(auto batch, sliced->CombineChunksToBatch()); + return batch->ToStructArray(); +} + +void TestEmptySegmentKey(GroupByFunction group_by, + std::function>()> get_table) { + ASSERT_OK_AND_ASSIGN(auto table, get_table()); + ASSERT_OK_AND_ASSIGN(auto output, GetEmptySegmentKeyOutput()); + TestSegmentKey(group_by, table, output, {}); +} + +TEST_P(SegmentedKeyGroupBy, EmptySegmentKeyChunked) { + TestEmptySegmentKey(GetParam(), GetEmptySegmentInputAsChunked); +} + +TEST_P(SegmentedKeyGroupBy, EmptySegmentKeyCombined) { + TestEmptySegmentKey(GetParam(), GetEmptySegmentInputAsCombined); +} + +// adds a named copy of the last (single-segment-key) column to the obtained table +Result> GetMultiSegmentInput( + std::function>()> get_table, + const std::string& add_name) { + ARROW_ASSIGN_OR_RAISE(auto table, get_table()); + int last = table->num_columns() - 1; + auto add_field = field(add_name, table->schema()->field(last)->type()); + return table->AddColumn(table->num_columns(), add_field, table->column(last)); +} + +Result> GetMultiSegmentInputAsChunked( + const std::string& add_name) { + return GetMultiSegmentInput(GetSingleSegmentInputAsChunked, add_name); +} + +Result> GetMultiSegmentInputAsCombined( + const std::string& add_name) { + return GetMultiSegmentInput(GetSingleSegmentInputAsCombined, add_name); +} + +// adds a named copy of the last (single-segment-key) column to the expected output table +Result> GetMultiSegmentKeyOutput( + const std::string& add_name) { + ARROW_ASSIGN_OR_RAISE(auto chunked, GetSingleSegmentKeyOutput()); + ARROW_ASSIGN_OR_RAISE(auto table, Table::FromChunkedStructArray(chunked)); + int last = table->num_columns() - 1; + auto add_field = field(add_name, table->schema()->field(last)->type()); + ARROW_ASSIGN_OR_RAISE(auto added, + table->AddColumn(last + 1, add_field, table->column(last))); + ARROW_ASSIGN_OR_RAISE(auto batch, added->CombineChunksToBatch()); + ARROW_ASSIGN_OR_RAISE(auto array, batch->ToStructArray()); + return ChunkedArray::Make({array->Slice(0, 4), array->Slice(4, 4)}, array->type()); +} + +void TestMultiSegmentKey( + GroupByFunction group_by, + std::function>(const std::string&)> get_table) { + std::string add_name = "segment_key2"; + ASSERT_OK_AND_ASSIGN(auto table, get_table(add_name)); + ASSERT_OK_AND_ASSIGN(auto output, GetMultiSegmentKeyOutput("key_2")); + TestSegmentKey( + group_by, table, output, + {table->GetColumnByName("segment_key"), table->GetColumnByName(add_name)}); +} + +TEST_P(SegmentedKeyGroupBy, MultiSegmentKeyChunked) { + TestMultiSegmentKey(GetParam(), GetMultiSegmentInputAsChunked); +} + +TEST_P(SegmentedKeyGroupBy, MultiSegmentKeyCombined) { + TestMultiSegmentKey(GetParam(), GetMultiSegmentInputAsCombined); +} + +// GroupByDirectImpl and GroupByWithArrays are not included in the testing values because +// they would lead to passing 2 arguments to a single-argument kernel (like `count`) +INSTANTIATE_TEST_SUITE_P(SegmentedScalarGroupBy, SegmentedScalarGroupBy, + ::testing::Values(SegmentedGroupByUsingExecPlan)); + +INSTANTIATE_TEST_SUITE_P(SegmentedKeyGroupBy, SegmentedKeyGroupBy, + ::testing::Values(GroupByDirectImpl, GroupByWithArrays, + SegmentedGroupByUsingExecPlan)); + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index d003137d3e5..d19e90f6e65 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -19,6 +19,9 @@ #include #include +#include + +#include "arrow/array/builder_primitive.h" #include "arrow/compute/exec/key_hash.h" #include "arrow/compute/exec/key_map.h" @@ -30,6 +33,7 @@ #include "arrow/compute/registry.h" #include "arrow/compute/row/compare_internal.h" #include "arrow/type.h" +#include "arrow/type_traits.h" #include "arrow/util/bitmap_ops.h" #include "arrow/util/checked_cast.h" #include "arrow/util/cpu_info.h" @@ -39,12 +43,521 @@ namespace arrow { using internal::checked_cast; +using internal::PrimitiveScalarBase; namespace compute { namespace { -struct GrouperImpl : Grouper { +constexpr uint32_t kNoGroupId = std::numeric_limits::max(); + +using group_id_t = std::remove_const::type; +using GroupIdType = CTypeTraits::ArrowType; +auto group_id_type = std::make_shared(); + +inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) { + DCHECK_GT(data.type->byte_width(), 0); + int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width(); + return data.GetValues(1, absolute_byte_offset); +} + +inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) { + DCHECK_GT(data.type->byte_width(), 0); + int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width(); + return data.GetValues(1, absolute_byte_offset); +} + +template +Status CheckForGetNextSegment(const std::vector& values, int64_t length, + int64_t offset, const std::vector& key_types) { + if (offset < 0 || offset > length) { + return Status::Invalid("invalid grouping segmenter offset: ", offset); + } + if (values.size() != key_types.size()) { + return Status::Invalid("expected batch size ", key_types.size(), " but got ", + values.size()); + } + for (size_t i = 0; i < key_types.size(); i++) { + const auto& value = values[i]; + const auto& key_type = key_types[i]; + if (*value.type() != *key_type.type) { + return Status::Invalid("expected batch value ", i, " of type ", *key_type.type, + " but got ", *value.type()); + } + } + return Status::OK(); +} + +template +enable_if_t::value || std::is_same::value, + Status> +CheckForGetNextSegment(const Batch& batch, int64_t offset, + const std::vector& key_types) { + return CheckForGetNextSegment(batch.values, batch.length, offset, key_types); +} + +struct BaseGroupingSegmenter : public GroupingSegmenter { + explicit BaseGroupingSegmenter(const std::vector& key_types) + : key_types_(key_types) {} + + const std::vector& key_types() const override { return key_types_; } + + std::vector key_types_; +}; + +GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, + bool extends) { + return GroupingSegment{offset, length, offset + length >= batch_length, extends}; +} + +int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, + const uint8_t* array_bytes, int64_t offset, int64_t length) { + int64_t cursor, byte_cursor; + for (cursor = offset, byte_cursor = match_width * cursor; cursor < length; + cursor++, byte_cursor += match_width) { + if (memcmp(match_bytes, array_bytes + byte_cursor, + static_cast(match_width)) != 0) { + break; + } + } + return std::min(cursor, length - offset); +} + +using ExtendFunc = std::function; +constexpr bool kDefaultExtends = true; +constexpr bool kEmptyExtends = true; + +Result GetNextSegmentChunked( + const std::shared_ptr& chunked_array, int64_t offset, + ExtendFunc extend) { + if (offset >= chunked_array->length()) { + return MakeSegment(chunked_array->length(), chunked_array->length(), 0, + kEmptyExtends); + } + int64_t remaining_offset = offset; + const auto& arrays = chunked_array->chunks(); + for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) { + // look up chunk containing offset + int64_t array_length = arrays[i]->length(); + if (remaining_offset < array_length) { + // found - switch to matching + int64_t match_width = arrays[i]->type()->byte_width(); + const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset); + int64_t total_match_length = 0; + for (; i < arrays.size(); i++) { + int64_t array_length = arrays[i]->length(); + if (array_length <= 0) continue; + const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data()); + int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes, + remaining_offset, array_length); + total_match_length += match_length; + remaining_offset = 0; + if (match_length < array_length - remaining_offset) break; + } + bool extends = extend(match_bytes); + return MakeSegment(chunked_array->length(), offset, total_match_length, extends); + } + remaining_offset -= array_length; + } + return Status::Invalid("segmenting invalid chunked array value"); +} + +struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter { + static std::unique_ptr Make() { + return std::make_unique(); + } + + NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {} + + Status Reset() override { return Status::OK(); } + + template + Result GetNextSegmentImpl(const Batch& batch, int64_t offset) { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {})); + return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends); + } + + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + return GetNextSegmentImpl(batch, offset); + } + + Result GetNextSegment(const ExecBatch& batch, + int64_t offset) override { + return GetNextSegmentImpl(batch, offset); + } +}; + +struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { + static Result> Make(TypeHolder key_type) { + return std::make_unique(key_type); + } + + explicit SimpleKeyGroupingSegmenter(TypeHolder key_type) + : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {} + + Status CheckType(const DataType& type) { + if (!is_fixed_width(type)) { + return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type); + } + return Status::OK(); + } + + Status Reset() override { + save_key_data_.resize(0); + return Status::OK(); + } + + bool Extend(const void* data) { + size_t byte_width = static_cast(key_type_.type->byte_width()); + bool extends = save_key_data_.size() != byte_width + ? kDefaultExtends + : 0 == memcmp(save_key_data_.data(), data, byte_width); + save_key_data_.resize(byte_width); + memcpy(save_key_data_.data(), data, byte_width); + return extends; + } + + Result GetNextSegment(const Scalar& scalar, int64_t offset, + int64_t length) { + ARROW_RETURN_NOT_OK(CheckType(*scalar.type)); + if (!scalar.is_valid) { + return Status::Invalid("segmenting an invalid scalar"); + } + auto data = checked_cast(scalar).data(); + bool extends = length > 0 ? Extend(data) : kEmptyExtends; + return MakeSegment(length, 0, length, extends); + } + + Result GetNextSegment(const DataType& array_type, + const uint8_t* array_bytes, int64_t offset, + int64_t length) { + RETURN_NOT_OK(CheckType(array_type)); + int64_t byte_width = array_type.byte_width(); + int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width, + array_bytes, offset, length); + bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends; + return MakeSegment(length, offset, match_length, extends); + } + + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_})); + if (offset == batch.length) { + return MakeSegment(batch.length, offset, 0, kEmptyExtends); + } + const auto& value = batch.values[0]; + if (value.is_scalar()) { + return GetNextSegment(*value.scalar, offset, batch.length); + } + ARROW_DCHECK(value.is_array()); + const auto& array = value.array; + if (array.GetNullCount() > 0) { + return Status::NotImplemented("segmenting a nullable array"); + } + return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length); + } + + Result GetNextSegment(const ExecBatch& batch, + int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_})); + if (offset == batch.length) { + return MakeSegment(batch.length, offset, 0, kEmptyExtends); + } + const auto& value = batch.values[0]; + if (value.is_scalar()) { + return GetNextSegment(*value.scalar(), offset, batch.length); + } + if (value.is_array()) { + auto array = value.array(); + if (array->GetNullCount() > 0) { + return Status::NotImplemented("segmenting a nullable array"); + } + return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length); + } + if (value.is_chunked_array()) { + auto array = value.chunked_array(); + if (array->null_count() > 0) { + return Status::NotImplemented("segmenting a nullable array"); + } + return GetNextSegmentChunked(array, offset, bound_extend_); + } + return Status::Invalid("segmenting unsupported value kind ", value.kind()); + } + + private: + TypeHolder key_type_; + std::vector save_key_data_; + ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); }; +}; + +struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { + static Result> Make( + const std::vector& key_types, ExecContext* ctx) { + ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx)); // check types + return std::make_unique(key_types, ctx); + } + + AnyKeysGroupingSegmenter(const std::vector& key_types, ExecContext* ctx) + : BaseGroupingSegmenter(key_types), + ctx_(ctx), + grouper_(nullptr), + save_group_id_(kNoGroupId) {} + + Status Reset() override { + grouper_ = nullptr; + save_group_id_ = kNoGroupId; + return Status::OK(); + } + + bool Extend(const void* data) { + auto group_id = *static_cast(data); + bool extends = + save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id; + save_group_id_ = group_id; + return extends; + } + + template + Result MapGroupIdAt(const Batch& batch, int64_t offset) { + if (offset < 0 || offset >= batch.length) { + return Status::Invalid("requesting group id out of bounds"); + } + if (!grouper_) return kNoGroupId; + ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset, + /*consume_length=*/1)); + if (!(datum.is_array() || datum.is_chunked_array())) { + return Status::Invalid("accessing unsupported datum kind ", datum.kind()); + } + const std::shared_ptr& data = + datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data(); + ARROW_DCHECK(data->GetNullCount() == 0); + DCHECK_EQ(data->type->id(), GroupIdType::type_id); + DCHECK_EQ(1, data->length); + const group_id_t* values = data->GetValues(1); + return values[0]; + } + + template + Result GetNextSegmentImpl(const Batch& batch, int64_t offset) { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_)); + if (offset == batch.length) { + return MakeSegment(batch.length, offset, 0, kEmptyExtends); + } + // ARROW-18311: make Grouper support Reset() + // so it can be cached instead of recreated below + // + // the group id must be computed prior to resetting the grouper, since it is compared + // to save_group_id_, and after resetting the grouper produces incomparable group ids + ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset)); + ExtendFunc bound_extend = [this, group_id](const void* data) { + bool extends = Extend(&group_id); + save_group_id_ = *static_cast(data); + return extends; + }; + ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_)); // TODO: reset it + ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset)); + if (datum.is_array()) { + const std::shared_ptr& data = datum.array(); + ARROW_DCHECK(data->GetNullCount() == 0); + DCHECK_EQ(data->type->id(), GroupIdType::type_id); + const group_id_t* values = data->GetValues(1); + int64_t cursor; + for (cursor = 1; cursor < data->length; cursor++) { + if (values[0] != values[cursor]) break; + } + int64_t length = std::min(cursor, batch.length - offset); + bool extends = length > 0 ? bound_extend(values) : kEmptyExtends; + return MakeSegment(batch.length, offset, length, extends); + } else if (datum.is_chunked_array()) { + ARROW_ASSIGN_OR_RAISE( + auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend)); + segment.offset += offset; + return segment; + } else { + return Status::Invalid("segmenting unsupported datum kind ", datum.kind()); + } + } + + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + return GetNextSegmentImpl(batch, offset); + } + + Result GetNextSegment(const ExecBatch& batch, + int64_t offset) override { + return GetNextSegmentImpl(batch, offset); + } + + private: + ExecContext* const ctx_; + std::unique_ptr grouper_; + group_id_t save_group_id_; +}; + +Status CheckForConsume(int64_t batch_length, int64_t& consume_offset, + int64_t* consume_length) { + if (consume_offset < 0) { + return Status::Invalid("invalid grouper consume offset: ", consume_offset); + } + if (*consume_length < 0) { + *consume_length = batch_length - consume_offset; + } + return Status::OK(); +} + +} // namespace + +Result> GroupingSegmenter::Make( + const std::vector& key_types, bool nullable_keys, ExecContext* ctx) { + if (key_types.size() == 0) { + return NoKeysGroupingSegmenter::Make(); + } else if (!nullable_keys && key_types.size() == 1) { + const DataType* type = key_types[0].type; + if (type != NULLPTR && is_fixed_width(*type)) { + return SimpleKeyGroupingSegmenter::Make(key_types[0]); + } + } + return AnyKeysGroupingSegmenter::Make(key_types, ctx); +} + +namespace { + +struct BaseGrouper : public Grouper { + int IndexOfChunk(const ExecBatch& batch) { + int i = 0; + for (const auto& value : batch.values) { + if (value.is_chunked_array()) { + return i; + } + ++i; + } + return -1; + } + + bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) { + auto first_chunked_array = batch.values[index_of_chunk].chunked_array(); + if (first_chunked_array < 0) { + // having no chunks is considered consistent + return true; + } + int num_chunks = first_chunked_array->num_chunks(); + int64_t length = first_chunked_array->length(); + for (const auto& value : batch.values) { + if (!value.is_chunked_array()) { + continue; + } + auto curr_chunk = value.chunked_array(); + if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) { + return false; + } + } + if (num_chunks > 0) { + for (int i = 0; i < num_chunks; i++) { + int64_t chunk_length = first_chunked_array->chunk(i)->length(); + for (const auto& value : batch.values) { + if (!value.is_chunked_array()) { + continue; + } + auto curr_chunk = value.chunked_array(); + if (chunk_length != curr_chunk->chunk(i)->length()) { + return false; + } + } + } + } + return true; + } + + using Grouper::Consume; + + Result Consume(const ExecBatch& batch, int64_t consume_offset, + int64_t consume_length) override { + ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); + int index_of_chunk = IndexOfChunk(batch); + if (index_of_chunk < 0) { + return Consume(ExecSpan(batch), consume_offset, consume_length); + } + if (!HasConsistentChunks(batch, index_of_chunk)) { + return Status::Invalid("consuming inconsistent chunks"); + } + auto first_chunked_array = batch.values[index_of_chunk].chunked_array(); + int num_chunks = first_chunked_array->num_chunks(); + ArrayVector chunks; + chunks.reserve(num_chunks); + int64_t length_passed = 0, consume_remain = consume_length; + for (int chunk_idx = 0; chunk_idx < num_chunks && consume_remain > 0; chunk_idx++) { + int64_t chunk_length = first_chunked_array->chunk(chunk_idx)->length(); + int64_t offset = length_passed; + length_passed += chunk_length; + if (length_passed <= consume_offset) continue; + if (offset >= consume_offset + consume_length) break; + std::vector values; + int64_t array_offset = std::max((int64_t)0, consume_offset - offset); + int64_t array_length = std::min(chunk_length - array_offset, consume_remain); + consume_remain -= array_length; + size_t i = 0; + for (const auto& batch_value : batch.values) { + if (batch_value.is_scalar()) { + values.emplace_back(batch_value.scalar().get()); + } else if (batch_value.is_chunked_array()) { + const auto& data = *batch_value.chunked_array()->chunk(chunk_idx)->data(); + DCHECK_LE(array_offset + array_length, data.length); + ArraySpan array_span(data); + array_span.offset = data.offset + array_offset; + array_span.length = array_length; + values.emplace_back(array_span); + } else { + return Status::Invalid("consuming batch value ", i, " of unsupported kind ", + batch_value.kind()); + } + ++i; + } + ARROW_ASSIGN_OR_RAISE(auto consume, Consume(ExecSpan(values, array_length), 0, -1)); + DCHECK(consume.is_array()); + auto chunk = consume.make_array(); + chunks.push_back(chunk); + } + ARROW_ASSIGN_OR_RAISE(auto chunked_array, ChunkedArray::Make(std::move(chunks))); + return Datum(chunked_array); + } +}; + +struct GrouperNoKeysImpl : Grouper { + Result> MakeConstantGroupIdArray(int64_t length, + group_id_t value) { + std::unique_ptr a_builder; + RETURN_NOT_OK(MakeBuilder(default_memory_pool(), group_id_type, &a_builder)); + using GroupIdBuilder = typename TypeTraits::BuilderType; + auto builder = checked_cast(a_builder.get()); + if (length != 0) { + RETURN_NOT_OK(builder->Resize(length)); + } + for (int64_t i = 0; i < length; i++) { + builder->UnsafeAppend(value); + } + std::shared_ptr array; + RETURN_NOT_OK(builder->Finish(&array)); + return std::move(array); + } + Result Consume(const ExecSpan& batch, int64_t consume_offset, + int64_t consume_length) override { + ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(consume_length, 0)); + return Datum(array); + } + Result Consume(const ExecBatch& batch, int64_t consume_offset, + int64_t consume_length) override { + ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(consume_length, 0)); + return Datum(array); + } + Result GetUniques() override { + auto data = ArrayData::Make(uint32(), 1, 0); + auto values = data->GetMutableValues(0); + values[0] = 0; + ExecBatch out({Datum(data)}, 1); + return std::move(out); + } + uint32_t num_groups() const override { return 1; } +}; + +struct GrouperImpl : public BaseGrouper { static Result> Make( const std::vector& key_types, ExecContext* ctx) { auto impl = std::make_unique(); @@ -95,7 +608,15 @@ struct GrouperImpl : Grouper { return std::move(impl); } - Result Consume(const ExecSpan& batch) override { + using BaseGrouper::Consume; + + Result Consume(const ExecSpan& batch, int64_t consume_offset, + int64_t consume_length) override { + ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); + if (consume_offset != 0 || consume_length != batch.length) { + auto batch_slice = batch.ToExecBatch().Slice(consume_offset, consume_length); + return Consume(ExecSpan(batch_slice), 0, -1); + } std::vector offsets_batch(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { encoders_[i]->AddLength(batch[i], batch.length, offsets_batch.data()); @@ -179,11 +700,14 @@ struct GrouperImpl : Grouper { std::vector> encoders_; }; -struct GrouperFastImpl : Grouper { +struct GrouperFastImpl : public BaseGrouper { static constexpr int kBitmapPaddingForSIMD = 64; // bits static constexpr int kPaddingForSIMD = 32; // bytes static bool CanUse(const std::vector& key_types) { + if (key_types.size() == 0) { + return false; + } #if ARROW_LITTLE_ENDIAN for (size_t i = 0; i < key_types.size(); ++i) { if (is_large_binary_like(key_types[i].id())) { @@ -265,7 +789,15 @@ struct GrouperFastImpl : Grouper { ~GrouperFastImpl() { map_.cleanup(); } - Result Consume(const ExecSpan& batch) override { + using BaseGrouper::Consume; + + Result Consume(const ExecSpan& batch, int64_t consume_offset, + int64_t consume_length) override { + ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); + if (consume_offset != 0 || consume_length != batch.length) { + auto batch_slice = batch.ToExecBatch().Slice(consume_offset, consume_length); + return Consume(ExecSpan(batch_slice), 0, -1); + } // ARROW-14027: broadcast scalar arguments for now for (int i = 0; i < batch.num_values(); i++) { if (batch[i].is_scalar()) { diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index ce09adf09b3..55e8cfc6c28 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -30,6 +30,49 @@ namespace arrow { namespace compute { +/// \brief A segment of contiguous rows for grouping +struct ARROW_EXPORT GroupingSegment { + int64_t offset; + int64_t length; + bool is_open; + bool extends; +}; + +inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) { + return segment1.offset == segment2.offset && segment1.length == segment2.length && + segment1.is_open == segment2.is_open && segment1.extends == segment2.extends; +} +inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) { + return !(segment1 == segment2); +} + +/// \brief Computes grouping segments for a batch. Each segment covers rows with identical +/// values in the batch. The values in the batch are often selected as keys from a larger +/// batch. +class ARROW_EXPORT GroupingSegmenter { + public: + virtual ~GroupingSegmenter() = default; + + /// \brief Construct a GroupingSegmenter which receives the specified key types + static Result> Make( + const std::vector& key_types, bool nullable_keys = false, + ExecContext* ctx = default_exec_context()); + + /// \brief Return the key types of this segmenter + virtual const std::vector& key_types() const = 0; + + /// \brief Reset this grouping segmenter + virtual Status Reset() = 0; + + /// \brief Get the next segment for the given batch starting from the given offset + virtual Result GetNextSegment(const ExecSpan& batch, + int64_t offset) = 0; + + /// \brief Get the next segment for the given batch starting from the given offset + virtual Result GetNextSegment(const ExecBatch& batch, + int64_t offset) = 0; +}; + /// Consumes batches of keys and yields batches of the group ids. class ARROW_EXPORT Grouper { public: @@ -39,10 +82,19 @@ class ARROW_EXPORT Grouper { static Result> Make(const std::vector& key_types, ExecContext* ctx = default_exec_context()); - /// Consume a batch of keys, producing the corresponding group ids as an integer array. + /// Consume a batch of keys, producing the corresponding group ids as an integer array, + /// over a slice defined by an offset and length, which defaults to the batch length. + /// Currently only uint32 indices will be produced, eventually the bit width will only + /// be as wide as necessary. + virtual Result Consume(const ExecSpan& batch, int64_t consume_offset = 0, + int64_t consume_length = -1) = 0; + + /// Consume a batch of keys, producing the corresponding group ids as an integer array, + /// over a slice defined by an offset and length, which defaults to the batch length. /// Currently only uint32 indices will be produced, eventually the bit width will only /// be as wide as necessary. - virtual Result Consume(const ExecSpan& batch) = 0; + virtual Result Consume(const ExecBatch& batch, int64_t consume_offset = 0, + int64_t consume_length = -1) = 0; /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; diff --git a/cpp/src/arrow/scalar.h b/cpp/src/arrow/scalar.h index 31dfdcbc84f..d23b33e28f7 100644 --- a/cpp/src/arrow/scalar.h +++ b/cpp/src/arrow/scalar.h @@ -136,6 +136,8 @@ struct ARROW_EXPORT PrimitiveScalarBase : public Scalar { : Scalar(std::move(type), false) {} using Scalar::Scalar; + /// \brief Get a const pointer to the value of this scalar. May be null. + virtual const void* data() const = 0; /// \brief Get a mutable pointer to the value of this scalar. May be null. virtual void* mutable_data() = 0; /// \brief Get an immutable view of the value of this scalar as bytes. @@ -157,6 +159,7 @@ struct ARROW_EXPORT PrimitiveScalar : public PrimitiveScalarBase { ValueType value{}; + const void* data() const override { return &value; } void* mutable_data() override { return &value; } std::string_view view() const override { return std::string_view(reinterpret_cast(&value), sizeof(ValueType)); @@ -241,6 +244,9 @@ struct ARROW_EXPORT BaseBinaryScalar : public internal::PrimitiveScalarBase { std::shared_ptr value; + const void* data() const override { + return value ? reinterpret_cast(value->data()) : NULLPTR; + } void* mutable_data() override { return value ? reinterpret_cast(value->mutable_data()) : NULLPTR; } @@ -434,6 +440,10 @@ struct ARROW_EXPORT DecimalScalar : public internal::PrimitiveScalarBase { DecimalScalar(ValueType value, std::shared_ptr type) : internal::PrimitiveScalarBase(std::move(type), true), value(value) {} + const void* data() const override { + return reinterpret_cast(value.native_endian_bytes()); + } + void* mutable_data() override { return reinterpret_cast(value.mutable_native_endian_bytes()); } @@ -603,6 +613,9 @@ struct ARROW_EXPORT DictionaryScalar : public internal::PrimitiveScalarBase { Result> GetEncodedValue() const; + const void* data() const override { + return internal::checked_cast(*value.index).data(); + } void* mutable_data() override { return internal::checked_cast(*value.index) .mutable_data(); From 70acbe6697206ff378e40d1552c70bf77ee335f1 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Fri, 24 Feb 2023 15:06:24 -0500 Subject: [PATCH 02/27] requested fixes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 119 ++++++-------- cpp/src/arrow/compute/exec/options.h | 11 +- .../compute/kernels/hash_aggregate_test.cc | 24 +-- cpp/src/arrow/compute/row/grouper.cc | 150 +----------------- cpp/src/arrow/compute/row/grouper.h | 32 +++- 5 files changed, 94 insertions(+), 242 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index e368de0be5a..a4639c8deb3 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -20,6 +20,7 @@ #include #include #include +#include #include "arrow/compute/exec.h" #include "arrow/compute/exec/exec_plan.h" @@ -36,6 +37,25 @@ #include "arrow/util/thread_pool.h" #include "arrow/util/tracing_internal.h" +// This file implements both regular and segmented group-by aggregation, which is a +// generalization of ordered aggregation in which the key columns are not required to be +// ordered. +// +// In (regular) group-by aggregation, the input rows are partitioned into groups using a +// set of columns called keys, where in a given group each row has the same values for +// these columns. In segmented group-by aggregation, a second set of columns called +// segment-keys is used to refine the partitioning. However, segment-keys are different in +// that they partition only consecutive rows into a single group. Such a partition of +// consecutive rows is called a segment. For example, consider a column X with values +// [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X] yields +// a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation with +// segment-keys [X] yields [[0], [1], [2]]. +// +// The implementation first segments the input using the segment-keys, then groups by the +// keys. When a segment end is reached while scanning the input, output is pushed and the +// accumulating state is cleared. If no segment-keys are given, then the entire input is +// taken as one segment. + namespace arrow { using internal::checked_cast; @@ -44,46 +64,6 @@ namespace compute { namespace { -/// \brief A gated shared mutex is similar to a shared mutex, in that it allows either -/// multiple shared readers or a unique writer access to the mutex, except that a waiting -/// writer gates future readers by preventing them from reacquiring shared access until it -/// has acquired and released the mutex. This is useful for ensuring a writer is never -/// starved by readers. -struct GatedSharedMutex { - std::mutex gate; - std::shared_mutex mutex; -}; - -/// \brief Acquires unique access to a gatex mutex. This is useful for a unique writer. -class GatedUniqueLock { - public: - // acquires the gate first, to ensure future readers will wait for its release - explicit GatedUniqueLock(GatedSharedMutex& gated_shared_mutex) - : lock_gate_(gated_shared_mutex.gate), lock_mutex_(gated_shared_mutex.mutex) {} - - private: - std::unique_lock lock_gate_; - std::unique_lock lock_mutex_; -}; - -/// \brief Acquires shared access to a gatex mutex. This is useful for a shared reader. -class GatedSharedLock { - struct TouchGate { - explicit TouchGate(GatedSharedMutex& gated_shared_mutex) { - std::unique_lock lock_gate(gated_shared_mutex.gate); - } - }; - - public: - // acquires and immediately releases the gate first, to ensure no writer is waiting - explicit GatedSharedLock(GatedSharedMutex& gated_shared_mutex) - : touch_gate_(gated_shared_mutex), lock_mutex_(gated_shared_mutex.mutex) {} - - private: - TouchGate touch_gate_; - std::shared_lock lock_mutex_; -}; - std::vector ExtendWithGroupIdType(const std::vector& in_types) { std::vector aggr_in_types; aggr_in_types.reserve(in_types.size() + 1); @@ -221,9 +201,10 @@ Status HandleSegments(std::unique_ptr& segmenter, return Status::OK(); } -Status GetScalarFields(std::vector& values, const ExecBatch& input_batch, +Status GetScalarFields(std::vector* values_ptr, const ExecBatch& input_batch, const std::vector& field_ids) { DCHECK_GT(input_batch.length, 0); + std::vector& values = *values_ptr; int64_t row = input_batch.length - 1; values.clear(); values.resize(field_ids.size()); @@ -371,7 +352,6 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { const char* kind_name() const override { return "ScalarAggregateNode"; } Status DoConsume(const ExecSpan& batch, size_t thread_index) { - GatedSharedLock lock(gated_shared_mutex_); for (size_t i = 0; i < kernels_.size(); ++i) { util::tracing::Span span; START_COMPUTE_SPAN(span, aggs_[i].function, @@ -396,12 +376,13 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto scope = TraceInputReceived(batch); DCHECK_EQ(input, inputs_[0]); - auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) { + auto thread_index = plan_->query_context()->GetThreadIndex(); + auto handler = [this, thread_index](const ExecBatch& full_batch, + const GroupingSegment& segment) { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); auto exec_batch = full_batch.Slice(segment.offset, segment.length); - auto batch = ExecSpan(exec_batch); - RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex())); - RETURN_NOT_OK(GetScalarFields(segmenter_values_, exec_batch, segment_field_ids_)); + RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); + RETURN_NOT_OK(GetScalarFields(&segmenter_values_, exec_batch, segment_field_ids_)); if (!segment.is_open) RETURN_NOT_OK(OutputResult()); return Status::OK(); }; @@ -414,6 +395,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { } Status InputFinished(ExecNode* input, int total_batches) override { + auto scope = TraceFinish(); EVENT_ON_CURRENT_SPAN("InputFinished", {{"batches.length", total_batches}}); DCHECK_EQ(input, inputs_[0]); if (input_counter_.SetTotal(total_batches)) { @@ -446,7 +428,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { } private: - Status ReconstructAggregates() { + Status ResetAggregates() { const auto& input_schema = *inputs()[0]->output_schema(); auto exec_ctx = plan()->query_context()->exec_context(); for (size_t i = 0; i < kernels_.size(); ++i) { @@ -463,12 +445,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { return Status::OK(); } - Status OutputResult(bool is_last = false, bool traced = false) { - if (is_last && !traced) { - auto scope = TraceFinish(); - return OutputResult(is_last, /*traced=*/true); - } - GatedUniqueLock lock(gated_shared_mutex_); + Status OutputResult(bool is_last = false) { ExecBatch batch{{}, 1}; batch.values.resize(kernels_.size() + segment_field_ids_.size()); @@ -491,7 +468,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { if (is_last) { ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); } else { - ARROW_RETURN_NOT_OK(ReconstructAggregates()); + ARROW_RETURN_NOT_OK(ResetAggregates()); } return Status::OK(); } @@ -507,8 +484,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { std::vector>> states_; AtomicCounter input_counter_; - int64_t total_output_batches_ = 0; - GatedSharedMutex gated_shared_mutex_; + int total_output_batches_ = 0; }; class GroupByNode : public ExecNode, public TracedNode { @@ -570,6 +546,16 @@ class GroupByNode : public ExecNode, public TracedNode { segment_key_field_ids[i] = match[0]; } + // Check key fields and segment key fields are disjoint + std::unordered_set key_field_id_set(key_field_ids.begin(), key_field_ids.end()); + for (const auto& segment_key_field_id : segment_key_field_ids) { + if (key_field_id_set.find(segment_key_field_id) != key_field_id_set.end()) { + return Status::Invalid("Group-by aggregation with field '", + input_schema->field(segment_key_field_id)->name(), + "' as both key and segment key"); + } + } + // Find input field indices for aggregates std::vector> agg_src_fieldsets(aggs.size()); for (size_t i = 0; i < aggs.size(); ++i) { @@ -635,7 +621,7 @@ class GroupByNode : public ExecNode, public TracedNode { std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels)); } - Status ReconstructAggregates() { + Status ResetAggregates() { auto ctx = plan()->query_context()->exec_context(); ARROW_ASSIGN_OR_RAISE(agg_kernels_, GetKernels(ctx, aggs_, agg_src_types_)); @@ -649,7 +635,6 @@ class GroupByNode : public ExecNode, public TracedNode { const char* kind_name() const override { return "GroupByNode"; } Status Consume(ExecSpan batch) { - GatedSharedLock lock(gated_shared_mutex_); size_t thread_index = plan_->query_context()->GetThreadIndex(); if (thread_index >= local_states_.size()) { return Status::IndexError("thread index ", thread_index, " is out of range [0, ", @@ -773,12 +758,7 @@ class GroupByNode : public ExecNode, public TracedNode { return output_->InputReceived(this, out_data_.Slice(batch_size * n, batch_size)); } - Status OutputResult(bool is_last = false, bool traced = false) { - if (is_last && !traced) { - auto scope = TraceFinish(); - return OutputResult(is_last, /*traced=*/true); - } - GatedUniqueLock lock(gated_shared_mutex_); + Status OutputResult(bool is_last = false) { // To simplify merging, ensure that the first grouper is nonempty for (size_t i = 0; i < local_states_.size(); i++) { if (local_states_[i].grouper) { @@ -793,15 +773,14 @@ class GroupByNode : public ExecNode, public TracedNode { int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size()); total_output_batches_ += num_output_batches; if (is_last) { - ARROW_RETURN_NOT_OK( - output_->InputFinished(this, static_cast(total_output_batches_))); + ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_, num_output_batches)); } else { for (int64_t i = 0; i < num_output_batches; i++) { ARROW_RETURN_NOT_OK(OutputNthBatch(i)); } - ARROW_RETURN_NOT_OK(ReconstructAggregates()); + ARROW_RETURN_NOT_OK(ResetAggregates()); } return Status::OK(); } @@ -817,7 +796,7 @@ class GroupByNode : public ExecNode, public TracedNode { auto batch = ExecSpan(exec_batch); RETURN_NOT_OK(Consume(batch)); RETURN_NOT_OK( - GetScalarFields(segmenter_values_, exec_batch, segment_key_field_ids_)); + GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_)); if (!segment.is_open) RETURN_NOT_OK(OutputResult()); return Status::OK(); }; @@ -831,6 +810,7 @@ class GroupByNode : public ExecNode, public TracedNode { } Status InputFinished(ExecNode* input, int total_batches) override { + auto scope = TraceFinish(); DCHECK_EQ(input, inputs_[0]); if (input_counter_.SetTotal(total_batches)) { @@ -936,11 +916,10 @@ class GroupByNode : public ExecNode, public TracedNode { std::vector agg_kernels_; AtomicCounter input_counter_; - int64_t total_output_batches_ = 0; + int total_output_batches_ = 0; std::vector local_states_; ExecBatch out_data_; - GatedSharedMutex gated_shared_mutex_; }; } // namespace diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index 4578ab9332d..2c16c3a4d14 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -210,6 +210,13 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is /// expected to be a HashAggregate function. If the keys attribute is an empty vector, /// then each aggregate is assumed to be a ScalarAggregate function. +/// +/// If the segment_keys attribute is a non-empty vector, then segmented aggregation, as +/// described above, applies. +/// +/// The keys and segment_keys vectors must be disjoint. +/// +/// See also doc in `aggregate_node.cc` class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { public: explicit AggregateNodeOptions(std::vector aggregates, @@ -221,9 +228,9 @@ class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { // aggregations which will be applied to the targetted fields std::vector aggregates; - // keys by which aggregations will be grouped + // keys by which aggregations will be grouped (optional) std::vector keys; - // keys by which aggregations will be segmented + // keys by which aggregations will be segmented (optional) std::vector segment_keys; }; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 86e40096887..62fd66d1c33 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -166,7 +166,7 @@ Result MakeGroupByOutput(const std::vector& output_batches, bool need_sort = !naive; for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) { - if (output_schema->field(i)->type()->id() == Type::DICTIONARY) { + if (output_schema->field(static_cast(i))->type()->id() == Type::DICTIONARY) { need_sort = false; } } @@ -341,8 +341,8 @@ Result RunGroupBy(const std::vector& arguments, key_names[i] = std::move(name); } base += keys.size(); - size_t j = segmented ? keys.size() : keys.size(); - std::string prefix(segmented ? "key_" : "key_"); + size_t j = keys.size(); + std::string prefix("key_"); for (size_t i = 0; i < segment_keys.size(); ++i) { auto name = prefix + std::to_string(j++); scan_fields[base + i] = field(name, segment_keys[i].type()); @@ -574,20 +574,6 @@ auto batch_to_span = [](const ExecBatch& batch) -> Result { return ExecSpan(batch); }; -auto batch_make_chunked = [](const ExecBatch& batch) -> Result { - int64_t length = batch.length; - DCHECK_GT(length, 1); - std::vector values; - for (auto value : batch.values) { - DCHECK(value.is_array()); - auto array = value.make_array(); - ARROW_ASSIGN_OR_RAISE( - auto chunked, ChunkedArray::Make({array->Slice(0, 1), array->Slice(1, length)})); - values.emplace_back(chunked); - } - return ExecBatch(values, length); -}; - } // namespace TEST(GroupingSegmenter, Basics) { @@ -598,10 +584,6 @@ TEST(GroupingSegmenter, SpanBasics) { test_grouping_segmenter_basics(batch_identity, batch_to_span); } -TEST(GroupingSegmenter, ChunkedBasics) { - test_grouping_segmenter_basics(batch_make_chunked, batch_identity); -} - TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(); } struct TestGrouper { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index d19e90f6e65..8b36178b878 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -127,41 +127,6 @@ using ExtendFunc = std::function; constexpr bool kDefaultExtends = true; constexpr bool kEmptyExtends = true; -Result GetNextSegmentChunked( - const std::shared_ptr& chunked_array, int64_t offset, - ExtendFunc extend) { - if (offset >= chunked_array->length()) { - return MakeSegment(chunked_array->length(), chunked_array->length(), 0, - kEmptyExtends); - } - int64_t remaining_offset = offset; - const auto& arrays = chunked_array->chunks(); - for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) { - // look up chunk containing offset - int64_t array_length = arrays[i]->length(); - if (remaining_offset < array_length) { - // found - switch to matching - int64_t match_width = arrays[i]->type()->byte_width(); - const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset); - int64_t total_match_length = 0; - for (; i < arrays.size(); i++) { - int64_t array_length = arrays[i]->length(); - if (array_length <= 0) continue; - const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data()); - int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes, - remaining_offset, array_length); - total_match_length += match_length; - remaining_offset = 0; - if (match_length < array_length - remaining_offset) break; - } - bool extends = extend(match_bytes); - return MakeSegment(chunked_array->length(), offset, total_match_length, extends); - } - remaining_offset -= array_length; - } - return Status::Invalid("segmenting invalid chunked array value"); -} - struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter { static std::unique_ptr Make() { return std::make_unique(); @@ -207,6 +172,8 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { return Status::OK(); } + // Checks whether the given grouping data extends the current segment, i.e., is equal to + // previously seen grouping data, which is updated with each invocation. bool Extend(const void* data) { size_t byte_width = static_cast(key_type_.type->byte_width()); bool extends = save_key_data_.size() != byte_width @@ -225,7 +192,7 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { } auto data = checked_cast(scalar).data(); bool extends = length > 0 ? Extend(data) : kEmptyExtends; - return MakeSegment(length, 0, length, extends); + return MakeSegment(length, offset, length, extends); } Result GetNextSegment(const DataType& array_type, @@ -273,20 +240,12 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { } return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length); } - if (value.is_chunked_array()) { - auto array = value.chunked_array(); - if (array->null_count() > 0) { - return Status::NotImplemented("segmenting a nullable array"); - } - return GetNextSegmentChunked(array, offset, bound_extend_); - } return Status::Invalid("segmenting unsupported value kind ", value.kind()); } private: TypeHolder key_type_; std::vector save_key_data_; - ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); }; }; struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { @@ -318,13 +277,10 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { template Result MapGroupIdAt(const Batch& batch, int64_t offset) { - if (offset < 0 || offset >= batch.length) { - return Status::Invalid("requesting group id out of bounds"); - } if (!grouper_) return kNoGroupId; ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset, /*consume_length=*/1)); - if (!(datum.is_array() || datum.is_chunked_array())) { + if (!datum.is_array()) { return Status::Invalid("accessing unsupported datum kind ", datum.kind()); } const std::shared_ptr& data = @@ -367,11 +323,6 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { int64_t length = std::min(cursor, batch.length - offset); bool extends = length > 0 ? bound_extend(values) : kEmptyExtends; return MakeSegment(batch.length, offset, length, extends); - } else if (datum.is_chunked_array()) { - ARROW_ASSIGN_OR_RAISE( - auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend)); - segment.offset += offset; - return segment; } else { return Status::Invalid("segmenting unsupported datum kind ", datum.kind()); } @@ -421,102 +372,11 @@ Result> GroupingSegmenter::Make( namespace { struct BaseGrouper : public Grouper { - int IndexOfChunk(const ExecBatch& batch) { - int i = 0; - for (const auto& value : batch.values) { - if (value.is_chunked_array()) { - return i; - } - ++i; - } - return -1; - } - - bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) { - auto first_chunked_array = batch.values[index_of_chunk].chunked_array(); - if (first_chunked_array < 0) { - // having no chunks is considered consistent - return true; - } - int num_chunks = first_chunked_array->num_chunks(); - int64_t length = first_chunked_array->length(); - for (const auto& value : batch.values) { - if (!value.is_chunked_array()) { - continue; - } - auto curr_chunk = value.chunked_array(); - if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) { - return false; - } - } - if (num_chunks > 0) { - for (int i = 0; i < num_chunks; i++) { - int64_t chunk_length = first_chunked_array->chunk(i)->length(); - for (const auto& value : batch.values) { - if (!value.is_chunked_array()) { - continue; - } - auto curr_chunk = value.chunked_array(); - if (chunk_length != curr_chunk->chunk(i)->length()) { - return false; - } - } - } - } - return true; - } - using Grouper::Consume; Result Consume(const ExecBatch& batch, int64_t consume_offset, int64_t consume_length) override { - ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); - int index_of_chunk = IndexOfChunk(batch); - if (index_of_chunk < 0) { - return Consume(ExecSpan(batch), consume_offset, consume_length); - } - if (!HasConsistentChunks(batch, index_of_chunk)) { - return Status::Invalid("consuming inconsistent chunks"); - } - auto first_chunked_array = batch.values[index_of_chunk].chunked_array(); - int num_chunks = first_chunked_array->num_chunks(); - ArrayVector chunks; - chunks.reserve(num_chunks); - int64_t length_passed = 0, consume_remain = consume_length; - for (int chunk_idx = 0; chunk_idx < num_chunks && consume_remain > 0; chunk_idx++) { - int64_t chunk_length = first_chunked_array->chunk(chunk_idx)->length(); - int64_t offset = length_passed; - length_passed += chunk_length; - if (length_passed <= consume_offset) continue; - if (offset >= consume_offset + consume_length) break; - std::vector values; - int64_t array_offset = std::max((int64_t)0, consume_offset - offset); - int64_t array_length = std::min(chunk_length - array_offset, consume_remain); - consume_remain -= array_length; - size_t i = 0; - for (const auto& batch_value : batch.values) { - if (batch_value.is_scalar()) { - values.emplace_back(batch_value.scalar().get()); - } else if (batch_value.is_chunked_array()) { - const auto& data = *batch_value.chunked_array()->chunk(chunk_idx)->data(); - DCHECK_LE(array_offset + array_length, data.length); - ArraySpan array_span(data); - array_span.offset = data.offset + array_offset; - array_span.length = array_length; - values.emplace_back(array_span); - } else { - return Status::Invalid("consuming batch value ", i, " of unsupported kind ", - batch_value.kind()); - } - ++i; - } - ARROW_ASSIGN_OR_RAISE(auto consume, Consume(ExecSpan(values, array_length), 0, -1)); - DCHECK(consume.is_array()); - auto chunk = consume.make_array(); - chunks.push_back(chunk); - } - ARROW_ASSIGN_OR_RAISE(auto chunked_array, ChunkedArray::Make(std::move(chunks))); - return Datum(chunked_array); + return Consume(ExecSpan(batch), consume_offset, consume_length); } }; diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 55e8cfc6c28..c87dca8f6c6 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -32,9 +32,13 @@ namespace compute { /// \brief A segment of contiguous rows for grouping struct ARROW_EXPORT GroupingSegment { + /// \brief the offset into the batch where the segment starts int64_t offset; + /// \brief the length of the segment int64_t length; + /// \brief whether the segment may be extended by a next one bool is_open; + /// \brief whether the segment extends a preceeding one bool extends; }; @@ -46,14 +50,34 @@ inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& s return !(segment1 == segment2); } -/// \brief Computes grouping segments for a batch. Each segment covers rows with identical -/// values in the batch. The values in the batch are often selected as keys from a larger -/// batch. +/// \brief a helper class to divide a batch into segments of equal values +/// +/// For example, given a batch with two rows: +/// +/// A A +/// A A +/// A B +/// A B +/// A A +/// +/// Then the batch could be divided into 3 segments. The first would be rows 0 & 1, +/// the second would be rows 2 & 3, and the third would be row 4. +/// +/// Further, a segmenter keeps track of the last value seen. This allows it to calculate +/// segments which span batches. In our above example the last batch we emit would set +/// the "open" flag, which indicates whether the segment may extend into the next batch. +/// +/// If the next call to the segmenter starts with `A A` then that segment would set the +/// "extends" flag, which indicates whether the segment continues the last open batch. class ARROW_EXPORT GroupingSegmenter { public: virtual ~GroupingSegmenter() = default; - /// \brief Construct a GroupingSegmenter which receives the specified key types + /// \brief Construct a GroupingSegmenter which segments on the specified key types + /// + /// \param[in] key_types the specified key types + /// \param[in] nullable_keys whether values of the specified keys may be null + /// \param[in] ctx the execution context to use static Result> Make( const std::vector& key_types, bool nullable_keys = false, ExecContext* ctx = default_exec_context()); From 78af85e5381c826ed2f1e6c9965a0b754cf61928 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Fri, 24 Feb 2023 15:32:17 -0500 Subject: [PATCH 03/27] fix integer conversion --- cpp/src/arrow/compute/exec/aggregate_node.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index a4639c8deb3..f105d58b884 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -771,7 +771,7 @@ class GroupByNode : public ExecNode, public TracedNode { ARROW_ASSIGN_OR_RAISE(out_data_, Finalize()); int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size()); - total_output_batches_ += num_output_batches; + total_output_batches_ += static_cast(num_output_batches); if (is_last) { ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_, From 9627ddfc810a035d9028e341558f7192499d98a3 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sat, 25 Feb 2023 04:02:14 -0500 Subject: [PATCH 04/27] simplify --- cpp/src/arrow/compute/exec/aggregate_node.cc | 23 ++++--- .../compute/kernels/hash_aggregate_test.cc | 65 +++++++------------ cpp/src/arrow/compute/row/grouper.cc | 49 ++------------ cpp/src/arrow/compute/row/grouper.h | 4 -- 4 files changed, 42 insertions(+), 99 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index f105d58b884..08ef1c25367 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -191,10 +191,11 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& batch, const std::vector& ids, const BatchHandler& handle_batch) { int64_t offset = 0; - ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids)); + ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids)); + ExecSpan segment_batch(segment_exec_batch); while (true) { ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset)); - if (segment.offset >= segment_batch.length) break; + if (segment.offset >= segment_batch.length) break; // condition of no-next-segment ARROW_RETURN_NOT_OK(handle_batch(batch, segment)); offset = segment.offset + segment.length; } @@ -247,7 +248,16 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { target_fieldsets_(std::move(target_fieldsets)), aggs_(std::move(aggs)), kernels_(std::move(kernels)), - states_(std::move(states)) {} + states_(std::move(states)) { + const auto& input_schema = *this->inputs()[0]->output_schema(); + for (size_t i = 0; i < kernels_.size(); ++i) { + std::vector in_types; + for (const auto& target : target_fieldsets_[i]) { + in_types.emplace_back(input_schema.field(target)->type().get()); + } + in_typesets_.push_back(std::move(in_types)); + } + } static Result Make(ExecPlan* plan, std::vector inputs, const ExecNodeOptions& options) { @@ -429,13 +439,9 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { private: Status ResetAggregates() { - const auto& input_schema = *inputs()[0]->output_schema(); auto exec_ctx = plan()->query_context()->exec_context(); for (size_t i = 0; i < kernels_.size(); ++i) { - std::vector in_types; - for (const auto& target : target_fieldsets_[i]) { - in_types.emplace_back(input_schema.field(target)->type().get()); - } + const std::vector& in_types = in_typesets_[i]; states_[i].resize(plan()->query_context()->max_concurrency()); KernelContext kernel_ctx{exec_ctx}; RETURN_NOT_OK(Kernel::InitAll( @@ -481,6 +487,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { const std::vector aggs_; const std::vector kernels_; + std::vector> in_typesets_; std::vector>> states_; AtomicCounter input_counter_; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 62fd66d1c33..18d067cc65a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -480,8 +480,7 @@ void TestGroupClassSupportedKeys() { ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})})); } -template -void TestSegments(std::unique_ptr& segmenter, const Batch& batch, +void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, std::vector expected_segments) { int64_t offset = 0; for (auto expected_segment : expected_segments) { @@ -497,70 +496,66 @@ TEST(GroupingSegmenter, SupportedKeys) { TestGroupClassSupportedKeys(); } -namespace { - -template -void test_grouping_segmenter_basics(SetupBatch setup, ConvertBatch convert) { +TEST(GroupingSegmenter, Basics) { std::vector bad_types2 = {int32(), float32()}; std::vector types2 = {int32(), int32()}; std::vector bad_types1 = {float32()}; std::vector types1 = {int32()}; std::vector types0 = {}; - ASSERT_OK_AND_ASSIGN(auto batch2, - setup(ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]"))); - ASSERT_OK_AND_ASSIGN(auto batch1, setup(ExecBatchFromJSON(types1, "[[1], [1], [2]]"))); + auto batch2 = ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]"); + auto batch1 = ExecBatchFromJSON(types1, "[[1], [1], [2]]"); ExecBatch batch0({}, 3); { SCOPED_TRACE("offset"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); - ASSERT_OK_AND_ASSIGN(auto converted0, convert(batch0)); + ExecSpan span0(batch0); for (int64_t offset : {-1, 4}) { EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("invalid grouping segmenter offset"), - segmenter->GetNextSegment(converted0, offset)); + segmenter->GetNextSegment(span0, offset)); } } { SCOPED_TRACE("types0 segmenting of batch2"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); - ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "), - segmenter->GetNextSegment(converted2, 0)); - ASSERT_OK_AND_ASSIGN(auto converted0, convert(batch0)); - TestSegments(segmenter, converted0, {{0, 3, true, true}, {3, 0, true, true}}); + segmenter->GetNextSegment(span2, 0)); + ExecSpan span0(batch0); + TestSegments(segmenter, span0, {{0, 3, true, true}, {3, 0, true, true}}); } { SCOPED_TRACE("bad_types1 segmenting of batch1"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types1)); - ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); + ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "), - segmenter->GetNextSegment(converted1, 0)); + segmenter->GetNextSegment(span1, 0)); } { SCOPED_TRACE("types1 segmenting of batch2"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types1)); - ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "), - segmenter->GetNextSegment(converted2, 0)); - ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); - TestSegments(segmenter, converted1, + segmenter->GetNextSegment(span2, 0)); + ExecSpan span1(batch1); + TestSegments(segmenter, span1, {{0, 2, false, true}, {2, 1, true, false}, {3, 0, true, true}}); } { SCOPED_TRACE("bad_types2 segmenting of batch2"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types2)); - ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); + ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "), - segmenter->GetNextSegment(converted2, 0)); + segmenter->GetNextSegment(span2, 0)); } { SCOPED_TRACE("types2 segmenting of batch1"); ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types2)); - ASSERT_OK_AND_ASSIGN(auto converted1, convert(batch1)); + ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "), - segmenter->GetNextSegment(converted1, 0)); - ASSERT_OK_AND_ASSIGN(auto converted2, convert(batch2)); - TestSegments(segmenter, converted2, + segmenter->GetNextSegment(span1, 0)); + ExecSpan span2(batch2); + TestSegments(segmenter, span2, {{0, 1, false, true}, {1, 1, false, false}, {2, 1, true, false}, @@ -568,22 +563,6 @@ void test_grouping_segmenter_basics(SetupBatch setup, ConvertBatch convert) { } } -auto batch_identity = [](const ExecBatch& batch) -> Result { return batch; }; - -auto batch_to_span = [](const ExecBatch& batch) -> Result { - return ExecSpan(batch); -}; - -} // namespace - -TEST(GroupingSegmenter, Basics) { - test_grouping_segmenter_basics(batch_identity, batch_identity); -} - -TEST(GroupingSegmenter, SpanBasics) { - test_grouping_segmenter_basics(batch_identity, batch_to_span); -} - TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(); } struct TestGrouper { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 8b36178b878..23cfd5d1b50 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -136,20 +136,10 @@ struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter { Status Reset() override { return Status::OK(); } - template - Result GetNextSegmentImpl(const Batch& batch, int64_t offset) { + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {})); return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends); } - - Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { - return GetNextSegmentImpl(batch, offset); - } - - Result GetNextSegment(const ExecBatch& batch, - int64_t offset) override { - return GetNextSegmentImpl(batch, offset); - } }; struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { @@ -223,26 +213,6 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length); } - Result GetNextSegment(const ExecBatch& batch, - int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_})); - if (offset == batch.length) { - return MakeSegment(batch.length, offset, 0, kEmptyExtends); - } - const auto& value = batch.values[0]; - if (value.is_scalar()) { - return GetNextSegment(*value.scalar(), offset, batch.length); - } - if (value.is_array()) { - auto array = value.array(); - if (array->GetNullCount() > 0) { - return Status::NotImplemented("segmenting a nullable array"); - } - return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length); - } - return Status::Invalid("segmenting unsupported value kind ", value.kind()); - } - private: TypeHolder key_type_; std::vector save_key_data_; @@ -275,6 +245,8 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { return extends; } + // Runs the grouper on a single row. This is used to determine the group id of the + // first row of a new segment to see if it extends the previous segment. template Result MapGroupIdAt(const Batch& batch, int64_t offset) { if (!grouper_) return kNoGroupId; @@ -283,8 +255,7 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { if (!datum.is_array()) { return Status::Invalid("accessing unsupported datum kind ", datum.kind()); } - const std::shared_ptr& data = - datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data(); + const std::shared_ptr& data = datum.array(); ARROW_DCHECK(data->GetNullCount() == 0); DCHECK_EQ(data->type->id(), GroupIdType::type_id); DCHECK_EQ(1, data->length); @@ -292,8 +263,7 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { return values[0]; } - template - Result GetNextSegmentImpl(const Batch& batch, int64_t offset) { + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_)); if (offset == batch.length) { return MakeSegment(batch.length, offset, 0, kEmptyExtends); @@ -328,15 +298,6 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { } } - Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { - return GetNextSegmentImpl(batch, offset); - } - - Result GetNextSegment(const ExecBatch& batch, - int64_t offset) override { - return GetNextSegmentImpl(batch, offset); - } - private: ExecContext* const ctx_; std::unique_ptr grouper_; diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index c87dca8f6c6..62a4d4258db 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -91,10 +91,6 @@ class ARROW_EXPORT GroupingSegmenter { /// \brief Get the next segment for the given batch starting from the given offset virtual Result GetNextSegment(const ExecSpan& batch, int64_t offset) = 0; - - /// \brief Get the next segment for the given batch starting from the given offset - virtual Result GetNextSegment(const ExecBatch& batch, - int64_t offset) = 0; }; /// Consumes batches of keys and yields batches of the group ids. From 624059650813e5aa2f5a9a11a7b0b92a9a999287 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sat, 25 Feb 2023 04:50:01 -0500 Subject: [PATCH 05/27] remove unused --- cpp/src/arrow/compute/row/grouper.cc | 6 ------ 1 file changed, 6 deletions(-) diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 23cfd5d1b50..0f11c71ccf5 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -55,12 +55,6 @@ using group_id_t = std::remove_const::type; using GroupIdType = CTypeTraits::ArrowType; auto group_id_type = std::make_shared(); -inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) { - DCHECK_GT(data.type->byte_width(), 0); - int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width(); - return data.GetValues(1, absolute_byte_offset); -} - inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) { DCHECK_GT(data.type->byte_width(), 0); int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width(); From 01a946b3cfe955cbc6267895a6163a68b8638b01 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Wed, 1 Mar 2023 16:10:55 -0500 Subject: [PATCH 06/27] requested fixes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 7 ++----- cpp/src/arrow/compute/exec/options.h | 2 +- cpp/src/arrow/compute/row/grouper.cc | 4 ++-- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 08ef1c25367..e2400be0837 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -16,7 +16,6 @@ // under the License. #include -#include #include #include #include @@ -461,7 +460,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { {{"function.name", aggs_[i].function}, {"function.options", aggs_[i].options ? aggs_[i].options->ToString() : ""}, - {"function.kind", std::string(kind_name()) + "::Output"}}); + {"function.kind", std::string(kind_name()) + "::Finalize"}}); KernelContext ctx{plan()->query_context()->exec_context()}; ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll( kernels_[i], &ctx, std::move(states_[i]))); @@ -473,10 +472,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { total_output_batches_++; if (is_last) { ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); - } else { - ARROW_RETURN_NOT_OK(ResetAggregates()); } - return Status::OK(); + return ResetAggregates(); } std::unique_ptr segmenter_; diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index 2c16c3a4d14..2d46e9fd86a 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -204,7 +204,7 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { /// However segment-keys are also used for determining grouping segments, which should be /// large, and allow streaming a partial aggregation result after processing each segment. /// One common use-case for segment-keys is ordered aggregation, in which the segment-key -/// attribute specifies a column with non-decreasing values or a lexigographically-ordered +/// attribute specifies a column with non-decreasing values or a lexicographically-ordered /// set of such columns. /// /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 0f11c71ccf5..d604ee1f4ed 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -53,7 +53,7 @@ constexpr uint32_t kNoGroupId = std::numeric_limits::max(); using group_id_t = std::remove_const::type; using GroupIdType = CTypeTraits::ArrowType; -auto group_id_type = std::make_shared(); +auto g_group_id_type = std::make_shared(); inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) { DCHECK_GT(data.type->byte_width(), 0); @@ -339,7 +339,7 @@ struct GrouperNoKeysImpl : Grouper { Result> MakeConstantGroupIdArray(int64_t length, group_id_t value) { std::unique_ptr a_builder; - RETURN_NOT_OK(MakeBuilder(default_memory_pool(), group_id_type, &a_builder)); + RETURN_NOT_OK(MakeBuilder(default_memory_pool(), g_group_id_type, &a_builder)); using GroupIdBuilder = typename TypeTraits::BuilderType; auto builder = checked_cast(a_builder.get()); if (length != 0) { From 54a5809fdbcb514043d1ca14447eaa2c4b9052bb Mon Sep 17 00:00:00 2001 From: Li Jin Date: Thu, 2 Mar 2023 17:06:11 -0500 Subject: [PATCH 07/27] Simply some segment code; add documentation; some refactor/renames --- cpp/src/arrow/compute/exec/aggregate_node.cc | 94 ++++++------ .../compute/kernels/hash_aggregate_test.cc | 36 +++-- cpp/src/arrow/compute/row/grouper.cc | 134 +++++++++--------- cpp/src/arrow/compute/row/grouper.h | 41 +++--- 4 files changed, 159 insertions(+), 146 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index e2400be0837..ee5c9436fc0 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -185,15 +185,19 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema, *ss << ']'; } +// Handle the input batch +// If a segment is closed by this batch, then we output the aggregation for the segment +// If a segment is not closed by this batch, then we add the batch to the segment template -Status HandleSegments(std::unique_ptr& segmenter, - const ExecBatch& batch, const std::vector& ids, - const BatchHandler& handle_batch) { +Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& batch, + const std::vector& ids, const BatchHandler& handle_batch) { int64_t offset = 0; ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids)); ExecSpan segment_batch(segment_exec_batch); + while (true) { - ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset)); + ARROW_ASSIGN_OR_RAISE(compute::SegmentPiece segment, + segmenter->GetNextSegmentPiece(segment_batch, offset)); if (segment.offset >= segment_batch.length) break; // condition of no-next-segment ARROW_RETURN_NOT_OK(handle_batch(batch, segment)); offset = segment.offset + segment.length; @@ -233,11 +237,12 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { public: ScalarAggregateNode(ExecPlan* plan, std::vector inputs, std::shared_ptr output_schema, - std::unique_ptr segmenter, + std::unique_ptr segmenter, std::vector segment_field_ids, std::vector> target_fieldsets, std::vector aggs, std::vector kernels, + std::vector> kernel_intypes, std::vector>> states) : ExecNode(plan, std::move(inputs), {"target"}, /*output_schema=*/std::move(output_schema)), @@ -247,16 +252,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { target_fieldsets_(std::move(target_fieldsets)), aggs_(std::move(aggs)), kernels_(std::move(kernels)), - states_(std::move(states)) { - const auto& input_schema = *this->inputs()[0]->output_schema(); - for (size_t i = 0; i < kernels_.size(); ++i) { - std::vector in_types; - for (const auto& target : target_fieldsets_[i]) { - in_types.emplace_back(input_schema.field(target)->type().get()); - } - in_typesets_.push_back(std::move(in_types)); - } - } + kernel_intypes_(std::move(kernel_intypes)), + states_(std::move(states)) {} static Result Make(ExecPlan* plan, std::vector inputs, const ExecNodeOptions& options) { @@ -281,7 +278,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { std::vector segment_field_ids(segment_keys.size()); std::vector segment_key_types(segment_keys.size()); for (size_t i = 0; i < segment_keys.size(); i++) { - ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema)); + ARROW_ASSIGN_OR_RAISE(FieldPath match, segment_keys[i].FindOne(input_schema)); if (match.indices().size() > 1) { // ARROW-18369: Support nested references as segment ids return Status::Invalid("Nested references cannot be used as segment ids"); @@ -290,9 +287,10 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { segment_key_types[i] = input_schema.field(match[0])->type().get(); } - ARROW_ASSIGN_OR_RAISE( - auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx)); + ARROW_ASSIGN_OR_RAISE(auto segmenter, + RowSegmenter::Make(std::move(segment_key_types), exec_ctx)); + std::vector> kernel_intypes(aggregates.size()); std::vector kernels(aggregates.size()); std::vector>> states(kernels.size()); FieldVector fields(kernels.size() + segment_keys.size()); @@ -323,7 +321,9 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { for (const auto& target : target_fieldsets[i]) { in_types.emplace_back(input_schema.field(target)->type().get()); } - ARROW_ASSIGN_OR_RAISE(const Kernel* kernel, function->DispatchExact(in_types)); + kernel_intypes[i] = in_types; + ARROW_ASSIGN_OR_RAISE(const Kernel* kernel, + function->DispatchExact(kernel_intypes[i])); kernels[i] = static_cast(kernel); if (aggregates[i].options == nullptr) { @@ -337,13 +337,14 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { KernelContext kernel_ctx{exec_ctx}; states[i].resize(plan->query_context()->max_concurrency()); RETURN_NOT_OK(Kernel::InitAll( - &kernel_ctx, KernelInitArgs{kernels[i], in_types, aggregates[i].options.get()}, + &kernel_ctx, + KernelInitArgs{kernels[i], kernel_intypes[i], aggregates[i].options.get()}, &states[i])); // pick one to resolve the kernel signature kernel_ctx.SetState(states[i][0].get()); ARROW_ASSIGN_OR_RAISE(auto out_type, kernels[i]->signature->out_type().Resolve( - &kernel_ctx, in_types)); + &kernel_ctx, kernel_intypes[i])); fields[i] = field(aggregate_options.aggregates[i].name, out_type.GetSharedPtr()); } @@ -355,7 +356,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { return plan->EmplaceNode( plan, std::move(inputs), schema(std::move(fields)), std::move(segmenter), std::move(segment_field_ids), std::move(target_fieldsets), std::move(aggregates), - std::move(kernels), std::move(states)); + std::move(kernels), std::move(kernel_intypes), std::move(states)); } const char* kind_name() const override { return "ScalarAggregateNode"; } @@ -387,12 +388,21 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto thread_index = plan_->query_context()->GetThreadIndex(); auto handler = [this, thread_index](const ExecBatch& full_batch, - const GroupingSegment& segment) { - if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); + const SegmentPiece& segment) { + // (1) The segment piece is starting of a new segment and points to + // the beginning of the batch, then it means no data in the batch belongs + // to the current segment. We can output and reset kernel states. + if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); + + // We add segment piece to the current segment aggregation auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); RETURN_NOT_OK(GetScalarFields(&segmenter_values_, exec_batch, segment_field_ids_)); - if (!segment.is_open) RETURN_NOT_OK(OutputResult()); + + // If the segment piece closes the current segment, we can output segment + // aggregation. + if (!segment.is_open) RETURN_NOT_OK(OutputResult(false)); + return Status::OK(); }; RETURN_NOT_OK(HandleSegments(segmenter_, batch, segment_field_ids_, handler)); @@ -437,20 +447,20 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { } private: - Status ResetAggregates() { + Status ResetKernelStates() { auto exec_ctx = plan()->query_context()->exec_context(); for (size_t i = 0; i < kernels_.size(); ++i) { - const std::vector& in_types = in_typesets_[i]; states_[i].resize(plan()->query_context()->max_concurrency()); KernelContext kernel_ctx{exec_ctx}; RETURN_NOT_OK(Kernel::InitAll( - &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()}, + &kernel_ctx, + KernelInitArgs{kernels_[i], kernel_intypes_[i], aggs_[i].options.get()}, &states_[i])); } return Status::OK(); } - Status OutputResult(bool is_last = false) { + Status OutputResult(bool is_last) { ExecBatch batch{{}, 1}; batch.values.resize(kernels_.size() + segment_field_ids_.size()); @@ -472,11 +482,13 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { total_output_batches_++; if (is_last) { ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_)); + } else { + ARROW_RETURN_NOT_OK(ResetKernelStates()); } - return ResetAggregates(); + return Status::OK(); } - std::unique_ptr segmenter_; + std::unique_ptr segmenter_; const std::vector segment_field_ids_; std::vector segmenter_values_; @@ -484,7 +496,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { const std::vector aggs_; const std::vector kernels_; - std::vector> in_typesets_; + // Input type holders for each kernel, used for state initialization + std::vector> kernel_intypes_; std::vector>> states_; AtomicCounter input_counter_; @@ -495,7 +508,7 @@ class GroupByNode : public ExecNode, public TracedNode { public: GroupByNode(ExecNode* input, std::shared_ptr output_schema, std::vector key_field_ids, std::vector segment_key_field_ids, - std::unique_ptr segmenter, + std::unique_ptr segmenter, std::vector> agg_src_types, std::vector> agg_src_fieldsets, std::vector aggs, @@ -588,7 +601,7 @@ class GroupByNode : public ExecNode, public TracedNode { auto ctx = plan->query_context()->exec_context(); ARROW_ASSIGN_OR_RAISE(auto segmenter, - GroupingSegmenter::Make(std::move(segment_key_types), ctx)); + RowSegmenter::Make(std::move(segment_key_types), ctx)); // Construct aggregates ARROW_ASSIGN_OR_RAISE(auto agg_kernels, GetKernels(ctx, aggs, agg_src_types)); @@ -627,12 +640,7 @@ class GroupByNode : public ExecNode, public TracedNode { Status ResetAggregates() { auto ctx = plan()->query_context()->exec_context(); - - ARROW_ASSIGN_OR_RAISE(agg_kernels_, GetKernels(ctx, aggs_, agg_src_types_)); - - ARROW_ASSIGN_OR_RAISE(auto agg_states, - InitKernels(agg_kernels_, ctx, aggs_, agg_src_types_)); - + ARROW_RETURN_NOT_OK(InitKernels(agg_kernels_, ctx, aggs_, agg_src_types_)); return Status::OK(); } @@ -794,7 +802,7 @@ class GroupByNode : public ExecNode, public TracedNode { DCHECK_EQ(input, inputs_[0]); - auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) { + auto handler = [this](const ExecBatch& full_batch, const SegmentPiece& segment) { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); auto exec_batch = full_batch.Slice(segment.offset, segment.length); auto batch = ExecSpan(exec_batch); @@ -909,7 +917,7 @@ class GroupByNode : public ExecNode, public TracedNode { } int output_task_group_id_; - std::unique_ptr segmenter_; + std::unique_ptr segmenter_; std::vector segmenter_values_; const std::vector key_field_ids_; @@ -917,7 +925,7 @@ class GroupByNode : public ExecNode, public TracedNode { const std::vector> agg_src_types_; const std::vector> agg_src_fieldsets_; const std::vector aggs_; - std::vector agg_kernels_; + const std::vector agg_kernels_; AtomicCounter input_counter_; int total_output_batches_ = 0; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 18d067cc65a..a43d0655cd2 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -480,11 +480,11 @@ void TestGroupClassSupportedKeys() { ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})})); } -void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, - std::vector expected_segments) { +void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, + std::vector expected_segments) { int64_t offset = 0; for (auto expected_segment : expected_segments) { - ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset)); + ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegmentPiece(batch, offset)); ASSERT_EQ(expected_segment, segment); offset = segment.offset + segment.length; } @@ -492,11 +492,9 @@ void TestSegments(std::unique_ptr& segmenter, const ExecSpan& } // namespace -TEST(GroupingSegmenter, SupportedKeys) { - TestGroupClassSupportedKeys(); -} +TEST(RowSegmenter, SupportedKeys) { TestGroupClassSupportedKeys(); } -TEST(GroupingSegmenter, Basics) { +TEST(RowSegmenter, Basics) { std::vector bad_types2 = {int32(), float32()}; std::vector types2 = {int32(), int32()}; std::vector bad_types1 = {float32()}; @@ -507,53 +505,53 @@ TEST(GroupingSegmenter, Basics) { ExecBatch batch0({}, 3); { SCOPED_TRACE("offset"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types0)); ExecSpan span0(batch0); for (int64_t offset : {-1, 4}) { EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("invalid grouping segmenter offset"), - segmenter->GetNextSegment(span0, offset)); + segmenter->GetNextSegmentPiece(span0, offset)); } } { SCOPED_TRACE("types0 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types0)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "), - segmenter->GetNextSegment(span2, 0)); + segmenter->GetNextSegmentPiece(span2, 0)); ExecSpan span0(batch0); TestSegments(segmenter, span0, {{0, 3, true, true}, {3, 0, true, true}}); } { SCOPED_TRACE("bad_types1 segmenting of batch1"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types1)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types1)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "), - segmenter->GetNextSegment(span1, 0)); + segmenter->GetNextSegmentPiece(span1, 0)); } { SCOPED_TRACE("types1 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types1)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types1)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "), - segmenter->GetNextSegment(span2, 0)); + segmenter->GetNextSegmentPiece(span2, 0)); ExecSpan span1(batch1); TestSegments(segmenter, span1, {{0, 2, false, true}, {2, 1, true, false}, {3, 0, true, true}}); } { SCOPED_TRACE("bad_types2 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(bad_types2)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types2)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "), - segmenter->GetNextSegment(span2, 0)); + segmenter->GetNextSegmentPiece(span2, 0)); } { SCOPED_TRACE("types2 segmenting of batch1"); - ASSERT_OK_AND_ASSIGN(auto segmenter, GroupingSegmenter::Make(types2)); + ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types2)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "), - segmenter->GetNextSegment(span1, 0)); + segmenter->GetNextSegmentPiece(span1, 0)); ExecSpan span2(batch2); TestSegments(segmenter, span2, {{0, 1, false, true}, diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index d604ee1f4ed..549ab56ace7 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -62,8 +62,9 @@ inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0 } template -Status CheckForGetNextSegment(const std::vector& values, int64_t length, - int64_t offset, const std::vector& key_types) { +Status CheckForGetNextSegmentPiece(const std::vector& values, int64_t length, + int64_t offset, + const std::vector& key_types) { if (offset < 0 || offset > length) { return Status::Invalid("invalid grouping segmenter offset: ", offset); } @@ -85,13 +86,13 @@ Status CheckForGetNextSegment(const std::vector& values, int64_t length, template enable_if_t::value || std::is_same::value, Status> -CheckForGetNextSegment(const Batch& batch, int64_t offset, - const std::vector& key_types) { - return CheckForGetNextSegment(batch.values, batch.length, offset, key_types); +CheckForGetNextSegmentPiece(const Batch& batch, int64_t offset, + const std::vector& key_types) { + return CheckForGetNextSegmentPiece(batch.values, batch.length, offset, key_types); } -struct BaseGroupingSegmenter : public GroupingSegmenter { - explicit BaseGroupingSegmenter(const std::vector& key_types) +struct BaseRowSegmenter : public RowSegmenter { + explicit BaseRowSegmenter(const std::vector& key_types) : key_types_(key_types) {} const std::vector& key_types() const override { return key_types_; } @@ -99,9 +100,9 @@ struct BaseGroupingSegmenter : public GroupingSegmenter { std::vector key_types_; }; -GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, - bool extends) { - return GroupingSegment{offset, length, offset + length >= batch_length, extends}; +SegmentPiece MakeSegmentPiece(int64_t batch_length, int64_t offset, int64_t length, + bool extends) { + return SegmentPiece{offset, length, offset + length >= batch_length, extends}; } int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, @@ -121,28 +122,29 @@ using ExtendFunc = std::function; constexpr bool kDefaultExtends = true; constexpr bool kEmptyExtends = true; -struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter { - static std::unique_ptr Make() { - return std::make_unique(); +struct NoKeysSegmenter : public BaseRowSegmenter { + static std::unique_ptr Make() { + return std::make_unique(); } - NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {} + NoKeysSegmenter() : BaseRowSegmenter({}) {} Status Reset() override { return Status::OK(); } - Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {})); - return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends); + Result GetNextSegmentPiece(const ExecSpan& batch, + int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {})); + return MakeSegmentPiece(batch.length, offset, batch.length - offset, kDefaultExtends); } }; -struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { - static Result> Make(TypeHolder key_type) { - return std::make_unique(key_type); +struct SimpleKeySegmenter : public BaseRowSegmenter { + static Result> Make(TypeHolder key_type) { + return std::make_unique(key_type); } - explicit SimpleKeyGroupingSegmenter(TypeHolder key_type) - : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {} + explicit SimpleKeySegmenter(TypeHolder key_type) + : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {} Status CheckType(const DataType& type) { if (!is_fixed_width(type)) { @@ -168,43 +170,45 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { return extends; } - Result GetNextSegment(const Scalar& scalar, int64_t offset, - int64_t length) { + Result GetNextSegmentPiece(const Scalar& scalar, int64_t offset, + int64_t length) { ARROW_RETURN_NOT_OK(CheckType(*scalar.type)); if (!scalar.is_valid) { return Status::Invalid("segmenting an invalid scalar"); } auto data = checked_cast(scalar).data(); bool extends = length > 0 ? Extend(data) : kEmptyExtends; - return MakeSegment(length, offset, length, extends); + return MakeSegmentPiece(length, offset, length, extends); } - Result GetNextSegment(const DataType& array_type, - const uint8_t* array_bytes, int64_t offset, - int64_t length) { + Result GetNextSegmentPiece(const DataType& array_type, + const uint8_t* array_bytes, int64_t offset, + int64_t length) { RETURN_NOT_OK(CheckType(array_type)); int64_t byte_width = array_type.byte_width(); int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width, array_bytes, offset, length); bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends; - return MakeSegment(length, offset, match_length, extends); + return MakeSegmentPiece(length, offset, match_length, extends); } - Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_})); + Result GetNextSegmentPiece(const ExecSpan& batch, + int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {key_type_})); if (offset == batch.length) { - return MakeSegment(batch.length, offset, 0, kEmptyExtends); + return MakeSegmentPiece(batch.length, offset, 0, kEmptyExtends); } const auto& value = batch.values[0]; if (value.is_scalar()) { - return GetNextSegment(*value.scalar, offset, batch.length); + return GetNextSegmentPiece(*value.scalar, offset, batch.length); } ARROW_DCHECK(value.is_array()); const auto& array = value.array; if (array.GetNullCount() > 0) { return Status::NotImplemented("segmenting a nullable array"); } - return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length); + return GetNextSegmentPiece(*array.type, GetValuesAsBytes(array), offset, + batch.length); } private: @@ -212,15 +216,15 @@ struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter { std::vector save_key_data_; }; -struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { - static Result> Make( +struct AnyKeysSegmenter : public BaseRowSegmenter { + static Result> Make( const std::vector& key_types, ExecContext* ctx) { ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx)); // check types - return std::make_unique(key_types, ctx); + return std::make_unique(key_types, ctx); } - AnyKeysGroupingSegmenter(const std::vector& key_types, ExecContext* ctx) - : BaseGroupingSegmenter(key_types), + AnyKeysSegmenter(const std::vector& key_types, ExecContext* ctx) + : BaseRowSegmenter(key_types), ctx_(ctx), grouper_(nullptr), save_group_id_(kNoGroupId) {} @@ -245,7 +249,7 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { Result MapGroupIdAt(const Batch& batch, int64_t offset) { if (!grouper_) return kNoGroupId; ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset, - /*consume_length=*/1)); + /*length=*/1)); if (!datum.is_array()) { return Status::Invalid("accessing unsupported datum kind ", datum.kind()); } @@ -257,10 +261,11 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { return values[0]; } - Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_)); + Result GetNextSegmentPiece(const ExecSpan& batch, + int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, key_types_)); if (offset == batch.length) { - return MakeSegment(batch.length, offset, 0, kEmptyExtends); + return MakeSegmentPiece(batch.length, offset, 0, kEmptyExtends); } // ARROW-18311: make Grouper support Reset() // so it can be cached instead of recreated below @@ -286,7 +291,7 @@ struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter { } int64_t length = std::min(cursor, batch.length - offset); bool extends = length > 0 ? bound_extend(values) : kEmptyExtends; - return MakeSegment(batch.length, offset, length, extends); + return MakeSegmentPiece(batch.length, offset, length, extends); } else { return Status::Invalid("segmenting unsupported datum kind ", datum.kind()); } @@ -311,17 +316,17 @@ Status CheckForConsume(int64_t batch_length, int64_t& consume_offset, } // namespace -Result> GroupingSegmenter::Make( +Result> RowSegmenter::Make( const std::vector& key_types, bool nullable_keys, ExecContext* ctx) { if (key_types.size() == 0) { - return NoKeysGroupingSegmenter::Make(); + return NoKeysSegmenter::Make(); } else if (!nullable_keys && key_types.size() == 1) { const DataType* type = key_types[0].type; if (type != NULLPTR && is_fixed_width(*type)) { - return SimpleKeyGroupingSegmenter::Make(key_types[0]); + return SimpleKeySegmenter::Make(key_types[0]); } } - return AnyKeysGroupingSegmenter::Make(key_types, ctx); + return AnyKeysSegmenter::Make(key_types, ctx); } namespace { @@ -329,9 +334,8 @@ namespace { struct BaseGrouper : public Grouper { using Grouper::Consume; - Result Consume(const ExecBatch& batch, int64_t consume_offset, - int64_t consume_length) override { - return Consume(ExecSpan(batch), consume_offset, consume_length); + Result Consume(const ExecBatch& batch, int64_t offset, int64_t length) override { + return Consume(ExecSpan(batch), offset, length); } }; @@ -352,14 +356,12 @@ struct GrouperNoKeysImpl : Grouper { RETURN_NOT_OK(builder->Finish(&array)); return std::move(array); } - Result Consume(const ExecSpan& batch, int64_t consume_offset, - int64_t consume_length) override { - ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(consume_length, 0)); + Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { + ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(length, 0)); return Datum(array); } - Result Consume(const ExecBatch& batch, int64_t consume_offset, - int64_t consume_length) override { - ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(consume_length, 0)); + Result Consume(const ExecBatch& batch, int64_t offset, int64_t length) override { + ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(length, 0)); return Datum(array); } Result GetUniques() override { @@ -425,11 +427,10 @@ struct GrouperImpl : public BaseGrouper { using BaseGrouper::Consume; - Result Consume(const ExecSpan& batch, int64_t consume_offset, - int64_t consume_length) override { - ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); - if (consume_offset != 0 || consume_length != batch.length) { - auto batch_slice = batch.ToExecBatch().Slice(consume_offset, consume_length); + Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { + ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); + if (offset != 0 || length != batch.length) { + auto batch_slice = batch.ToExecBatch().Slice(offset, length); return Consume(ExecSpan(batch_slice), 0, -1); } std::vector offsets_batch(batch.length + 1); @@ -606,11 +607,10 @@ struct GrouperFastImpl : public BaseGrouper { using BaseGrouper::Consume; - Result Consume(const ExecSpan& batch, int64_t consume_offset, - int64_t consume_length) override { - ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, consume_offset, &consume_length)); - if (consume_offset != 0 || consume_length != batch.length) { - auto batch_slice = batch.ToExecBatch().Slice(consume_offset, consume_length); + Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { + ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); + if (offset != 0 || length != batch.length) { + auto batch_slice = batch.ToExecBatch().Slice(offset, length); return Consume(ExecSpan(batch_slice), 0, -1); } // ARROW-14027: broadcast scalar arguments for now diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 62a4d4258db..16f753144d4 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -30,23 +30,30 @@ namespace arrow { namespace compute { -/// \brief A segment of contiguous rows for grouping -struct ARROW_EXPORT GroupingSegment { +/// \brief A segment piece. +/// A segment is a chunk of continous rows that has the same segment key. (For example, +/// in ordered time series processing, segment key can be "date", and a segment can +/// be rows that belong to the same date.) A segment can span across multiple exec +/// batches. A segment piece is a chunk of continous rows that has the same segment key +/// within a given batch. When a segment span cross batches, it will have multiple segment +/// pieces. Segment piece never span cross batches. The segment piece data structure only +/// makes sense when used along with a exec batch. +struct ARROW_EXPORT SegmentPiece { /// \brief the offset into the batch where the segment starts int64_t offset; /// \brief the length of the segment int64_t length; - /// \brief whether the segment may be extended by a next one + /// \brief whether the segment piece may be extended by a next one bool is_open; - /// \brief whether the segment extends a preceeding one + /// \brief whether the segment piece extends a preceeding one bool extends; }; -inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) { +inline bool operator==(const SegmentPiece& segment1, const SegmentPiece& segment2) { return segment1.offset == segment2.offset && segment1.length == segment2.length && segment1.is_open == segment2.is_open && segment1.extends == segment2.extends; } -inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) { +inline bool operator!=(const SegmentPiece& segment1, const SegmentPiece& segment2) { return !(segment1 == segment2); } @@ -69,28 +76,28 @@ inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& s /// /// If the next call to the segmenter starts with `A A` then that segment would set the /// "extends" flag, which indicates whether the segment continues the last open batch. -class ARROW_EXPORT GroupingSegmenter { +class ARROW_EXPORT RowSegmenter { public: - virtual ~GroupingSegmenter() = default; + virtual ~RowSegmenter() = default; /// \brief Construct a GroupingSegmenter which segments on the specified key types /// /// \param[in] key_types the specified key types /// \param[in] nullable_keys whether values of the specified keys may be null /// \param[in] ctx the execution context to use - static Result> Make( + static Result> Make( const std::vector& key_types, bool nullable_keys = false, ExecContext* ctx = default_exec_context()); /// \brief Return the key types of this segmenter virtual const std::vector& key_types() const = 0; - /// \brief Reset this grouping segmenter + /// \brief Reset this segmenter virtual Status Reset() = 0; - /// \brief Get the next segment for the given batch starting from the given offset - virtual Result GetNextSegment(const ExecSpan& batch, - int64_t offset) = 0; + /// \brief Get the next segment piece for the given batch starting from the given offset + virtual Result GetNextSegmentPiece(const ExecSpan& batch, + int64_t offset) = 0; }; /// Consumes batches of keys and yields batches of the group ids. @@ -106,15 +113,15 @@ class ARROW_EXPORT Grouper { /// over a slice defined by an offset and length, which defaults to the batch length. /// Currently only uint32 indices will be produced, eventually the bit width will only /// be as wide as necessary. - virtual Result Consume(const ExecSpan& batch, int64_t consume_offset = 0, - int64_t consume_length = -1) = 0; + virtual Result Consume(const ExecSpan& batch, int64_t offset = 0, + int64_t length = -1) = 0; /// Consume a batch of keys, producing the corresponding group ids as an integer array, /// over a slice defined by an offset and length, which defaults to the batch length. /// Currently only uint32 indices will be produced, eventually the bit width will only /// be as wide as necessary. - virtual Result Consume(const ExecBatch& batch, int64_t consume_offset = 0, - int64_t consume_length = -1) = 0; + virtual Result Consume(const ExecBatch& batch, int64_t offset = 0, + int64_t length = -1) = 0; /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; From e96ac7763090577de5311ebfaeea21095519c8d5 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 3 Mar 2023 11:22:34 -0500 Subject: [PATCH 08/27] Rename segment piece back to segment --- cpp/src/arrow/compute/exec/aggregate_node.cc | 26 ++++++++-------- .../compute/kernels/hash_aggregate_test.cc | 16 +++++----- cpp/src/arrow/compute/row/grouper.cc | 18 +++++------ cpp/src/arrow/compute/row/grouper.h | 31 ++++++++++--------- 4 files changed, 46 insertions(+), 45 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index ee5c9436fc0..d9502d79380 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -45,15 +45,15 @@ // these columns. In segmented group-by aggregation, a second set of columns called // segment-keys is used to refine the partitioning. However, segment-keys are different in // that they partition only consecutive rows into a single group. Such a partition of -// consecutive rows is called a segment. For example, consider a column X with values +// consecutive rows is called a segment group. For example, consider a column X with values // [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X] yields // a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation with // segment-keys [X] yields [[0], [1], [2]]. // // The implementation first segments the input using the segment-keys, then groups by the -// keys. When a segment end is reached while scanning the input, output is pushed and the -// accumulating state is cleared. If no segment-keys are given, then the entire input is -// taken as one segment. +// keys. When a segment group end is reached while scanning the input, output is pushed and +// the accumulating state is cleared. If no segment-keys are given, then the entire input is +// taken as one segment group. namespace arrow { @@ -196,8 +196,8 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& ExecSpan segment_batch(segment_exec_batch); while (true) { - ARROW_ASSIGN_OR_RAISE(compute::SegmentPiece segment, - segmenter->GetNextSegmentPiece(segment_batch, offset)); + ARROW_ASSIGN_OR_RAISE(compute::Segment segment, + segmenter->GetNextSegment(segment_batch, offset)); if (segment.offset >= segment_batch.length) break; // condition of no-next-segment ARROW_RETURN_NOT_OK(handle_batch(batch, segment)); offset = segment.offset + segment.length; @@ -388,7 +388,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto thread_index = plan_->query_context()->GetThreadIndex(); auto handler = [this, thread_index](const ExecBatch& full_batch, - const SegmentPiece& segment) { + const Segment& segment) { // (1) The segment piece is starting of a new segment and points to // the beginning of the batch, then it means no data in the batch belongs // to the current segment. We can output and reset kernel states. @@ -638,7 +638,7 @@ class GroupByNode : public ExecNode, public TracedNode { std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels)); } - Status ResetAggregates() { + Status ResetKernelStates() { auto ctx = plan()->query_context()->exec_context(); ARROW_RETURN_NOT_OK(InitKernels(agg_kernels_, ctx, aggs_, agg_src_types_)); return Status::OK(); @@ -770,7 +770,7 @@ class GroupByNode : public ExecNode, public TracedNode { return output_->InputReceived(this, out_data_.Slice(batch_size * n, batch_size)); } - Status OutputResult(bool is_last = false) { + Status OutputResult(bool is_last) { // To simplify merging, ensure that the first grouper is nonempty for (size_t i = 0; i < local_states_.size(); i++) { if (local_states_[i].grouper) { @@ -792,7 +792,7 @@ class GroupByNode : public ExecNode, public TracedNode { for (int64_t i = 0; i < num_output_batches; i++) { ARROW_RETURN_NOT_OK(OutputNthBatch(i)); } - ARROW_RETURN_NOT_OK(ResetAggregates()); + ARROW_RETURN_NOT_OK(ResetKernelStates()); } return Status::OK(); } @@ -802,14 +802,14 @@ class GroupByNode : public ExecNode, public TracedNode { DCHECK_EQ(input, inputs_[0]); - auto handler = [this](const ExecBatch& full_batch, const SegmentPiece& segment) { - if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult()); + auto handler = [this](const ExecBatch& full_batch, const Segment& segment) { + if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); auto exec_batch = full_batch.Slice(segment.offset, segment.length); auto batch = ExecSpan(exec_batch); RETURN_NOT_OK(Consume(batch)); RETURN_NOT_OK( GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_)); - if (!segment.is_open) RETURN_NOT_OK(OutputResult()); + if (!segment.is_open) RETURN_NOT_OK(OutputResult(false)); return Status::OK(); }; ARROW_RETURN_NOT_OK( diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index a43d0655cd2..16dc6c5684a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -481,10 +481,10 @@ void TestGroupClassSupportedKeys() { } void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, - std::vector expected_segments) { + std::vector expected_segments) { int64_t offset = 0; for (auto expected_segment : expected_segments) { - ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegmentPiece(batch, offset)); + ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset)); ASSERT_EQ(expected_segment, segment); offset = segment.offset + segment.length; } @@ -510,7 +510,7 @@ TEST(RowSegmenter, Basics) { for (int64_t offset : {-1, 4}) { EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("invalid grouping segmenter offset"), - segmenter->GetNextSegmentPiece(span0, offset)); + segmenter->GetNextSegment(span0, offset)); } } { @@ -518,7 +518,7 @@ TEST(RowSegmenter, Basics) { ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types0)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "), - segmenter->GetNextSegmentPiece(span2, 0)); + segmenter->GetNextSegment(span2, 0)); ExecSpan span0(batch0); TestSegments(segmenter, span0, {{0, 3, true, true}, {3, 0, true, true}}); } @@ -527,14 +527,14 @@ TEST(RowSegmenter, Basics) { ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types1)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "), - segmenter->GetNextSegmentPiece(span1, 0)); + segmenter->GetNextSegment(span1, 0)); } { SCOPED_TRACE("types1 segmenting of batch2"); ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types1)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "), - segmenter->GetNextSegmentPiece(span2, 0)); + segmenter->GetNextSegment(span2, 0)); ExecSpan span1(batch1); TestSegments(segmenter, span1, {{0, 2, false, true}, {2, 1, true, false}, {3, 0, true, true}}); @@ -544,14 +544,14 @@ TEST(RowSegmenter, Basics) { ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types2)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "), - segmenter->GetNextSegmentPiece(span2, 0)); + segmenter->GetNextSegment(span2, 0)); } { SCOPED_TRACE("types2 segmenting of batch1"); ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types2)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "), - segmenter->GetNextSegmentPiece(span1, 0)); + segmenter->GetNextSegment(span1, 0)); ExecSpan span2(batch2); TestSegments(segmenter, span2, {{0, 1, false, true}, diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 549ab56ace7..7f2d35a179e 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -100,9 +100,9 @@ struct BaseRowSegmenter : public RowSegmenter { std::vector key_types_; }; -SegmentPiece MakeSegmentPiece(int64_t batch_length, int64_t offset, int64_t length, +Segment MakeSegmentPiece(int64_t batch_length, int64_t offset, int64_t length, bool extends) { - return SegmentPiece{offset, length, offset + length >= batch_length, extends}; + return Segment{offset, length, offset + length >= batch_length, extends}; } int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, @@ -131,7 +131,7 @@ struct NoKeysSegmenter : public BaseRowSegmenter { Status Reset() override { return Status::OK(); } - Result GetNextSegmentPiece(const ExecSpan& batch, + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {})); return MakeSegmentPiece(batch.length, offset, batch.length - offset, kDefaultExtends); @@ -170,7 +170,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { return extends; } - Result GetNextSegmentPiece(const Scalar& scalar, int64_t offset, + Result GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) { ARROW_RETURN_NOT_OK(CheckType(*scalar.type)); if (!scalar.is_valid) { @@ -181,7 +181,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { return MakeSegmentPiece(length, offset, length, extends); } - Result GetNextSegmentPiece(const DataType& array_type, + Result GetNextSegment(const DataType& array_type, const uint8_t* array_bytes, int64_t offset, int64_t length) { RETURN_NOT_OK(CheckType(array_type)); @@ -192,7 +192,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { return MakeSegmentPiece(length, offset, match_length, extends); } - Result GetNextSegmentPiece(const ExecSpan& batch, + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {key_type_})); if (offset == batch.length) { @@ -200,14 +200,14 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { } const auto& value = batch.values[0]; if (value.is_scalar()) { - return GetNextSegmentPiece(*value.scalar, offset, batch.length); + return GetNextSegment(*value.scalar, offset, batch.length); } ARROW_DCHECK(value.is_array()); const auto& array = value.array; if (array.GetNullCount() > 0) { return Status::NotImplemented("segmenting a nullable array"); } - return GetNextSegmentPiece(*array.type, GetValuesAsBytes(array), offset, + return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length); } @@ -261,7 +261,7 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { return values[0]; } - Result GetNextSegmentPiece(const ExecSpan& batch, + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, key_types_)); if (offset == batch.length) { diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 16f753144d4..d322038d81c 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -30,30 +30,31 @@ namespace arrow { namespace compute { -/// \brief A segment piece. -/// A segment is a chunk of continous rows that has the same segment key. (For example, -/// in ordered time series processing, segment key can be "date", and a segment can -/// be rows that belong to the same date.) A segment can span across multiple exec -/// batches. A segment piece is a chunk of continous rows that has the same segment key -/// within a given batch. When a segment span cross batches, it will have multiple segment -/// pieces. Segment piece never span cross batches. The segment piece data structure only +/// \brief A segment. +/// A segment group is a chunk of continous rows that has the same segment key. (For example, +/// in ordered time series processing, segment key can be "date", and a segment group can +/// be all the rows that belong to the same date.) A segment group can span across multiple exec +/// batches. +/// A segment is a chunk of continous rows that has the same segment key +/// within a given batch. When a ? span cross batches, it will have multiple segments. +/// A segment never span cross batches. The segment data structure only /// makes sense when used along with a exec batch. -struct ARROW_EXPORT SegmentPiece { +struct ARROW_EXPORT Segment { /// \brief the offset into the batch where the segment starts int64_t offset; /// \brief the length of the segment int64_t length; - /// \brief whether the segment piece may be extended by a next one + /// \brief whether the segment may be extended by a next one bool is_open; - /// \brief whether the segment piece extends a preceeding one + /// \brief whether the segment extends a preceeding one bool extends; }; -inline bool operator==(const SegmentPiece& segment1, const SegmentPiece& segment2) { +inline bool operator==(const Segment& segment1, const Segment& segment2) { return segment1.offset == segment2.offset && segment1.length == segment2.length && segment1.is_open == segment2.is_open && segment1.extends == segment2.extends; } -inline bool operator!=(const SegmentPiece& segment1, const SegmentPiece& segment2) { +inline bool operator!=(const Segment& segment1, const Segment& segment2) { return !(segment1 == segment2); } @@ -80,7 +81,7 @@ class ARROW_EXPORT RowSegmenter { public: virtual ~RowSegmenter() = default; - /// \brief Construct a GroupingSegmenter which segments on the specified key types + /// \brief Construct a Segmenter which segments on the specified key types /// /// \param[in] key_types the specified key types /// \param[in] nullable_keys whether values of the specified keys may be null @@ -95,8 +96,8 @@ class ARROW_EXPORT RowSegmenter { /// \brief Reset this segmenter virtual Status Reset() = 0; - /// \brief Get the next segment piece for the given batch starting from the given offset - virtual Result GetNextSegmentPiece(const ExecSpan& batch, + /// \brief Get the next segment for the given batch starting from the given offset + virtual Result GetNextSegment(const ExecSpan& batch, int64_t offset) = 0; }; From 1c4579b949ca64e0a469430d68afbfc22135f3eb Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 3 Mar 2023 11:41:11 -0500 Subject: [PATCH 09/27] Format change --- cpp/src/arrow/compute/exec/aggregate_node.cc | 14 ++--- cpp/src/arrow/compute/row/grouper.cc | 54 +++++++++----------- cpp/src/arrow/compute/row/grouper.h | 18 +++---- 3 files changed, 38 insertions(+), 48 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index d9502d79380..eca05c0166f 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -45,15 +45,15 @@ // these columns. In segmented group-by aggregation, a second set of columns called // segment-keys is used to refine the partitioning. However, segment-keys are different in // that they partition only consecutive rows into a single group. Such a partition of -// consecutive rows is called a segment group. For example, consider a column X with values -// [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X] yields -// a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation with -// segment-keys [X] yields [[0], [1], [2]]. +// consecutive rows is called a segment group. For example, consider a column X with +// values [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X] +// yields a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation +// with segment-keys [X] yields [[0], [1], [2]]. // // The implementation first segments the input using the segment-keys, then groups by the -// keys. When a segment group end is reached while scanning the input, output is pushed and -// the accumulating state is cleared. If no segment-keys are given, then the entire input is -// taken as one segment group. +// keys. When a segment group end is reached while scanning the input, output is pushed +// and the accumulating state is cleared. If no segment-keys are given, then the entire +// input is taken as one segment group. namespace arrow { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 7f2d35a179e..ebc0fcae2f7 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -62,9 +62,8 @@ inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0 } template -Status CheckForGetNextSegmentPiece(const std::vector& values, int64_t length, - int64_t offset, - const std::vector& key_types) { +Status CheckForGetNextSegment(const std::vector& values, int64_t length, + int64_t offset, const std::vector& key_types) { if (offset < 0 || offset > length) { return Status::Invalid("invalid grouping segmenter offset: ", offset); } @@ -86,9 +85,9 @@ Status CheckForGetNextSegmentPiece(const std::vector& values, int64_t len template enable_if_t::value || std::is_same::value, Status> -CheckForGetNextSegmentPiece(const Batch& batch, int64_t offset, - const std::vector& key_types) { - return CheckForGetNextSegmentPiece(batch.values, batch.length, offset, key_types); +CheckForGetNextSegment(const Batch& batch, int64_t offset, + const std::vector& key_types) { + return CheckForGetNextSegment(batch.values, batch.length, offset, key_types); } struct BaseRowSegmenter : public RowSegmenter { @@ -100,8 +99,7 @@ struct BaseRowSegmenter : public RowSegmenter { std::vector key_types_; }; -Segment MakeSegmentPiece(int64_t batch_length, int64_t offset, int64_t length, - bool extends) { +Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) { return Segment{offset, length, offset + length >= batch_length, extends}; } @@ -131,10 +129,9 @@ struct NoKeysSegmenter : public BaseRowSegmenter { Status Reset() override { return Status::OK(); } - Result GetNextSegment(const ExecSpan& batch, - int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {})); - return MakeSegmentPiece(batch.length, offset, batch.length - offset, kDefaultExtends); + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {})); + return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends); } }; @@ -148,7 +145,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { Status CheckType(const DataType& type) { if (!is_fixed_width(type)) { - return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type); + return Status::Invalid("SimpleKeySegmenter does not support type ", type); } return Status::OK(); } @@ -170,33 +167,30 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { return extends; } - Result GetNextSegment(const Scalar& scalar, int64_t offset, - int64_t length) { + Result GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) { ARROW_RETURN_NOT_OK(CheckType(*scalar.type)); if (!scalar.is_valid) { return Status::Invalid("segmenting an invalid scalar"); } auto data = checked_cast(scalar).data(); bool extends = length > 0 ? Extend(data) : kEmptyExtends; - return MakeSegmentPiece(length, offset, length, extends); + return MakeSegment(length, offset, length, extends); } - Result GetNextSegment(const DataType& array_type, - const uint8_t* array_bytes, int64_t offset, - int64_t length) { + Result GetNextSegment(const DataType& array_type, const uint8_t* array_bytes, + int64_t offset, int64_t length) { RETURN_NOT_OK(CheckType(array_type)); int64_t byte_width = array_type.byte_width(); int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width, array_bytes, offset, length); bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends; - return MakeSegmentPiece(length, offset, match_length, extends); + return MakeSegment(length, offset, match_length, extends); } - Result GetNextSegment(const ExecSpan& batch, - int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, {key_type_})); + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_})); if (offset == batch.length) { - return MakeSegmentPiece(batch.length, offset, 0, kEmptyExtends); + return MakeSegment(batch.length, offset, 0, kEmptyExtends); } const auto& value = batch.values[0]; if (value.is_scalar()) { @@ -207,8 +201,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { if (array.GetNullCount() > 0) { return Status::NotImplemented("segmenting a nullable array"); } - return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, - batch.length); + return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length); } private: @@ -261,11 +254,10 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { return values[0]; } - Result GetNextSegment(const ExecSpan& batch, - int64_t offset) override { - ARROW_RETURN_NOT_OK(CheckForGetNextSegmentPiece(batch, offset, key_types_)); + Result GetNextSegment(const ExecSpan& batch, int64_t offset) override { + ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_)); if (offset == batch.length) { - return MakeSegmentPiece(batch.length, offset, 0, kEmptyExtends); + return MakeSegment(batch.length, offset, 0, kEmptyExtends); } // ARROW-18311: make Grouper support Reset() // so it can be cached instead of recreated below @@ -291,7 +283,7 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { } int64_t length = std::min(cursor, batch.length - offset); bool extends = length > 0 ? bound_extend(values) : kEmptyExtends; - return MakeSegmentPiece(batch.length, offset, length, extends); + return MakeSegment(batch.length, offset, length, extends); } else { return Status::Invalid("segmenting unsupported datum kind ", datum.kind()); } diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index d322038d81c..6d9ed32c27a 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -31,14 +31,13 @@ namespace arrow { namespace compute { /// \brief A segment. -/// A segment group is a chunk of continous rows that has the same segment key. (For example, -/// in ordered time series processing, segment key can be "date", and a segment group can -/// be all the rows that belong to the same date.) A segment group can span across multiple exec -/// batches. -/// A segment is a chunk of continous rows that has the same segment key -/// within a given batch. When a ? span cross batches, it will have multiple segments. -/// A segment never span cross batches. The segment data structure only -/// makes sense when used along with a exec batch. +/// A segment group is a chunk of continous rows that has the same segment key. (For +/// example, in ordered time series processing, segment key can be "date", and a segment +/// group can be all the rows that belong to the same date.) A segment group can span +/// across multiple exec batches. A segment is a chunk of continous rows that has the same +/// segment key within a given batch. When a ? span cross batches, it will have multiple +/// segments. A segment never span cross batches. The segment data structure only makes +/// sense when used along with a exec batch. struct ARROW_EXPORT Segment { /// \brief the offset into the batch where the segment starts int64_t offset; @@ -97,8 +96,7 @@ class ARROW_EXPORT RowSegmenter { virtual Status Reset() = 0; /// \brief Get the next segment for the given batch starting from the given offset - virtual Result GetNextSegment(const ExecSpan& batch, - int64_t offset) = 0; + virtual Result GetNextSegment(const ExecSpan& batch, int64_t offset) = 0; }; /// Consumes batches of keys and yields batches of the group ids. From 192dc879e76ff25a4b24874ffa9560ac086a7843 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 3 Mar 2023 11:46:00 -0500 Subject: [PATCH 10/27] Fix comment w.r.t segment/segment group --- cpp/src/arrow/compute/exec/aggregate_node.cc | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index eca05c0166f..b0afb23581a 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -389,17 +389,17 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto thread_index = plan_->query_context()->GetThreadIndex(); auto handler = [this, thread_index](const ExecBatch& full_batch, const Segment& segment) { - // (1) The segment piece is starting of a new segment and points to + // (1) The segment is starting of a new segment group and points to // the beginning of the batch, then it means no data in the batch belongs - // to the current segment. We can output and reset kernel states. + // to the current segment group. We can output and reset kernel states. if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); - // We add segment piece to the current segment aggregation + // We add segment to the current segment group aggregation auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); RETURN_NOT_OK(GetScalarFields(&segmenter_values_, exec_batch, segment_field_ids_)); - // If the segment piece closes the current segment, we can output segment + // If the segment closes the current segment group, we can output segment group // aggregation. if (!segment.is_open) RETURN_NOT_OK(OutputResult(false)); From c92f8d01dd1f08f5c5912c35f65b3cccd201979b Mon Sep 17 00:00:00 2001 From: Li Jin Date: Fri, 3 Mar 2023 13:39:19 -0500 Subject: [PATCH 11/27] Fix docstring for segment --- cpp/src/arrow/compute/row/grouper.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 6d9ed32c27a..540cb91181a 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -30,14 +30,14 @@ namespace arrow { namespace compute { -/// \brief A segment. -/// A segment group is a chunk of continous rows that has the same segment key. (For +/// \brief A segment +/// A segment group is a chunk of continous rows that have the same segment key. (For /// example, in ordered time series processing, segment key can be "date", and a segment /// group can be all the rows that belong to the same date.) A segment group can span /// across multiple exec batches. A segment is a chunk of continous rows that has the same -/// segment key within a given batch. When a ? span cross batches, it will have multiple -/// segments. A segment never span cross batches. The segment data structure only makes -/// sense when used along with a exec batch. +/// segment key within a given batch. When a segment group span cross batches, it will +/// have multiple segments. A segment never spans cross batches. The segment data +/// structure only makes sense when used along with a exec batch. struct ARROW_EXPORT Segment { /// \brief the offset into the batch where the segment starts int64_t offset; From 111a7b971d8ed216b3f893f5864b5b37346db27b Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sat, 4 Mar 2023 15:02:03 -0500 Subject: [PATCH 12/27] add plan tests --- cpp/src/arrow/compute/exec/plan_test.cc | 103 ++++++++++++++++++++++++ 1 file changed, 103 insertions(+) diff --git a/cpp/src/arrow/compute/exec/plan_test.cc b/cpp/src/arrow/compute/exec/plan_test.cc index 5b2af718df7..66cfa2563b6 100644 --- a/cpp/src/arrow/compute/exec/plan_test.cc +++ b/cpp/src/arrow/compute/exec/plan_test.cc @@ -1476,5 +1476,108 @@ TEST(ExecPlan, SourceEnforcesBatchLimit) { } } +TEST(ExecPlanExecution, SegmentedAggregationWithMultiThreading) { + BatchesWithSchema data; + data.batches = {ExecBatchFromJSON({int32()}, "[[1]]")}; + data.schema = schema({field("i32", int32())}); + Declaration plan = Declaration::Sequence( + {{"source", + SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{ + {"count", nullptr, "i32", "count(i32)"}, + }, + /*keys=*/{"i32"}, /*segment_leys=*/{"i32"}}}}); + EXPECT_RAISES_WITH_MESSAGE_THAT(NotImplemented, HasSubstr("multi-threaded"), + DeclarationToExecBatches(std::move(plan))); +} + +TEST(ExecPlanExecution, SegmentedAggregationWithOneSegment) { + BatchesWithSchema data; + data.batches = { + ExecBatchFromJSON({int32(), int32(), int32()}, "[[1, 1, 1], [1, 2, 1], [1, 1, 2]]"), + ExecBatchFromJSON({int32(), int32(), int32()}, + "[[1, 2, 2], [1, 1, 3], [1, 2, 3]]")}; + data.schema = schema({ + field("a", int32()), + field("b", int32()), + field("c", int32()), + }); + + Declaration plan = Declaration::Sequence( + {{"source", + SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{ + {"hash_sum", nullptr, "c", "sum(c)"}, + {"hash_mean", nullptr, "c", "mean(c)"}, + }, + /*keys=*/{"b"}, /*segment_leys=*/{"a"}}}}); + ASSERT_OK_AND_ASSIGN(BatchesWithCommonSchema actual_batches, + DeclarationToExecBatches(std::move(plan), /*use_threads=*/false)); + + auto expected = ExecBatchFromJSON({int64(), float64(), int32(), int32()}, + R"([[6, 2, 1, 1], [6, 2, 2, 1]])"); + AssertExecBatchesEqualIgnoringOrder(actual_batches.schema, actual_batches.batches, + {expected}); +} + +TEST(ExecPlanExecution, SegmentedAggregationWithTwoSegments) { + BatchesWithSchema data; + data.batches = { + ExecBatchFromJSON({int32(), int32(), int32()}, "[[1, 1, 1], [1, 2, 1], [1, 1, 2]]"), + ExecBatchFromJSON({int32(), int32(), int32()}, + "[[2, 2, 2], [2, 1, 3], [2, 2, 3]]")}; + data.schema = schema({ + field("a", int32()), + field("b", int32()), + field("c", int32()), + }); + + Declaration plan = Declaration::Sequence( + {{"source", + SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{ + {"hash_sum", nullptr, "c", "sum(c)"}, + {"hash_mean", nullptr, "c", "mean(c)"}, + }, + /*keys=*/{"b"}, /*segment_leys=*/{"a"}}}}); + ASSERT_OK_AND_ASSIGN(BatchesWithCommonSchema actual_batches, + DeclarationToExecBatches(std::move(plan), /*use_threads=*/false)); + + auto expected = ExecBatchFromJSON( + {int64(), float64(), int32(), int32()}, + R"([[3, 1.5, 1, 1], [1, 1, 2, 1], [3, 3, 1, 2], [5, 2.5, 2, 2]])"); + AssertExecBatchesEqualIgnoringOrder(actual_batches.schema, actual_batches.batches, + {expected}); +} + +TEST(ExecPlanExecution, SegmentedAggregationWithBatchCrossingSegment) { + BatchesWithSchema data; + data.batches = { + ExecBatchFromJSON({int32(), int32(), int32()}, "[[1, 1, 1], [1, 1, 1], [2, 2, 2]]"), + ExecBatchFromJSON({int32(), int32(), int32()}, + "[[2, 2, 2], [3, 3, 3], [3, 3, 3]]")}; + data.schema = schema({ + field("a", int32()), + field("b", int32()), + field("c", int32()), + }); + + Declaration plan = Declaration::Sequence( + {{"source", + SourceNodeOptions{data.schema, data.gen(/*parallel=*/false, /*slow=*/false)}}, + {"aggregate", AggregateNodeOptions{/*aggregates=*/{ + {"hash_sum", nullptr, "c", "sum(c)"}, + {"hash_mean", nullptr, "c", "mean(c)"}, + }, + /*keys=*/{"b"}, /*segment_leys=*/{"a"}}}}); + ASSERT_OK_AND_ASSIGN(BatchesWithCommonSchema actual_batches, + DeclarationToExecBatches(std::move(plan), /*use_threads=*/false)); + + auto expected = ExecBatchFromJSON({int64(), float64(), int32(), int32()}, + R"([[2, 1, 1, 1], [4, 2, 2, 2], [6, 3, 3, 3]])"); + AssertExecBatchesEqualIgnoringOrder(actual_batches.schema, actual_batches.batches, + {expected}); +} + } // namespace compute } // namespace arrow From d85dac38ce95f5dbb179e70cf7c35dc12003d0f2 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sat, 4 Mar 2023 16:29:29 -0500 Subject: [PATCH 13/27] requested changes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 21 ++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index b0afb23581a..ed70bc18133 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -53,7 +53,7 @@ // The implementation first segments the input using the segment-keys, then groups by the // keys. When a segment group end is reached while scanning the input, output is pushed // and the accumulating state is cleared. If no segment-keys are given, then the entire -// input is taken as one segment group. +// input is taken as one segment group. One batch per segment group is sent to output. namespace arrow { @@ -205,8 +205,8 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& return Status::OK(); } -Status GetScalarFields(std::vector* values_ptr, const ExecBatch& input_batch, - const std::vector& field_ids) { +Status SelectConstantFields(std::vector* values_ptr, const ExecBatch& input_batch, + const std::vector& field_ids) { DCHECK_GT(input_batch.length, 0); std::vector& values = *values_ptr; int64_t row = input_batch.length - 1; @@ -397,7 +397,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { // We add segment to the current segment group aggregation auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); - RETURN_NOT_OK(GetScalarFields(&segmenter_values_, exec_batch, segment_field_ids_)); + RETURN_NOT_OK( + SelectConstantFields(&segmenter_values_, exec_batch, segment_field_ids_)); // If the segment closes the current segment group, we can output segment group // aggregation. @@ -450,7 +451,6 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { Status ResetKernelStates() { auto exec_ctx = plan()->query_context()->exec_context(); for (size_t i = 0; i < kernels_.size(); ++i) { - states_[i].resize(plan()->query_context()->max_concurrency()); KernelContext kernel_ctx{exec_ctx}; RETURN_NOT_OK(Kernel::InitAll( &kernel_ctx, @@ -488,8 +488,11 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { return Status::OK(); } + /// \brief A segmenter for the segment-keys std::unique_ptr segmenter_; + /// \brief Field indices corresponding to the segment-keys const std::vector segment_field_ids_; + /// \brief Holds values of the current batch that were selected for the segment-keys std::vector segmenter_values_; const std::vector> target_fieldsets_; @@ -501,6 +504,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { std::vector>> states_; AtomicCounter input_counter_; + /// \brief Total number of output batches produced int total_output_batches_ = 0; }; @@ -808,7 +812,7 @@ class GroupByNode : public ExecNode, public TracedNode { auto batch = ExecSpan(exec_batch); RETURN_NOT_OK(Consume(batch)); RETURN_NOT_OK( - GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_)); + SelectConstantFields(&segmenter_values_, exec_batch, segment_key_field_ids_)); if (!segment.is_open) RETURN_NOT_OK(OutputResult(false)); return Status::OK(); }; @@ -917,17 +921,22 @@ class GroupByNode : public ExecNode, public TracedNode { } int output_task_group_id_; + /// \brief A segmenter for the segment-keys std::unique_ptr segmenter_; + /// \brief Holds values of the current batch that were selected for the segment-keys std::vector segmenter_values_; const std::vector key_field_ids_; + /// \brief Field indices corresponding to the segment-keys const std::vector segment_key_field_ids_; + /// \brief Types of input fields per aggregate const std::vector> agg_src_types_; const std::vector> agg_src_fieldsets_; const std::vector aggs_; const std::vector agg_kernels_; AtomicCounter input_counter_; + /// \brief Total number of output batches produced int total_output_batches_ = 0; std::vector local_states_; From 28fce61e9469a1fe586a291ff930ee106556c1cd Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sat, 4 Mar 2023 17:02:33 -0500 Subject: [PATCH 14/27] reinstate state resize --- cpp/src/arrow/compute/exec/aggregate_node.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index ed70bc18133..ced9ff99d1f 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -451,6 +451,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { Status ResetKernelStates() { auto exec_ctx = plan()->query_context()->exec_context(); for (size_t i = 0; i < kernels_.size(); ++i) { + states_[i].resize(plan()->query_context()->max_concurrency()); KernelContext kernel_ctx{exec_ctx}; RETURN_NOT_OK(Kernel::InitAll( &kernel_ctx, From 218e04ce50d6aae792b797573c2111aa3e5912a3 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sun, 5 Mar 2023 07:16:39 -0500 Subject: [PATCH 15/27] add tests and fix --- .../compute/kernels/hash_aggregate_test.cc | 63 ++++++++++++++++++- cpp/src/arrow/compute/row/grouper.cc | 2 +- 2 files changed, 63 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 16dc6c5684a..8017a53d6b8 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -482,8 +482,9 @@ void TestGroupClassSupportedKeys() { void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, std::vector expected_segments) { - int64_t offset = 0; + int64_t offset = 0, segment_num = 0; for (auto expected_segment : expected_segments) { + SCOPED_TRACE("segment #" + ToChars(segment_num++)); ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset)); ASSERT_EQ(expected_segment, segment); offset = segment.offset + segment.length; @@ -561,6 +562,66 @@ TEST(RowSegmenter, Basics) { } } +namespace { + +void test_row_segmenter_constant_batch(std::function shape_func) { + constexpr size_t n = 3; + std::vector types = {int32(), int32(), int32()}; + std::vector shapes(n); + for (size_t i = 0; i < n; i++) shapes[i] = shape_func(i); + auto full_batch = ExecBatchFromJSON(types, shapes, "[[1, 1, 1], [1, 1, 1], [1, 1, 1]]"); + auto test_by_size = [&](size_t size) -> Status { + SCOPED_TRACE("constant-batch with " + ToChars(size) + " key(s)"); + std::vector values(&full_batch.values[0], &full_batch.values[size]); + ExecBatch batch(values, full_batch.length); + std::vector key_types(&types[0], &types[size]); + ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); + TestSegments(segmenter, ExecSpan(batch), {{0, 3, true, true}, {3, 0, true, true}}); + return Status::OK(); + }; + for (size_t i = 0; i <= 3; i++) { + ASSERT_OK(test_by_size(i)); + } +} + +} // namespace + +TEST(RowSegmenter, ConstantArrayBatch) { + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }); +} + +TEST(RowSegmenter, ConstantScalarBatch) { + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }); +} + +TEST(RowSegmenter, ConstantMixedBatch) { + test_row_segmenter_constant_batch( + [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }); +} + +TEST(RowSegmenter, RowConstantBatch) { + constexpr size_t n = 3; + std::vector types = {int32(), int32(), int32()}; + auto full_batch = ExecBatchFromJSON(types, "[[1, 1, 1], [2, 2, 2], [3, 3, 3]]"); + std::vector expected_segments_for_size_0 = {{0, 3, true, true}, + {3, 0, true, true}}; + std::vector expected_segments = { + {0, 1, false, true}, {1, 1, false, false}, {2, 1, true, false}, {3, 0, true, true}}; + auto test_by_size = [&](size_t size) -> Status { + SCOPED_TRACE("constant-batch with " + ToChars(size) + " key(s)"); + std::vector values(&full_batch.values[0], &full_batch.values[size]); + ExecBatch batch(values, full_batch.length); + std::vector key_types(&types[0], &types[size]); + ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); + TestSegments(segmenter, ExecSpan(batch), + size == 0 ? expected_segments_for_size_0 : expected_segments); + return Status::OK(); + }; + for (size_t i = 0; i <= n; i++) { + ASSERT_OK(test_by_size(i)); + } +} + TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(); } struct TestGrouper { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index ebc0fcae2f7..eb2ca6c365c 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -113,7 +113,7 @@ int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, break; } } - return std::min(cursor, length - offset); + return std::min(cursor, length) - offset; } using ExtendFunc = std::function; From 4f18abd38cd70b5960ebe2d19855ff58db752efa Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Sun, 5 Mar 2023 11:27:58 -0500 Subject: [PATCH 16/27] fix vector access --- cpp/src/arrow/compute/kernels/hash_aggregate_test.cc | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 8017a53d6b8..e31c8b9701b 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -572,9 +572,10 @@ void test_row_segmenter_constant_batch(std::function shape_f auto full_batch = ExecBatchFromJSON(types, shapes, "[[1, 1, 1], [1, 1, 1], [1, 1, 1]]"); auto test_by_size = [&](size_t size) -> Status { SCOPED_TRACE("constant-batch with " + ToChars(size) + " key(s)"); - std::vector values(&full_batch.values[0], &full_batch.values[size]); + std::vector values(full_batch.values.begin(), + full_batch.values.begin() + size); ExecBatch batch(values, full_batch.length); - std::vector key_types(&types[0], &types[size]); + std::vector key_types(types.begin(), types.begin() + size); ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); TestSegments(segmenter, ExecSpan(batch), {{0, 3, true, true}, {3, 0, true, true}}); return Status::OK(); @@ -609,9 +610,10 @@ TEST(RowSegmenter, RowConstantBatch) { {0, 1, false, true}, {1, 1, false, false}, {2, 1, true, false}, {3, 0, true, true}}; auto test_by_size = [&](size_t size) -> Status { SCOPED_TRACE("constant-batch with " + ToChars(size) + " key(s)"); - std::vector values(&full_batch.values[0], &full_batch.values[size]); + std::vector values(full_batch.values.begin(), + full_batch.values.begin() + size); ExecBatch batch(values, full_batch.length); - std::vector key_types(&types[0], &types[size]); + std::vector key_types(types.begin(), types.begin() + size); ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); TestSegments(segmenter, ExecSpan(batch), size == 0 ? expected_segments_for_size_0 : expected_segments); From a633038c67914fedcd04a08a4aad43553efcbaa7 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Mon, 6 Mar 2023 15:25:21 -0500 Subject: [PATCH 17/27] requested changes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 6 +- .../compute/kernels/hash_aggregate_test.cc | 81 ++++++++++++------- cpp/src/arrow/compute/row/grouper.h | 3 +- 3 files changed, 58 insertions(+), 32 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index ced9ff99d1f..ac6635bc176 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -288,7 +288,8 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { } ARROW_ASSIGN_OR_RAISE(auto segmenter, - RowSegmenter::Make(std::move(segment_key_types), exec_ctx)); + RowSegmenter::Make(std::move(segment_key_types), + /*nullable_keys=*/false, exec_ctx)); std::vector> kernel_intypes(aggregates.size()); std::vector kernels(aggregates.size()); @@ -606,7 +607,8 @@ class GroupByNode : public ExecNode, public TracedNode { auto ctx = plan->query_context()->exec_context(); ARROW_ASSIGN_OR_RAISE(auto segmenter, - RowSegmenter::Make(std::move(segment_key_types), ctx)); + RowSegmenter::Make(std::move(segment_key_types), + /*nullable_keys=*/false, ctx)); // Construct aggregates ARROW_ASSIGN_OR_RAISE(auto agg_kernels, GetKernels(ctx, aggs, agg_src_types)); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index e31c8b9701b..5a12af634fc 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -201,6 +201,18 @@ Result RunGroupBy(const BatchesWithSchema& input, const std::vector& segment_key_names, const std::vector& aggregates, ExecContext* ctx, bool use_threads, bool segmented = false, bool naive = false) { + // When segment_keys is non-empty the `segmented` flag is always true; otherwise (when + // empty), it may still be set to true. In this case, the tester restructures (without + // changing the data of) the result of RunGroupBy from `std::vector` + // (output_batches) to `std::vector` (out_arrays), which have the structure + // typical of the case of a non-empty segment_keys (with multiple arrays per column, one + // array per segment) but only one array per column (because, technically, there is only + // one segment in this case). Thus, this case focuses on the structure of the result. + // + // The `naive` flag means that the output is expected to be that of like `NaiveGroupBy`, + // which in particular doesn't require sorting. The reason for the naive flag is that + // the expected output of some test-cases is naive and of some others it is not. The + // current `RunGroupBy` function deals with both kinds of expected output. std::vector keys(key_names.size()); for (size_t i = 0; i < key_names.size(); ++i) { keys[i] = FieldRef(key_names[i]); @@ -438,22 +450,24 @@ Result GroupByTest(GroupByFunction group_by, const std::vector& ar } template -void TestGroupClassSupportedKeys() { - ASSERT_OK(GroupClass::Make({boolean()})); +void TestGroupClassSupportedKeys( + std::function>(const std::vector&)> + make_func) { + ASSERT_OK(make_func({boolean()})); - ASSERT_OK(GroupClass::Make({int8(), uint16(), int32(), uint64()})); + ASSERT_OK(make_func({int8(), uint16(), int32(), uint64()})); - ASSERT_OK(GroupClass::Make({dictionary(int64(), utf8())})); + ASSERT_OK(make_func({dictionary(int64(), utf8())})); - ASSERT_OK(GroupClass::Make({float16(), float32(), float64()})); + ASSERT_OK(make_func({float16(), float32(), float64()})); - ASSERT_OK(GroupClass::Make({utf8(), binary(), large_utf8(), large_binary()})); + ASSERT_OK(make_func({utf8(), binary(), large_utf8(), large_binary()})); - ASSERT_OK(GroupClass::Make({fixed_size_binary(16), fixed_size_binary(32)})); + ASSERT_OK(make_func({fixed_size_binary(16), fixed_size_binary(32)})); - ASSERT_OK(GroupClass::Make({decimal128(32, 10), decimal256(76, 20)})); + ASSERT_OK(make_func({decimal128(32, 10), decimal256(76, 20)})); - ASSERT_OK(GroupClass::Make({date32(), date64()})); + ASSERT_OK(make_func({date32(), date64()})); for (auto unit : { TimeUnit::SECOND, @@ -461,23 +475,23 @@ void TestGroupClassSupportedKeys() { TimeUnit::MICRO, TimeUnit::NANO, }) { - ASSERT_OK(GroupClass::Make({timestamp(unit), duration(unit)})); + ASSERT_OK(make_func({timestamp(unit), duration(unit)})); } - ASSERT_OK(GroupClass::Make( - {day_time_interval(), month_interval(), month_day_nano_interval()})); + ASSERT_OK( + make_func({day_time_interval(), month_interval(), month_day_nano_interval()})); - ASSERT_OK(GroupClass::Make({null()})); + ASSERT_OK(make_func({null()})); - ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({field("", int64())})})); + ASSERT_RAISES(NotImplemented, make_func({struct_({field("", int64())})})); - ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({})})); + ASSERT_RAISES(NotImplemented, make_func({struct_({})})); - ASSERT_RAISES(NotImplemented, GroupClass::Make({list(int32())})); + ASSERT_RAISES(NotImplemented, make_func({list(int32())})); - ASSERT_RAISES(NotImplemented, GroupClass::Make({fixed_size_list(int32(), 5)})); + ASSERT_RAISES(NotImplemented, make_func({fixed_size_list(int32(), 5)})); - ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})})); + ASSERT_RAISES(NotImplemented, make_func({dense_union({field("", int32())})})); } void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batch, @@ -491,9 +505,20 @@ void TestSegments(std::unique_ptr& segmenter, const ExecSpan& batc } } +Result> MakeGrouper(const std::vector& key_types) { + return Grouper::Make(key_types, default_exec_context()); +} + +Result> MakeRowSegmenter( + const std::vector& key_types) { + return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context()); +} + } // namespace -TEST(RowSegmenter, SupportedKeys) { TestGroupClassSupportedKeys(); } +TEST(RowSegmenter, SupportedKeys) { + TestGroupClassSupportedKeys(MakeRowSegmenter); +} TEST(RowSegmenter, Basics) { std::vector bad_types2 = {int32(), float32()}; @@ -506,7 +531,7 @@ TEST(RowSegmenter, Basics) { ExecBatch batch0({}, 3); { SCOPED_TRACE("offset"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types0)); ExecSpan span0(batch0); for (int64_t offset : {-1, 4}) { EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, @@ -516,7 +541,7 @@ TEST(RowSegmenter, Basics) { } { SCOPED_TRACE("types0 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types0)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types0)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "), segmenter->GetNextSegment(span2, 0)); @@ -525,14 +550,14 @@ TEST(RowSegmenter, Basics) { } { SCOPED_TRACE("bad_types1 segmenting of batch1"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types1)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(bad_types1)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "), segmenter->GetNextSegment(span1, 0)); } { SCOPED_TRACE("types1 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types1)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types1)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "), segmenter->GetNextSegment(span2, 0)); @@ -542,14 +567,14 @@ TEST(RowSegmenter, Basics) { } { SCOPED_TRACE("bad_types2 segmenting of batch2"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(bad_types2)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(bad_types2)); ExecSpan span2(batch2); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "), segmenter->GetNextSegment(span2, 0)); } { SCOPED_TRACE("types2 segmenting of batch1"); - ASSERT_OK_AND_ASSIGN(auto segmenter, RowSegmenter::Make(types2)); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types2)); ExecSpan span1(batch1); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "), segmenter->GetNextSegment(span1, 0)); @@ -576,7 +601,7 @@ void test_row_segmenter_constant_batch(std::function shape_f full_batch.values.begin() + size); ExecBatch batch(values, full_batch.length); std::vector key_types(types.begin(), types.begin() + size); - ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); + ARROW_ASSIGN_OR_RAISE(auto segmenter, MakeRowSegmenter(key_types)); TestSegments(segmenter, ExecSpan(batch), {{0, 3, true, true}, {3, 0, true, true}}); return Status::OK(); }; @@ -614,7 +639,7 @@ TEST(RowSegmenter, RowConstantBatch) { full_batch.values.begin() + size); ExecBatch batch(values, full_batch.length); std::vector key_types(types.begin(), types.begin() + size); - ARROW_ASSIGN_OR_RAISE(auto segmenter, RowSegmenter::Make(key_types)); + ARROW_ASSIGN_OR_RAISE(auto segmenter, MakeRowSegmenter(key_types)); TestSegments(segmenter, ExecSpan(batch), size == 0 ? expected_segments_for_size_0 : expected_segments); return Status::OK(); @@ -624,7 +649,7 @@ TEST(RowSegmenter, RowConstantBatch) { } } -TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(); } +TEST(Grouper, SupportedKeys) { TestGroupClassSupportedKeys(MakeGrouper); } struct TestGrouper { explicit TestGrouper(std::vector types, std::vector shapes = {}) diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 540cb91181a..6cf62463701 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -86,8 +86,7 @@ class ARROW_EXPORT RowSegmenter { /// \param[in] nullable_keys whether values of the specified keys may be null /// \param[in] ctx the execution context to use static Result> Make( - const std::vector& key_types, bool nullable_keys = false, - ExecContext* ctx = default_exec_context()); + const std::vector& key_types, bool nullable_keys, ExecContext* ctx); /// \brief Return the key types of this segmenter virtual const std::vector& key_types() const = 0; From e52439c4d8ab7426d02d109eddf9983fe1c271f8 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Mon, 6 Mar 2023 16:30:10 -0500 Subject: [PATCH 18/27] Minor doc/rename changes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 23 ++++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index ac6635bc176..3eca2845eb6 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -205,8 +205,14 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& return Status::OK(); } -Status SelectConstantFields(std::vector* values_ptr, const ExecBatch& input_batch, - const std::vector& field_ids) { +/// @brief Extract values of segment keys from a segment batch +/// @param[out] values_ptr Vector to store the extract segmenter values +/// @param[in] input_batch Segment batch. Must have the a constant value for segment key +/// @param[in] field_ids Segment key field ids +/// @return +Status ExtractSegmenterValues(std::vector* values_ptr, + const ExecBatch& input_batch, + const std::vector& field_ids) { DCHECK_GT(input_batch.length, 0); std::vector& values = *values_ptr; int64_t row = input_batch.length - 1; @@ -399,7 +405,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); RETURN_NOT_OK( - SelectConstantFields(&segmenter_values_, exec_batch, segment_field_ids_)); + ExtractSegmenterValues(&segmenter_values_, exec_batch, segment_field_ids_)); // If the segment closes the current segment group, we can output segment group // aggregation. @@ -490,11 +496,12 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { return Status::OK(); } - /// \brief A segmenter for the segment-keys + // A segmenter for the segment-keys std::unique_ptr segmenter_; - /// \brief Field indices corresponding to the segment-keys + // Field indices corresponding to the segment-keys const std::vector segment_field_ids_; - /// \brief Holds values of the current batch that were selected for the segment-keys + // Holds the value of segment keys of the most recent input batch + // The values are updated everytime an input batch is processed std::vector segmenter_values_; const std::vector> target_fieldsets_; @@ -811,11 +818,13 @@ class GroupByNode : public ExecNode, public TracedNode { auto handler = [this](const ExecBatch& full_batch, const Segment& segment) { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); + // This is not zero copy - we should refactor the code to pass + // offset and length to Consume to avoid copying here auto exec_batch = full_batch.Slice(segment.offset, segment.length); auto batch = ExecSpan(exec_batch); RETURN_NOT_OK(Consume(batch)); RETURN_NOT_OK( - SelectConstantFields(&segmenter_values_, exec_batch, segment_key_field_ids_)); + ExtractSegmenterValues(&segmenter_values_, exec_batch, segment_key_field_ids_)); if (!segment.is_open) RETURN_NOT_OK(OutputResult(false)); return Status::OK(); }; From 7edcb18def26a24f391539de196a2f5d854011b3 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Mon, 6 Mar 2023 16:32:26 -0500 Subject: [PATCH 19/27] fix typo --- cpp/src/arrow/compute/exec/aggregate_node.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 3eca2845eb6..74fe0b8ece4 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -206,7 +206,7 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& } /// @brief Extract values of segment keys from a segment batch -/// @param[out] values_ptr Vector to store the extract segmenter values +/// @param[out] values_ptr Vector to store the extracted segment key values /// @param[in] input_batch Segment batch. Must have the a constant value for segment key /// @param[in] field_ids Segment key field ids /// @return From 48d39ebe6328fcbe2a02383011c2227ff41a41a1 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Tue, 7 Mar 2023 07:55:56 -0500 Subject: [PATCH 20/27] requested changes --- .../compute/kernels/hash_aggregate_test.cc | 47 ++++++++++++++--- cpp/src/arrow/compute/row/grouper.cc | 52 +++++++++---------- cpp/src/arrow/compute/row/grouper.h | 11 ++-- cpp/src/arrow/compute/row/grouper_internal.h | 27 ++++++++++ 4 files changed, 94 insertions(+), 43 deletions(-) create mode 100644 cpp/src/arrow/compute/row/grouper_internal.h diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 5a12af634fc..ad041e317bb 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -42,6 +42,7 @@ #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" #include "arrow/table.h" #include "arrow/testing/generator.h" #include "arrow/testing/gtest_util.h" @@ -209,7 +210,7 @@ Result RunGroupBy(const BatchesWithSchema& input, // array per segment) but only one array per column (because, technically, there is only // one segment in this case). Thus, this case focuses on the structure of the result. // - // The `naive` flag means that the output is expected to be that of like `NaiveGroupBy`, + // The `naive` flag means that the output is expected to be like that of `NaiveGroupBy`, // which in particular doesn't require sorting. The reason for the naive flag is that // the expected output of some test-cases is naive and of some others it is not. The // current `RunGroupBy` function deals with both kinds of expected output. @@ -514,6 +515,11 @@ Result> MakeRowSegmenter( return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context()); } +Result> MakeGenericSegmenter( + const std::vector& key_types) { + return MakeAnyKeysSegmenter(key_types, default_exec_context()); +} + } // namespace TEST(RowSegmenter, SupportedKeys) { @@ -589,8 +595,11 @@ TEST(RowSegmenter, Basics) { namespace { -void test_row_segmenter_constant_batch(std::function shape_func) { - constexpr size_t n = 3; +void test_row_segmenter_constant_batch( + std::function shape_func, + std::function>(const std::vector&)> + make_segmenter) { + constexpr size_t n = 3, repetitions = 3; std::vector types = {int32(), int32(), int32()}; std::vector shapes(n); for (size_t i = 0; i < n; i++) shapes[i] = shape_func(i); @@ -601,8 +610,11 @@ void test_row_segmenter_constant_batch(std::function shape_f full_batch.values.begin() + size); ExecBatch batch(values, full_batch.length); std::vector key_types(types.begin(), types.begin() + size); - ARROW_ASSIGN_OR_RAISE(auto segmenter, MakeRowSegmenter(key_types)); - TestSegments(segmenter, ExecSpan(batch), {{0, 3, true, true}, {3, 0, true, true}}); + ARROW_ASSIGN_OR_RAISE(auto segmenter, make_segmenter(key_types)); + for (size_t i = 0; i < repetitions; i++) { + TestSegments(segmenter, ExecSpan(batch), {{0, 3, true, true}, {3, 0, true, true}}); + ARROW_RETURN_NOT_OK(segmenter->Reset()); + } return Status::OK(); }; for (size_t i = 0; i <= 3; i++) { @@ -613,16 +625,35 @@ void test_row_segmenter_constant_batch(std::function shape_f } // namespace TEST(RowSegmenter, ConstantArrayBatch) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }); + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }, + MakeRowSegmenter); } TEST(RowSegmenter, ConstantScalarBatch) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }); + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }, + MakeRowSegmenter); } TEST(RowSegmenter, ConstantMixedBatch) { test_row_segmenter_constant_batch( - [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }); + [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }, + MakeRowSegmenter); +} + +TEST(RowSegmenter, ConstantArrayBatchWithAnyKeysSegmenter) { + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }, + MakeGenericSegmenter); +} + +TEST(RowSegmenter, ConstantScalarBatchWithAnyKeysSegmenter) { + test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }, + MakeGenericSegmenter); +} + +TEST(RowSegmenter, ConstantMixedBatchWithAnyKeysSegmenter) { + test_row_segmenter_constant_batch( + [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }, + MakeGenericSegmenter); } TEST(RowSegmenter, RowConstantBatch) { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index eb2ca6c365c..3446ce183d2 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -32,6 +32,7 @@ #include "arrow/compute/light_array.h" #include "arrow/compute/registry.h" #include "arrow/compute/row/compare_internal.h" +#include "arrow/compute/row/grouper_internal.h" #include "arrow/type.h" #include "arrow/type_traits.h" #include "arrow/util/bitmap_ops.h" @@ -103,6 +104,8 @@ Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool e return Segment{offset, length, offset + length >= batch_length, extends}; } +// Used by SimpleKeySegmenter::GetNextSegment to find the match-length of a value within a +// fixed-width buffer int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, const uint8_t* array_bytes, int64_t offset, int64_t length) { int64_t cursor, byte_cursor; @@ -117,8 +120,8 @@ int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width, } using ExtendFunc = std::function; -constexpr bool kDefaultExtends = true; -constexpr bool kEmptyExtends = true; +constexpr bool kDefaultExtends = true; // by default, the first segment extends +constexpr bool kEmptyExtends = true; // an empty segment extends too struct NoKeysSegmenter : public BaseRowSegmenter { static std::unique_ptr Make() { @@ -141,7 +144,10 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { } explicit SimpleKeySegmenter(TypeHolder key_type) - : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {} + : BaseRowSegmenter({key_type}), + key_type_(key_types_[0]), + save_key_data_(static_cast(key_type_.type->byte_width())), + extend_was_called_(false) {} Status CheckType(const DataType& type) { if (!is_fixed_width(type)) { @@ -151,19 +157,18 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { } Status Reset() override { - save_key_data_.resize(0); + extend_was_called_ = false; return Status::OK(); } // Checks whether the given grouping data extends the current segment, i.e., is equal to // previously seen grouping data, which is updated with each invocation. bool Extend(const void* data) { - size_t byte_width = static_cast(key_type_.type->byte_width()); - bool extends = save_key_data_.size() != byte_width + bool extends = !extend_was_called_ ? kDefaultExtends - : 0 == memcmp(save_key_data_.data(), data, byte_width); - save_key_data_.resize(byte_width); - memcpy(save_key_data_.data(), data, byte_width); + : 0 == memcmp(save_key_data_.data(), data, save_key_data_.size()); + extend_was_called_ = true; + memcpy(save_key_data_.data(), data, save_key_data_.size()); return extends; } @@ -180,6 +185,7 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { Result GetNextSegment(const DataType& array_type, const uint8_t* array_bytes, int64_t offset, int64_t length) { RETURN_NOT_OK(CheckType(array_type)); + DCHECK_LE(offset, length); int64_t byte_width = array_type.byte_width(); int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width, array_bytes, offset, length); @@ -206,7 +212,8 @@ struct SimpleKeySegmenter : public BaseRowSegmenter { private: TypeHolder key_type_; - std::vector save_key_data_; + std::vector save_key_data_; // previusly seen segment-key grouping data + bool extend_was_called_; }; struct AnyKeysSegmenter : public BaseRowSegmenter { @@ -308,6 +315,11 @@ Status CheckForConsume(int64_t batch_length, int64_t& consume_offset, } // namespace +Result> MakeAnyKeysSegmenter( + const std::vector& key_types, ExecContext* ctx) { + return AnyKeysSegmenter::Make(key_types, ctx); +} + Result> RowSegmenter::Make( const std::vector& key_types, bool nullable_keys, ExecContext* ctx) { if (key_types.size() == 0) { @@ -323,14 +335,6 @@ Result> RowSegmenter::Make( namespace { -struct BaseGrouper : public Grouper { - using Grouper::Consume; - - Result Consume(const ExecBatch& batch, int64_t offset, int64_t length) override { - return Consume(ExecSpan(batch), offset, length); - } -}; - struct GrouperNoKeysImpl : Grouper { Result> MakeConstantGroupIdArray(int64_t length, group_id_t value) { @@ -352,10 +356,6 @@ struct GrouperNoKeysImpl : Grouper { ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(length, 0)); return Datum(array); } - Result Consume(const ExecBatch& batch, int64_t offset, int64_t length) override { - ARROW_ASSIGN_OR_RAISE(auto array, MakeConstantGroupIdArray(length, 0)); - return Datum(array); - } Result GetUniques() override { auto data = ArrayData::Make(uint32(), 1, 0); auto values = data->GetMutableValues(0); @@ -366,7 +366,7 @@ struct GrouperNoKeysImpl : Grouper { uint32_t num_groups() const override { return 1; } }; -struct GrouperImpl : public BaseGrouper { +struct GrouperImpl : public Grouper { static Result> Make( const std::vector& key_types, ExecContext* ctx) { auto impl = std::make_unique(); @@ -417,8 +417,6 @@ struct GrouperImpl : public BaseGrouper { return std::move(impl); } - using BaseGrouper::Consume; - Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); if (offset != 0 || length != batch.length) { @@ -508,7 +506,7 @@ struct GrouperImpl : public BaseGrouper { std::vector> encoders_; }; -struct GrouperFastImpl : public BaseGrouper { +struct GrouperFastImpl : public Grouper { static constexpr int kBitmapPaddingForSIMD = 64; // bits static constexpr int kPaddingForSIMD = 32; // bytes @@ -597,8 +595,6 @@ struct GrouperFastImpl : public BaseGrouper { ~GrouperFastImpl() { map_.cleanup(); } - using BaseGrouper::Consume; - Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); if (offset != 0 || length != batch.length) { diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h index 6cf62463701..f9e7e2e97e7 100644 --- a/cpp/src/arrow/compute/row/grouper.h +++ b/cpp/src/arrow/compute/row/grouper.h @@ -92,6 +92,10 @@ class ARROW_EXPORT RowSegmenter { virtual const std::vector& key_types() const = 0; /// \brief Reset this segmenter + /// + /// A segmenter normally extends (see `Segment`) a segment from one batch to the next. + /// If segment-extenion is undesirable, for example when each batch is processed + /// independently, then `Reset` should be invoked before processing the next batch. virtual Status Reset() = 0; /// \brief Get the next segment for the given batch starting from the given offset @@ -114,13 +118,6 @@ class ARROW_EXPORT Grouper { virtual Result Consume(const ExecSpan& batch, int64_t offset = 0, int64_t length = -1) = 0; - /// Consume a batch of keys, producing the corresponding group ids as an integer array, - /// over a slice defined by an offset and length, which defaults to the batch length. - /// Currently only uint32 indices will be produced, eventually the bit width will only - /// be as wide as necessary. - virtual Result Consume(const ExecBatch& batch, int64_t offset = 0, - int64_t length = -1) = 0; - /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; diff --git a/cpp/src/arrow/compute/row/grouper_internal.h b/cpp/src/arrow/compute/row/grouper_internal.h new file mode 100644 index 00000000000..1926854b941 --- /dev/null +++ b/cpp/src/arrow/compute/row/grouper_internal.h @@ -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. + +#pragma once + +namespace arrow { +namespace compute { + +Result> MakeAnyKeysSegmenter( + const std::vector& key_types, ExecContext* ctx); + +} // namespace compute +} // namespace arrow From 967a07c1ca77bef150daac00f1d9ed60bcd243f6 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Tue, 7 Mar 2023 08:35:18 -0500 Subject: [PATCH 21/27] add export --- cpp/src/arrow/compute/row/grouper_internal.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/row/grouper_internal.h b/cpp/src/arrow/compute/row/grouper_internal.h index 1926854b941..eb3dfe8ba16 100644 --- a/cpp/src/arrow/compute/row/grouper_internal.h +++ b/cpp/src/arrow/compute/row/grouper_internal.h @@ -20,7 +20,7 @@ namespace arrow { namespace compute { -Result> MakeAnyKeysSegmenter( +ARROW_EXPORT Result> MakeAnyKeysSegmenter( const std::vector& key_types, ExecContext* ctx); } // namespace compute From d76323c8d9536b367bc1fe0ee523005d0b785e13 Mon Sep 17 00:00:00 2001 From: Li Jin Date: Tue, 7 Mar 2023 09:58:07 -0500 Subject: [PATCH 22/27] Fix docstring --- cpp/src/arrow/compute/exec/aggregate_node.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 74fe0b8ece4..7b8e48ab6dd 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -209,7 +209,6 @@ Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& /// @param[out] values_ptr Vector to store the extracted segment key values /// @param[in] input_batch Segment batch. Must have the a constant value for segment key /// @param[in] field_ids Segment key field ids -/// @return Status ExtractSegmenterValues(std::vector* values_ptr, const ExecBatch& input_batch, const std::vector& field_ids) { From 5e8cbc9b74980b02209639d0034aaab413f183ba Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Tue, 7 Mar 2023 12:33:39 -0500 Subject: [PATCH 23/27] requested changes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 1 + .../compute/kernels/hash_aggregate_test.cc | 90 ++++++++++++++----- cpp/src/arrow/compute/row/grouper.cc | 8 +- 3 files changed, 74 insertions(+), 25 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 7b8e48ab6dd..1fe6cf9086e 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -401,6 +401,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); // We add segment to the current segment group aggregation + // GH-34475: change to zero-copy slicing auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); RETURN_NOT_OK( diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index ad041e317bb..a8339f72cd7 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -73,8 +73,6 @@ using internal::ToChars; namespace compute { namespace { -bool kDefaultUseThreads = false; - using GroupByFunction = std::function( const std::vector&, const std::vector&, const std::vector&, const std::vector&, bool, bool)>; @@ -405,8 +403,8 @@ void ValidateGroupBy(GroupByFunction group_by, const std::vector& agg bool naive = true) { ASSERT_OK_AND_ASSIGN(Datum expected, NaiveGroupBy(arguments, keys, aggregates)); - ASSERT_OK_AND_ASSIGN( - Datum actual, group_by(arguments, keys, {}, aggregates, kDefaultUseThreads, naive)); + ASSERT_OK_AND_ASSIGN(Datum actual, group_by(arguments, keys, {}, aggregates, + /*use_threads=*/false, naive)); ASSERT_OK(expected.make_array()->ValidateFull()); ValidateOutput(actual); @@ -595,7 +593,7 @@ TEST(RowSegmenter, Basics) { namespace { -void test_row_segmenter_constant_batch( +void TestRowSegmenterConstantBatch( std::function shape_func, std::function>(const std::vector&)> make_segmenter) { @@ -625,33 +623,33 @@ void test_row_segmenter_constant_batch( } // namespace TEST(RowSegmenter, ConstantArrayBatch) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }, - MakeRowSegmenter); + TestRowSegmenterConstantBatch([](size_t i) { return ArgShape::ARRAY; }, + MakeRowSegmenter); } TEST(RowSegmenter, ConstantScalarBatch) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }, - MakeRowSegmenter); + TestRowSegmenterConstantBatch([](size_t i) { return ArgShape::SCALAR; }, + MakeRowSegmenter); } TEST(RowSegmenter, ConstantMixedBatch) { - test_row_segmenter_constant_batch( + TestRowSegmenterConstantBatch( [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }, MakeRowSegmenter); } TEST(RowSegmenter, ConstantArrayBatchWithAnyKeysSegmenter) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::ARRAY; }, - MakeGenericSegmenter); + TestRowSegmenterConstantBatch([](size_t i) { return ArgShape::ARRAY; }, + MakeGenericSegmenter); } TEST(RowSegmenter, ConstantScalarBatchWithAnyKeysSegmenter) { - test_row_segmenter_constant_batch([](size_t i) { return ArgShape::SCALAR; }, - MakeGenericSegmenter); + TestRowSegmenterConstantBatch([](size_t i) { return ArgShape::SCALAR; }, + MakeGenericSegmenter); } TEST(RowSegmenter, ConstantMixedBatchWithAnyKeysSegmenter) { - test_row_segmenter_constant_batch( + TestRowSegmenterConstantBatch( [](size_t i) { return i % 2 == 0 ? ArgShape::SCALAR : ArgShape::ARRAY; }, MakeGenericSegmenter); } @@ -2248,7 +2246,7 @@ TEST_P(GroupBy, MinMaxDecimal) { } } -TEST_P(GroupBy, MinMaxFixedSizeBinary) { +TEST_P(GroupBy, MinMaxBinary) { for (bool use_threads : {true, false}) { for (const auto& ty : BaseBinaryTypes()) { SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); @@ -2299,6 +2297,56 @@ TEST_P(GroupBy, MinMaxFixedSizeBinary) { } } +TEST_P(GroupBy, MinMaxFixedSizeBinary) { + const auto ty = fixed_size_binary(3); + for (bool use_threads : {true, false}) { + SCOPED_TRACE(use_threads ? "parallel/merged" : "serial"); + + auto table = TableFromJSON(schema({ + field("argument0", ty), + field("key", int64()), + }), + {R"([ + ["aaa", 1], + [null, 1] +])", + R"([ + ["bac", 2], + [null, 3], + ["234", null], + ["ddd", 1], + ["bcd", 2] +])", + R"([ + ["bab", 2], + ["123", null], + [null, 3] +])"}); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupByTest({table->GetColumnByName("argument0")}, + {table->GetColumnByName("key")}, + {{"hash_min_max", nullptr}}, use_threads)); + ValidateOutput(aggregated_and_grouped); + SortBy({"key_0"}, &aggregated_and_grouped); + + AssertDatumsEqual( + ArrayFromJSON( + struct_({ + field("hash_min_max", struct_({field("min", ty), field("max", ty)})), + field("key_0", int64()), + }), + R"([ + [{"min": "aaa", "max": "ddd"}, 1], + [{"min": "bab", "max": "bcd"}, 2], + [{"min": null, "max": null}, 3], + [{"min": "123", "max": "234"}, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); + } +} + TEST_P(GroupBy, MinOrMax) { auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64())}), {R"([ @@ -4560,11 +4608,11 @@ class SegmentedKeyGroupBy : public GroupBy {}; void TestSegment(GroupByFunction group_by, const std::shared_ptr
& table, Datum output, const std::vector& keys, - const std::vector& segment_keys, bool scalar) { + const std::vector& segment_keys, bool is_scalar_aggregate) { const char* names[] = { - scalar ? "count" : "hash_count", - scalar ? "sum" : "hash_sum", - scalar ? "min_max" : "hash_min_max", + is_scalar_aggregate ? "count" : "hash_count", + is_scalar_aggregate ? "sum" : "hash_sum", + is_scalar_aggregate ? "min_max" : "hash_min_max", }; ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, group_by( @@ -4579,7 +4627,7 @@ void TestSegment(GroupByFunction group_by, const std::shared_ptr
& table, {names[1], nullptr, "agg_1", names[1]}, {names[2], nullptr, "agg_2", names[2]}, }, - kDefaultUseThreads, /*naive=*/false)); + /*use_threads=*/false, /*naive=*/false)); AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true); } diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 3446ce183d2..3681f80561a 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -302,8 +302,8 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { group_id_t save_group_id_; }; -Status CheckForConsume(int64_t batch_length, int64_t& consume_offset, - int64_t* consume_length) { +Status CheckAndCapLengthForConsume(int64_t batch_length, int64_t& consume_offset, + int64_t* consume_length) { if (consume_offset < 0) { return Status::Invalid("invalid grouper consume offset: ", consume_offset); } @@ -418,7 +418,7 @@ struct GrouperImpl : public Grouper { } Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { - ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); + ARROW_RETURN_NOT_OK(CheckAndCapLengthForConsume(batch.length, offset, &length)); if (offset != 0 || length != batch.length) { auto batch_slice = batch.ToExecBatch().Slice(offset, length); return Consume(ExecSpan(batch_slice), 0, -1); @@ -596,7 +596,7 @@ struct GrouperFastImpl : public Grouper { ~GrouperFastImpl() { map_.cleanup(); } Result Consume(const ExecSpan& batch, int64_t offset, int64_t length) override { - ARROW_RETURN_NOT_OK(CheckForConsume(batch.length, offset, &length)); + ARROW_RETURN_NOT_OK(CheckAndCapLengthForConsume(batch.length, offset, &length)); if (offset != 0 || length != batch.length) { auto batch_slice = batch.ToExecBatch().Slice(offset, length); return Consume(ExecSpan(batch_slice), 0, -1); From d68efd4c295d9fedfaf6ccc7804dfb591c4ecff2 Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Tue, 7 Mar 2023 16:41:59 -0500 Subject: [PATCH 24/27] requested changes, additional tests --- .../compute/kernels/hash_aggregate_test.cc | 61 ++++++++++++++++--- 1 file changed, 54 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index a8339f72cd7..6c45299d763 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -591,6 +591,51 @@ TEST(RowSegmenter, Basics) { } } +TEST(RowSegmenter, NonOrdered) { + std::vector types = {int32()}; + auto batch = ExecBatchFromJSON(types, "[[1], [1], [2], [1], [2]]"); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types)); + TestSegments(segmenter, ExecSpan(batch), + {{0, 2, false, true}, + {2, 1, false, false}, + {3, 1, false, false}, + {4, 1, true, false}, + {5, 0, true, true}}); +} + +TEST(RowSegmenter, EmptyBatches) { + std::vector types = {int32()}; + std::vector batches = { + ExecBatchFromJSON(types, "[]"), ExecBatchFromJSON(types, "[]"), + ExecBatchFromJSON(types, "[[1]]"), ExecBatchFromJSON(types, "[]"), + ExecBatchFromJSON(types, "[[1]]"), ExecBatchFromJSON(types, "[]"), + ExecBatchFromJSON(types, "[[2], [2]]"), ExecBatchFromJSON(types, "[]"), + }; + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types)); + TestSegments(segmenter, ExecSpan(batches[0]), {}); + TestSegments(segmenter, ExecSpan(batches[1]), {}); + TestSegments(segmenter, ExecSpan(batches[2]), {{0, 1, true, true}}); + TestSegments(segmenter, ExecSpan(batches[3]), {}); + TestSegments(segmenter, ExecSpan(batches[4]), {{0, 1, true, true}}); + TestSegments(segmenter, ExecSpan(batches[5]), {}); + TestSegments(segmenter, ExecSpan(batches[6]), {{0, 2, true, false}}); + TestSegments(segmenter, ExecSpan(batches[7]), {}); +} + +TEST(RowSegmenter, MultipleSegments) { + std::vector types = {int32()}; + auto batch = ExecBatchFromJSON(types, "[[1], [1], [2], [5], [3], [3], [5], [5], [4]]"); + ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types)); + TestSegments(segmenter, ExecSpan(batch), + {{0, 2, false, true}, + {2, 1, false, false}, + {3, 1, false, false}, + {4, 2, false, false}, + {6, 2, false, false}, + {8, 1, true, false}, + {9, 0, true, true}}); +} + namespace { void TestRowSegmenterConstantBatch( @@ -4632,11 +4677,13 @@ void TestSegment(GroupByFunction group_by, const std::shared_ptr
& table, AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true); } +// test with empty keys, covering code in ScalarAggregateNode void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr
& table, Datum output, const std::vector& segment_keys) { TestSegment(group_by, table, output, {}, segment_keys, /*scalar=*/true); } +// test with given segment-keys and keys set to `{"key"}`, covering code in GroupByNode void TestSegmentKey(GroupByFunction group_by, const std::shared_ptr
& table, Datum output, const std::vector& segment_keys) { TestSegment(group_by, table, output, {table->GetColumnByName("key")}, segment_keys, @@ -4752,7 +4799,7 @@ TEST_P(SegmentedKeyGroupBy, SingleSegmentKeyCombined) { } // extracts one segment of the obtained (single-segment-key) table -Result> GetEmptySegmentInput( +Result> GetEmptySegmentKeysInput( std::function>()> get_table) { ARROW_ASSIGN_OR_RAISE(auto table, get_table()); auto sliced = table->Slice(0, 10); @@ -4762,12 +4809,12 @@ Result> GetEmptySegmentInput( return Table::FromChunkedStructArray(chunked); } -Result> GetEmptySegmentInputAsChunked() { - return GetEmptySegmentInput(GetSingleSegmentInputAsChunked); +Result> GetEmptySegmentKeysInputAsChunked() { + return GetEmptySegmentKeysInput(GetSingleSegmentInputAsChunked); } -Result> GetEmptySegmentInputAsCombined() { - return GetEmptySegmentInput(GetSingleSegmentInputAsCombined); +Result> GetEmptySegmentKeysInputAsCombined() { + return GetEmptySegmentKeysInput(GetSingleSegmentInputAsCombined); } // extracts the expected output for one segment @@ -4788,11 +4835,11 @@ void TestEmptySegmentKey(GroupByFunction group_by, } TEST_P(SegmentedKeyGroupBy, EmptySegmentKeyChunked) { - TestEmptySegmentKey(GetParam(), GetEmptySegmentInputAsChunked); + TestEmptySegmentKey(GetParam(), GetEmptySegmentKeysInputAsChunked); } TEST_P(SegmentedKeyGroupBy, EmptySegmentKeyCombined) { - TestEmptySegmentKey(GetParam(), GetEmptySegmentInputAsCombined); + TestEmptySegmentKey(GetParam(), GetEmptySegmentKeysInputAsCombined); } // adds a named copy of the last (single-segment-key) column to the obtained table From b11a6cab5d563ae2f1c86d4ac5f9e7d7fede132c Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Wed, 8 Mar 2023 03:24:07 -0500 Subject: [PATCH 25/27] requested changes --- cpp/src/arrow/compute/exec/aggregate_node.cc | 21 +++++++++----------- cpp/src/arrow/compute/exec/options.h | 2 -- 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index 1fe6cf9086e..62d4ac81d70 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -46,9 +46,9 @@ // segment-keys is used to refine the partitioning. However, segment-keys are different in // that they partition only consecutive rows into a single group. Such a partition of // consecutive rows is called a segment group. For example, consider a column X with -// values [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X] -// yields a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation -// with segment-keys [X] yields [[0], [1], [2]]. +// values [A, A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys +// [X] yields a row-index partitioning [[0, 1, 3], [2]] whereas a segmented-group-by +// aggregation with segment-keys [X] yields [[0, 1], [1], [3]]. // // The implementation first segments the input using the segment-keys, then groups by the // keys. When a segment group end is reached while scanning the input, output is pushed @@ -185,11 +185,11 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema, *ss << ']'; } -// Handle the input batch -// If a segment is closed by this batch, then we output the aggregation for the segment -// If a segment is not closed by this batch, then we add the batch to the segment +// Extract segments from a batch and run the given handler on them. Note that the +// handle may be called on open segments which are not yet finished. Typically a +// handler should accumulate those open segments until a closed segment is reached. template -Status HandleSegments(std::unique_ptr& segmenter, const ExecBatch& batch, +Status HandleSegments(RowSegmenter* segmenter, const ExecBatch& batch, const std::vector& ids, const BatchHandler& handle_batch) { int64_t offset = 0; ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids)); @@ -401,7 +401,6 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); // We add segment to the current segment group aggregation - // GH-34475: change to zero-copy slicing auto exec_batch = full_batch.Slice(segment.offset, segment.length); RETURN_NOT_OK(DoConsume(ExecSpan(exec_batch), thread_index)); RETURN_NOT_OK( @@ -413,7 +412,7 @@ class ScalarAggregateNode : public ExecNode, public TracedNode { return Status::OK(); }; - RETURN_NOT_OK(HandleSegments(segmenter_, batch, segment_field_ids_, handler)); + RETURN_NOT_OK(HandleSegments(segmenter_.get(), batch, segment_field_ids_, handler)); if (input_counter_.Increment()) { RETURN_NOT_OK(OutputResult(/*is_last=*/true)); @@ -818,8 +817,6 @@ class GroupByNode : public ExecNode, public TracedNode { auto handler = [this](const ExecBatch& full_batch, const Segment& segment) { if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false)); - // This is not zero copy - we should refactor the code to pass - // offset and length to Consume to avoid copying here auto exec_batch = full_batch.Slice(segment.offset, segment.length); auto batch = ExecSpan(exec_batch); RETURN_NOT_OK(Consume(batch)); @@ -829,7 +826,7 @@ class GroupByNode : public ExecNode, public TracedNode { return Status::OK(); }; ARROW_RETURN_NOT_OK( - HandleSegments(segmenter_, batch, segment_key_field_ids_, handler)); + HandleSegments(segmenter_.get(), batch, segment_key_field_ids_, handler)); if (input_counter_.Increment()) { ARROW_RETURN_NOT_OK(OutputResult(/*is_last=*/true)); diff --git a/cpp/src/arrow/compute/exec/options.h b/cpp/src/arrow/compute/exec/options.h index 2d46e9fd86a..419990407d0 100644 --- a/cpp/src/arrow/compute/exec/options.h +++ b/cpp/src/arrow/compute/exec/options.h @@ -215,8 +215,6 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions { /// described above, applies. /// /// The keys and segment_keys vectors must be disjoint. -/// -/// See also doc in `aggregate_node.cc` class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions { public: explicit AggregateNodeOptions(std::vector aggregates, From 9e094fcf99847475ac013e3b07e1ecb5df0f151b Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Wed, 8 Mar 2023 11:05:41 -0500 Subject: [PATCH 26/27] doc and simplify segmenter grouping --- cpp/src/arrow/compute/row/grouper.cc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index 3681f80561a..aeb8dbb4b4e 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -278,9 +278,12 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { return extends; }; ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_)); // TODO: reset it + // GH-34475: cache the grouper-consume result across invocations of GetNextSegment ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset)); if (datum.is_array()) { + // `data` is an array whose index-0 corresponds to index `offset` of `batch` const std::shared_ptr& data = datum.array(); + DCHECK_EQ(data->length, batch.length - offset); ARROW_DCHECK(data->GetNullCount() == 0); DCHECK_EQ(data->type->id(), GroupIdType::type_id); const group_id_t* values = data->GetValues(1); @@ -288,7 +291,7 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { for (cursor = 1; cursor < data->length; cursor++) { if (values[0] != values[cursor]) break; } - int64_t length = std::min(cursor, batch.length - offset); + int64_t length = cursor; bool extends = length > 0 ? bound_extend(values) : kEmptyExtends; return MakeSegment(batch.length, offset, length, extends); } else { From 6295691cac20a00b93a8a51ccda1ed9300ea3e1a Mon Sep 17 00:00:00 2001 From: Yaron Gvili Date: Wed, 8 Mar 2023 16:46:32 -0500 Subject: [PATCH 27/27] requested changes --- cpp/src/arrow/compute/kernels/hash_aggregate_test.cc | 3 +++ cpp/src/arrow/compute/row/grouper.cc | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 6c45299d763..fd631e0dc51 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -200,6 +200,9 @@ Result RunGroupBy(const BatchesWithSchema& input, const std::vector& segment_key_names, const std::vector& aggregates, ExecContext* ctx, bool use_threads, bool segmented = false, bool naive = false) { + // The `use_threads` flag determines whether threads are used in generating the input to + // the group-by. + // // When segment_keys is non-empty the `segmented` flag is always true; otherwise (when // empty), it may still be set to true. In this case, the tester restructures (without // changing the data of) the result of RunGroupBy from `std::vector` diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index aeb8dbb4b4e..75df42abd0f 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -267,7 +267,7 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { return MakeSegment(batch.length, offset, 0, kEmptyExtends); } // ARROW-18311: make Grouper support Reset() - // so it can be cached instead of recreated below + // so it can be reset instead of recreated below // // the group id must be computed prior to resetting the grouper, since it is compared // to save_group_id_, and after resetting the grouper produces incomparable group ids @@ -277,6 +277,7 @@ struct AnyKeysSegmenter : public BaseRowSegmenter { save_group_id_ = *static_cast(data); return extends; }; + // resetting drops grouper's group-ids, freeing-up memory for the next segment ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_)); // TODO: reset it // GH-34475: cache the grouper-consume result across invocations of GetNextSegment ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));