From 32cfbb4dcd85004a11a6efcc0edac39fc656cbb4 Mon Sep 17 00:00:00 2001 From: michalursa Date: Thu, 11 Feb 2021 15:39:25 -0800 Subject: [PATCH 01/49] ARROW-11591: [C++] Prototype version of hash group by --- cpp/src/arrow/compute/api_aggregate.h | 9 ++ cpp/src/arrow/compute/exec.cc | 14 ++- cpp/src/arrow/compute/kernel.h | 10 +- .../arrow/compute/kernels/aggregate_basic.cc | 115 +++++++++++++++++- .../compute/kernels/aggregate_internal.h | 2 +- .../arrow/compute/kernels/aggregate_test.cc | 115 ++++++++++++++++++ 6 files changed, 254 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index eef1587bb73..6998a8c934e 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -149,6 +149,15 @@ struct ARROW_EXPORT TDigestOptions : public FunctionOptions { uint32_t buffer_size; }; +// TODO(michalursa) add docstring +struct ARROW_EXPORT GroupByOptions : public FunctionOptions { + struct Aggregate { + std::string name; + const FunctionOptions* options; + }; + std::vector aggregates; +}; + /// @} /// \brief Count non-null (or null) values in an array. diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index 6443c96e918..60e77fd40d7 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -850,11 +850,17 @@ class ScalarAggExecutor : public KernelExecutorImpl { KernelContext batch_ctx(exec_context()); batch_ctx.SetState(batch_state.get()); - kernel_->consume(&batch_ctx, batch); - ARROW_CTX_RETURN_IF_ERROR(&batch_ctx); + if (kernel_->nomerge) { + kernel_->consume(kernel_ctx_, batch); + ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + } else { + kernel_->consume(&batch_ctx, batch); + ARROW_CTX_RETURN_IF_ERROR(&batch_ctx); + + kernel_->merge(kernel_ctx_, std::move(*batch_state), state()); + ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + } - kernel_->merge(kernel_ctx_, std::move(*batch_state), state()); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); return Status::OK(); } diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index c8f9cacfb34..a745cca35ef 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -684,21 +684,23 @@ struct ScalarAggregateKernel : public Kernel { ScalarAggregateKernel(std::shared_ptr sig, KernelInit init, ScalarAggregateConsume consume, ScalarAggregateMerge merge, - ScalarAggregateFinalize finalize) + ScalarAggregateFinalize finalize, bool nomerge = false) : Kernel(std::move(sig), init), consume(std::move(consume)), merge(std::move(merge)), - finalize(std::move(finalize)) {} + finalize(std::move(finalize)), + nomerge(nomerge) {} ScalarAggregateKernel(std::vector in_types, OutputType out_type, KernelInit init, ScalarAggregateConsume consume, - ScalarAggregateMerge merge, ScalarAggregateFinalize finalize) + ScalarAggregateMerge merge, ScalarAggregateFinalize finalize, bool nomerge = false) : ScalarAggregateKernel(KernelSignature::Make(std::move(in_types), out_type), init, - consume, merge, finalize) {} + consume, merge, finalize, nomerge) {} ScalarAggregateConsume consume; ScalarAggregateMerge merge; ScalarAggregateFinalize finalize; + bool nomerge; }; } // namespace compute diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 5cdd3bd1dd1..425d174c6c1 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -21,6 +21,7 @@ #include "arrow/compute/kernels/common.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" +#include namespace arrow { namespace compute { @@ -42,9 +43,9 @@ void AggregateFinalize(KernelContext* ctx, Datum* out) { } // namespace void AddAggKernel(std::shared_ptr sig, KernelInit init, - ScalarAggregateFunction* func, SimdLevel::type simd_level) { + ScalarAggregateFunction* func, SimdLevel::type simd_level, bool nomerge) { ScalarAggregateKernel kernel(std::move(sig), init, AggregateConsume, AggregateMerge, - AggregateFinalize); + AggregateFinalize, nomerge); // Set the simd level kernel.simd_level = simd_level; DCHECK_OK(func->AddKernel(kernel)); @@ -203,6 +204,102 @@ std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) return ::arrow::internal::make_unique(); } +struct GroupByImpl : public ScalarAggregator { + void Consume(KernelContext* ctx, const ExecBatch& batch) override { + std::vector> aggregands, keys; + + size_t i; + for (i = 0; i < aggregates.size(); ++i) { + aggregands.push_back(batch[i].array()); + } + while (i < static_cast(batch.num_values())) { + keys.push_back(batch[i++].array()); + } + + auto key64 = batch[aggregates.size()].array_as(); + if (key64->null_count() != 0) { + ctx->SetStatus(Status::NotImplemented("nulls in key column")); + return; + } + + const int64_t* key64_raw = key64->raw_values(); + + auto valuesDouble = batch[0].array_as(); + const double* valuesDouble_raw = valuesDouble->raw_values(); + + for (int64_t i = 0; i < batch.length; ++i) { + uint64_t key = key64_raw[i]; + double value = valuesDouble_raw[i]; + uint32_t groupid; + auto iter = map_.find(key); + if (iter == map_.end()) { + groupid = static_cast(keys_.size()); + keys_.push_back(key); + sums_.push_back(0.0); + map_.insert(std::make_pair(key, groupid)); + } else { + groupid = iter->second; + } + sums_[groupid] += value; + } + } + + void MergeFrom(KernelContext* ctx, KernelState&& src) override { + // TODO(michalursa) merge two hash tables + } + + void Finalize(KernelContext* ctx, Datum* out) override { + auto pool = ctx->memory_pool(); + size_t length = keys_.size(); + auto out_buffer = std::move(AllocateBuffer(sizeof(double) * length, pool)).ValueUnsafe(); + auto out_values = out_buffer->mutable_data(); + for (size_t i = 0; i < length; ++i) { + (reinterpret_cast(out_values))[i] = sums_[i]; + } + std::shared_ptr null_bitmap = nullptr; + Datum datum_sum = ArrayData::Make(float64(), length, {null_bitmap, std::move(out_buffer)}, 0); + + auto out_buffer_key = std::move(AllocateBuffer(sizeof(int64_t) * length, pool)).ValueUnsafe(); + auto out_keys = out_buffer_key->mutable_data(); + for (size_t i = 0; i < length; ++i) { + (reinterpret_cast(out_keys))[i] = keys_[i]; + } + std::shared_ptr null_bitmap_key = nullptr; + Datum datum_key = ArrayData::Make(int64(), length, {null_bitmap_key, std::move(out_buffer_key)}, 0); + + *out = Datum({std::move(datum_sum), std::move(datum_key)}); + } + + std::map map_; + std::vector keys_; + std::vector sums_; + std::vector aggregates; +}; + +std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArgs& args) { + // TODO(michalursa) do construction of group by implementation + auto impl = ::arrow::internal::make_unique(); + impl->aggregates = checked_cast(args.options)->aggregates; + + if (impl->aggregates.size() > args.inputs.size()) { + ctx->SetStatus(Status::Invalid("more aggegates than inputs!")); + return nullptr; + } + + size_t n_keys = args.inputs.size() - impl->aggregates.size(); + if (n_keys != 1) { + ctx->SetStatus(Status::NotImplemented("more than one key")); + return nullptr; + } + + if (args.inputs.back().type->id() != Type::INT64) { + ctx->SetStatus(Status::NotImplemented("key of type", args.inputs.back().type->ToString())); + return nullptr; + } + + return impl; +} + void AddBasicAggKernels(KernelInit init, const std::vector>& types, std::shared_ptr out_ty, ScalarAggregateFunction* func, @@ -260,6 +357,9 @@ const FunctionDoc all_doc{ ("Null values are ignored."), {"array"}}; +// TODO(michalursa) add FunctionDoc for group_by +const FunctionDoc group_by_doc{"", (""), {}}; + } // namespace void RegisterScalarAggregateBasic(FunctionRegistry* registry) { @@ -342,6 +442,17 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) { func = std::make_shared("all", Arity::Unary(), &all_doc); aggregate::AddBasicAggKernels(aggregate::AllInit, {boolean()}, boolean(), func.get()); DCHECK_OK(registry->AddFunction(std::move(func))); + + // group_by + func = std::make_shared("group_by", Arity::VarArgs(), &group_by_doc); + // aggregate::AddBasicAggKernels(aggregate::GroupByInit, {null()}, null(), func.get()); + { + InputType any_array(ValueDescr::ARRAY); + auto sig = KernelSignature::Make({any_array}, ValueDescr::Array(int64()), true); + AddAggKernel(std::move(sig), aggregate::GroupByInit, func.get(), SimdLevel::NONE, true); + } + DCHECK_OK(registry->AddFunction(std::move(func))); + // TODO(michalursa) add Kernels to the function named "group_by" } } // namespace internal diff --git a/cpp/src/arrow/compute/kernels/aggregate_internal.h b/cpp/src/arrow/compute/kernels/aggregate_internal.h index 67337f22c5b..c1757c97ffe 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/aggregate_internal.h @@ -57,7 +57,7 @@ struct ScalarAggregator : public KernelState { void AddAggKernel(std::shared_ptr sig, KernelInit init, ScalarAggregateFunction* func, - SimdLevel::type simd_level = SimdLevel::NONE); + SimdLevel::type simd_level = SimdLevel::NONE, bool nomerge = false); namespace detail { diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 569886a1351..234cb1cba2d 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -45,6 +45,121 @@ using internal::checked_pointer_cast; namespace compute { +// TODO(michalursa) add tests +TEST(GroupBy, SumOnly) { + auto key = ArrayFromJSON(int64(), + "[1, 2, 1," + "3, 2, 3]"); + auto aggregand = ArrayFromJSON(float64(), + "[1.0, 0.0, null, " + "3.25, 0.125, -0.25]"); + + ASSERT_EQ(key->length(), aggregand->length()); + + GroupByOptions options; + options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr}}; + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, CallFunction( + "group_by", { + aggregand, + key, + }, + &options + )); + + Datum summed = aggregated_and_grouped.collection()[0]; + Datum unique_keys = aggregated_and_grouped.collection()[1]; + + auto unique_keys_i64 = unique_keys.array_as(); + const int64_t* raw_values = unique_keys_i64->raw_values(); + ASSERT_EQ(unique_keys_i64->length(), 3); + auto summed_f64 = summed.array_as(); + + for (int64_t i = 0; i < unique_keys_i64->length(); ++i) { + int64_t key = unique_keys_i64->Value(i); + + if (key == 1) { + ASSERT_EQ(summed_f64->Value(i), 1.0); + } + if (key == 2) { + ASSERT_EQ(summed_f64->Value(i), 0.125); + } + if (key == 3) { + ASSERT_EQ(summed_f64->Value(i), 3.0); + } + } +} + +TEST(GroupBy, NoAggs) { + auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); + + GroupByOptions options; + options.aggregates = {}; + + // TODO: not implemented yet + /* + ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( + "group_by", { + key, + }, + &options + )); + */ +} + +TEST(GroupBy, CountOnly) { + auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 0.3, 0.1, -0.2]"); + + ASSERT_EQ(key->length(), aggregand->length()); + + CountOptions count_options{CountOptions::COUNT_NON_NULL}; + GroupByOptions options; + options.aggregates = { + { + GroupByOptions::Aggregate{"count", &count_options}, + } + }; + // TODO: not implemented yet + /* + ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( + "group_by", { + key, + key, + }, + &options + )); + */ +} + +TEST(GroupBy, SumAndCount) { + auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 0.3, 0.1, -0.2]"); + + ASSERT_EQ(key->length(), aggregand->length()); + + CountOptions count_options{CountOptions::COUNT_NON_NULL}; + GroupByOptions options; + options.aggregates = { + { + GroupByOptions::Aggregate{"sum", nullptr}, + GroupByOptions::Aggregate{"count", &count_options}, + } + }; + + // TODO: not implemented yet + /* + ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( + "group_by", { + aggregand, + key, + key, + }, + &options + )); + */ +} + // // Sum // From 3f657264cb978f03f66b402a9697ab07882ecd41 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 16 Feb 2021 17:36:39 -0500 Subject: [PATCH 02/49] extract sum and count to GroupedAggregator interface --- cpp/src/arrow/compute/api_aggregate.h | 11 +- .../arrow/compute/kernels/aggregate_basic.cc | 191 ++++++++++++++---- .../arrow/compute/kernels/aggregate_test.cc | 141 ++++++------- 3 files changed, 221 insertions(+), 122 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 6998a8c934e..ab0b3ae1941 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -152,10 +152,19 @@ struct ARROW_EXPORT TDigestOptions : public FunctionOptions { // TODO(michalursa) add docstring struct ARROW_EXPORT GroupByOptions : public FunctionOptions { struct Aggregate { - std::string name; + /// the name of the aggregation function + std::string function; + + /// options for the aggregation function const FunctionOptions* options; + + /// the name of the resulting column in output + std::string name; }; std::vector aggregates; + + /// the names of key columns + std::vector key_names; }; /// @} diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 425d174c6c1..0f946b9b134 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. +#include #include "arrow/compute/api_aggregate.h" #include "arrow/compute/kernels/aggregate_basic_internal.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/common.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" -#include namespace arrow { namespace compute { @@ -43,7 +43,8 @@ void AggregateFinalize(KernelContext* ctx, Datum* out) { } // namespace void AddAggKernel(std::shared_ptr sig, KernelInit init, - ScalarAggregateFunction* func, SimdLevel::type simd_level, bool nomerge) { + ScalarAggregateFunction* func, SimdLevel::type simd_level, + bool nomerge) { ScalarAggregateKernel kernel(std::move(sig), init, AggregateConsume, AggregateMerge, AggregateFinalize, nomerge); // Set the simd level @@ -92,6 +93,104 @@ struct CountImpl : public ScalarAggregator { int64_t nulls = 0; }; +struct GroupedAggregator { + // GroupedAggregator subclasses are expected to be constructible from + // const FunctionOptions*. Will probably need an Init method as well + virtual ~GroupedAggregator() = default; + + virtual void Consume(KernelContext*, const ExecBatch& batch, + const uint32_t* group_ids) = 0; + + virtual void Finalize(KernelContext* ctx, Datum* out) = 0; + + static Result> Make(std::string function, + const FunctionOptions* options); +}; + +struct GroupedCountImpl : public GroupedAggregator { + explicit GroupedCountImpl(const FunctionOptions* options) + : options(checked_cast(*options)) {} + + void Consume(KernelContext* ctx, const ExecBatch& batch, + const uint32_t* group_ids) override { + if (batch.length == 0) return; + + // maybe a batch of group_ids should include the min/max group id + auto max_group = *std::max_element(group_ids, group_ids + batch.length); + if (max_group >= counts.size()) { + counts.resize(max_group + 1, 0); + } + + if (options.count_mode == CountOptions::COUNT_NON_NULL) { + auto input = batch[0].make_array(); + + for (int64_t i = 0; i < input->length(); ++i) { + if (input->IsNull(i)) continue; + counts[group_ids[i]]++; + } + } else { + for (int64_t i = 0; i < batch.length; ++i) { + counts[group_ids[i]]++; + } + } + } + + void Finalize(KernelContext* ctx, Datum* out) override { + KERNEL_ASSIGN_OR_RAISE(auto counts_buf, ctx, + ctx->Allocate(sizeof(int64_t) * counts.size())); + std::copy(counts.begin(), counts.end(), + reinterpret_cast(counts_buf->mutable_data())); + *out = std::make_shared(counts.size(), std::move(counts_buf)); + } + + CountOptions options; + std::vector counts; +}; + +struct GroupedSumImpl : public GroupedAggregator { + explicit GroupedSumImpl(const FunctionOptions*) {} + + void Consume(KernelContext* ctx, const ExecBatch& batch, + const uint32_t* group_ids) override { + if (batch.length == 0) return; + + // maybe a batch of group_ids should include the min/max group id + auto max_group = *std::max_element(group_ids, group_ids + batch.length); + if (max_group >= sums.size()) { + sums.resize(max_group + 1, 0.0); + } + + DCHECK_EQ(batch[0].type()->id(), Type::DOUBLE); + auto input = batch[0].array_as(); + + for (int64_t i = 0; i < input->length(); ++i) { + if (input->IsNull(i)) continue; + sums[group_ids[i]] += input->Value(i); + } + } + + void Finalize(KernelContext* ctx, Datum* out) override { + KERNEL_ASSIGN_OR_RAISE(auto sums_buf, ctx, + ctx->Allocate(sizeof(double) * sums.size())); + std::copy(sums.begin(), sums.end(), + reinterpret_cast(sums_buf->mutable_data())); + *out = std::make_shared(sums.size(), std::move(sums_buf)); + } + + std::vector sums; +}; + +Result> GroupedAggregator::Make( + std::string function, const FunctionOptions* options) { + if (function == "count") { + return ::arrow::internal::make_unique(options); + } + if (function == "sum") { + return ::arrow::internal::make_unique(options); + } + return Status::NotImplemented("Grouped aggregate ", function); +} + std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique( static_cast(*args.options)); @@ -206,17 +305,17 @@ std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) struct GroupByImpl : public ScalarAggregator { void Consume(KernelContext* ctx, const ExecBatch& batch) override { - std::vector> aggregands, keys; + ArrayDataVector aggregands, keys; size_t i; - for (i = 0; i < aggregates.size(); ++i) { + for (i = 0; i < aggregators.size(); ++i) { aggregands.push_back(batch[i].array()); } while (i < static_cast(batch.num_values())) { keys.push_back(batch[i++].array()); } - auto key64 = batch[aggregates.size()].array_as(); + auto key64 = batch[aggregators.size()].array_as(); if (key64->null_count() != 0) { ctx->SetStatus(Status::NotImplemented("nulls in key column")); return; @@ -224,23 +323,23 @@ struct GroupByImpl : public ScalarAggregator { const int64_t* key64_raw = key64->raw_values(); - auto valuesDouble = batch[0].array_as(); - const double* valuesDouble_raw = valuesDouble->raw_values(); - + std::vector group_ids(batch.length); for (int64_t i = 0; i < batch.length; ++i) { uint64_t key = key64_raw[i]; - double value = valuesDouble_raw[i]; - uint32_t groupid; auto iter = map_.find(key); if (iter == map_.end()) { - groupid = static_cast(keys_.size()); + group_ids[i] = static_cast(keys_.size()); keys_.push_back(key); - sums_.push_back(0.0); - map_.insert(std::make_pair(key, groupid)); + map_.insert(std::make_pair(key, group_ids[i])); } else { - groupid = iter->second; + group_ids[i] = iter->second; } - sums_[groupid] += value; + } + + for (size_t i = 0; i < aggregators.size(); ++i) { + ExecBatch aggregand_batch{{aggregands[i]}, batch.length}; + aggregators[i]->Consume(ctx, aggregand_batch, group_ids.data()); + if (ctx->HasError()) return; } } @@ -249,51 +348,63 @@ struct GroupByImpl : public ScalarAggregator { } void Finalize(KernelContext* ctx, Datum* out) override { - auto pool = ctx->memory_pool(); - size_t length = keys_.size(); - auto out_buffer = std::move(AllocateBuffer(sizeof(double) * length, pool)).ValueUnsafe(); - auto out_values = out_buffer->mutable_data(); - for (size_t i = 0; i < length; ++i) { - (reinterpret_cast(out_values))[i] = sums_[i]; + FieldVector out_fields(aggregators.size() + 1); + ArrayDataVector out_columns(aggregators.size() + 1); + for (size_t i = 0; i < aggregators.size(); ++i) { + Datum aggregand; + aggregators[i]->Finalize(ctx, &aggregand); + if (ctx->HasError()) return; + out_columns[i] = aggregand.array(); + out_fields[i] = field(options.aggregates[i].name, aggregand.type()); } - std::shared_ptr null_bitmap = nullptr; - Datum datum_sum = ArrayData::Make(float64(), length, {null_bitmap, std::move(out_buffer)}, 0); - auto out_buffer_key = std::move(AllocateBuffer(sizeof(int64_t) * length, pool)).ValueUnsafe(); - auto out_keys = out_buffer_key->mutable_data(); - for (size_t i = 0; i < length; ++i) { - (reinterpret_cast(out_keys))[i] = keys_[i]; - } - std::shared_ptr null_bitmap_key = nullptr; - Datum datum_key = ArrayData::Make(int64(), length, {null_bitmap_key, std::move(out_buffer_key)}, 0); + int64_t length = keys_.size(); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(sizeof(int64_t) * length)); + std::copy(keys_.begin(), keys_.end(), + reinterpret_cast(key_buf->mutable_data())); + auto key = std::make_shared(length, std::move(key_buf)); + + out_columns.back() = key->data(); + out_fields.back() = field(options.key_names[0], key->type()); - *out = Datum({std::move(datum_sum), std::move(datum_key)}); + *out = ArrayData::Make(struct_(std::move(out_fields)), key->length(), + {/*null_bitmap=*/nullptr}, std::move(out_columns)); } std::map map_; std::vector keys_; - std::vector sums_; - std::vector aggregates; + + GroupByOptions options; + std::vector> aggregators; }; std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArgs& args) { // TODO(michalursa) do construction of group by implementation auto impl = ::arrow::internal::make_unique(); - impl->aggregates = checked_cast(args.options)->aggregates; + impl->options = *checked_cast(args.options); + const auto& aggregates = impl->options.aggregates; - if (impl->aggregates.size() > args.inputs.size()) { + if (aggregates.size() > args.inputs.size()) { ctx->SetStatus(Status::Invalid("more aggegates than inputs!")); return nullptr; } - size_t n_keys = args.inputs.size() - impl->aggregates.size(); + impl->aggregators.resize(aggregates.size()); + for (size_t i = 0; i < aggregates.size(); ++i) { + ctx->SetStatus(GroupedAggregator::Make(aggregates[i].function, aggregates[i].options) + .Value(&impl->aggregators[i])); + if (ctx->HasError()) return nullptr; + } + + size_t n_keys = args.inputs.size() - aggregates.size(); if (n_keys != 1) { ctx->SetStatus(Status::NotImplemented("more than one key")); return nullptr; } if (args.inputs.back().type->id() != Type::INT64) { - ctx->SetStatus(Status::NotImplemented("key of type", args.inputs.back().type->ToString())); + ctx->SetStatus( + Status::NotImplemented("key of type", args.inputs.back().type->ToString())); return nullptr; } @@ -444,12 +555,14 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) { DCHECK_OK(registry->AddFunction(std::move(func))); // group_by - func = std::make_shared("group_by", Arity::VarArgs(), &group_by_doc); + func = std::make_shared("group_by", Arity::VarArgs(), + &group_by_doc); // aggregate::AddBasicAggKernels(aggregate::GroupByInit, {null()}, null(), func.get()); { InputType any_array(ValueDescr::ARRAY); auto sig = KernelSignature::Make({any_array}, ValueDescr::Array(int64()), true); - AddAggKernel(std::move(sig), aggregate::GroupByInit, func.get(), SimdLevel::NONE, true); + AddAggKernel(std::move(sig), aggregate::GroupByInit, func.get(), SimdLevel::NONE, + true); } DCHECK_OK(registry->AddFunction(std::move(func))); // TODO(michalursa) add Kernels to the function named "group_by" diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 234cb1cba2d..1d8a4b88b41 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -48,116 +48,93 @@ namespace compute { // TODO(michalursa) add tests TEST(GroupBy, SumOnly) { auto key = ArrayFromJSON(int64(), - "[1, 2, 1," - "3, 2, 3]"); + "[1, 2, 1," + "3, 2, 3]"); auto aggregand = ArrayFromJSON(float64(), - "[1.0, 0.0, null, " - "3.25, 0.125, -0.25]"); + "[1.0, 0.0, null, " + "3.25, 0.125, -0.25]"); ASSERT_EQ(key->length(), aggregand->length()); GroupByOptions options; - options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr}}; + options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr, "f64 summed"}}; + options.key_names = {"i64 key"}; - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, CallFunction( - "group_by", { - aggregand, - key, - }, - &options - )); + ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", + { + aggregand, + key, + }, + &options)); - Datum summed = aggregated_and_grouped.collection()[0]; - Datum unique_keys = aggregated_and_grouped.collection()[1]; + auto aggregated_and_grouped = boxed.array_as(); - auto unique_keys_i64 = unique_keys.array_as(); - const int64_t* raw_values = unique_keys_i64->raw_values(); - ASSERT_EQ(unique_keys_i64->length(), 3); - auto summed_f64 = summed.array_as(); + auto f64_summed = checked_pointer_cast( + aggregated_and_grouped->GetFieldByName("f64 summed")); - for (int64_t i = 0; i < unique_keys_i64->length(); ++i) { - int64_t key = unique_keys_i64->Value(i); + auto i64_key = + checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); + + ASSERT_EQ(i64_key->length(), 3); + + for (int64_t i = 0; i < i64_key->length(); ++i) { + int64_t key = i64_key->Value(i); if (key == 1) { - ASSERT_EQ(summed_f64->Value(i), 1.0); + ASSERT_EQ(f64_summed->Value(i), 1.0); } if (key == 2) { - ASSERT_EQ(summed_f64->Value(i), 0.125); + ASSERT_EQ(f64_summed->Value(i), 0.125); } if (key == 3) { - ASSERT_EQ(summed_f64->Value(i), 3.0); + ASSERT_EQ(f64_summed->Value(i), 3.0); } } } -TEST(GroupBy, NoAggs) { - auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); - - GroupByOptions options; - options.aggregates = {}; - - // TODO: not implemented yet - /* - ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( - "group_by", { - key, - }, - &options - )); - */ -} - TEST(GroupBy, CountOnly) { - auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 0.3, 0.1, -0.2]"); + auto key = ArrayFromJSON(int64(), + "[1, 2, 1," + "3, 2, 3]"); + auto aggregand = ArrayFromJSON(float64(), + "[1.0, 0.0, null, " + "3.25, 0.125, -0.25]"); ASSERT_EQ(key->length(), aggregand->length()); - CountOptions count_options{CountOptions::COUNT_NON_NULL}; GroupByOptions options; + CountOptions count_options; + count_options.count_mode = CountOptions::COUNT_NON_NULL; options.aggregates = { - { - GroupByOptions::Aggregate{"count", &count_options}, - } - }; - // TODO: not implemented yet - /* - ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( - "group_by", { - key, - key, - }, - &options - )); - */ -} - -TEST(GroupBy, SumAndCount) { - auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3]"); - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 0.3, 0.1, -0.2]"); + GroupByOptions::Aggregate{"count", &count_options, "f64 counted"}}; + options.key_names = {"i64 key"}; - ASSERT_EQ(key->length(), aggregand->length()); + ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", + { + aggregand, + key, + }, + &options)); - CountOptions count_options{CountOptions::COUNT_NON_NULL}; - GroupByOptions options; - options.aggregates = { - { - GroupByOptions::Aggregate{"sum", nullptr}, - GroupByOptions::Aggregate{"count", &count_options}, - } - }; + auto aggregated_and_grouped = boxed.array_as(); + + auto f64_counted = checked_pointer_cast( + aggregated_and_grouped->GetFieldByName("f64 counted")); + + auto i64_key = + checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); - // TODO: not implemented yet - /* - ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, CallFunction( - "group_by", { - aggregand, - key, - key, - }, - &options - )); - */ + ASSERT_EQ(i64_key->length(), 3); + + for (int64_t i = 0; i < i64_key->length(); ++i) { + int64_t key = i64_key->Value(i); + + if (key == 1) { + ASSERT_EQ(f64_counted->Value(i), 1); + } else { + ASSERT_EQ(f64_counted->Value(i), 2); + } + } } // From 3792de8d99be4bde66c8adedb3028ba9ad315287 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Feb 2021 10:11:56 -0500 Subject: [PATCH 03/49] implement sum for more DataTypes --- cpp/src/arrow/buffer_builder.h | 18 ++ cpp/src/arrow/compute/kernel.h | 3 +- .../arrow/compute/kernels/aggregate_basic.cc | 248 +++++++++++++----- .../arrow/compute/kernels/aggregate_test.cc | 72 +++++ 4 files changed, 277 insertions(+), 64 deletions(-) diff --git a/cpp/src/arrow/buffer_builder.h b/cpp/src/arrow/buffer_builder.h index 41a47c91729..7a33ff85cd0 100644 --- a/cpp/src/arrow/buffer_builder.h +++ b/cpp/src/arrow/buffer_builder.h @@ -162,6 +162,12 @@ class ARROW_EXPORT BufferBuilder { return Status::OK(); } + Result> Finish(bool shrink_to_fit = true) { + std::shared_ptr out; + ARROW_RETURN_NOT_OK(Finish(&out, shrink_to_fit)); + return out; + } + void Reset() { buffer_ = NULLPTR; capacity_ = size_ = 0; @@ -256,6 +262,12 @@ class TypedBufferBuilder< return bytes_builder_.Finish(out, shrink_to_fit); } + Result> Finish(bool shrink_to_fit = true) { + std::shared_ptr out; + ARROW_RETURN_NOT_OK(Finish(&out, shrink_to_fit)); + return out; + } + void Reset() { bytes_builder_.Reset(); } int64_t length() const { return bytes_builder_.length() / sizeof(T); } @@ -371,6 +383,12 @@ class TypedBufferBuilder { return bytes_builder_.Finish(out, shrink_to_fit); } + Result> Finish(bool shrink_to_fit = true) { + std::shared_ptr out; + ARROW_RETURN_NOT_OK(Finish(&out, shrink_to_fit)); + return out; + } + void Reset() { bytes_builder_.Reset(); bit_length_ = false_count_ = 0; diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index a745cca35ef..40fe996c52b 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -693,7 +693,8 @@ struct ScalarAggregateKernel : public Kernel { ScalarAggregateKernel(std::vector in_types, OutputType out_type, KernelInit init, ScalarAggregateConsume consume, - ScalarAggregateMerge merge, ScalarAggregateFinalize finalize, bool nomerge = false) + ScalarAggregateMerge merge, ScalarAggregateFinalize finalize, + bool nomerge = false) : ScalarAggregateKernel(KernelSignature::Make(std::move(in_types), out_type), init, consume, merge, finalize, nomerge) {} diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 0f946b9b134..6b57c2bf0a2 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -16,10 +16,12 @@ // under the License. #include + #include "arrow/compute/api_aggregate.h" #include "arrow/compute/kernels/aggregate_basic_internal.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/common.h" +#include "arrow/util/bit_run_reader.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" @@ -94,8 +96,6 @@ struct CountImpl : public ScalarAggregator { }; struct GroupedAggregator { - // GroupedAggregator subclasses are expected to be constructible from - // const FunctionOptions*. Will probably need an Init method as well virtual ~GroupedAggregator() = default; virtual void Consume(KernelContext*, const ExecBatch& batch, @@ -103,93 +103,179 @@ struct GroupedAggregator { virtual void Finalize(KernelContext* ctx, Datum* out) = 0; - static Result> Make(std::string function, - const FunctionOptions* options); + virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; + + virtual int64_t num_groups() const = 0; + + void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { + if (length == 0) return; + + // maybe a batch of group_ids should include the min/max group id + int64_t max_group = *std::max_element(group_ids, group_ids + length); + auto old_size = num_groups(); + + if (max_group >= old_size) { + auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); + Resize(ctx, new_size); + } + } }; struct GroupedCountImpl : public GroupedAggregator { - explicit GroupedCountImpl(const FunctionOptions* options) - : options(checked_cast(*options)) {} + static std::unique_ptr Make(KernelContext* ctx, + const DataType& input_type, + const FunctionOptions* options) { + auto out = ::arrow::internal::make_unique(); + out->options = checked_cast(*options); + ctx->SetStatus(ctx->Allocate(0).Value(&out->counts)); + return out; + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups(); + KERNEL_RETURN_IF_ERROR(ctx, counts->TypedResize(new_num_groups)); + auto new_size = num_groups(); + + auto raw_counts = reinterpret_cast(counts->mutable_data()); + for (auto i = old_size; i < new_size; ++i) { + raw_counts[i] = 0; + } + } void Consume(KernelContext* ctx, const ExecBatch& batch, const uint32_t* group_ids) override { - if (batch.length == 0) return; + MaybeResize(ctx, batch.length, group_ids); + if (ctx->HasError()) return; - // maybe a batch of group_ids should include the min/max group id - auto max_group = *std::max_element(group_ids, group_ids + batch.length); - if (max_group >= counts.size()) { - counts.resize(max_group + 1, 0); - } + auto raw_counts = reinterpret_cast(counts->mutable_data()); if (options.count_mode == CountOptions::COUNT_NON_NULL) { auto input = batch[0].make_array(); for (int64_t i = 0; i < input->length(); ++i) { if (input->IsNull(i)) continue; - counts[group_ids[i]]++; + raw_counts[group_ids[i]]++; } } else { for (int64_t i = 0; i < batch.length; ++i) { - counts[group_ids[i]]++; + raw_counts[group_ids[i]]++; } } } void Finalize(KernelContext* ctx, Datum* out) override { - KERNEL_ASSIGN_OR_RAISE(auto counts_buf, ctx, - ctx->Allocate(sizeof(int64_t) * counts.size())); - std::copy(counts.begin(), counts.end(), - reinterpret_cast(counts_buf->mutable_data())); - *out = std::make_shared(counts.size(), std::move(counts_buf)); + auto length = num_groups(); + *out = std::make_shared(length, std::move(counts)); } + int64_t num_groups() const override { return counts->size() * sizeof(int64_t); } + CountOptions options; - std::vector counts; + std::shared_ptr counts; }; struct GroupedSumImpl : public GroupedAggregator { - explicit GroupedSumImpl(const FunctionOptions*) {} - - void Consume(KernelContext* ctx, const ExecBatch& batch, - const uint32_t* group_ids) override { - if (batch.length == 0) return; + // NB: whether we are accumulating into double, int64_t, or uint64_t + // we always have 64 bits per group in the sums buffer. + static constexpr size_t kSumSize = sizeof(int64_t); + + using ConsumeImpl = + std::function&, const uint32_t*, Buffer*)>; + + struct GetConsumeImpl { + template ::Type> + Status Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums) { + auto raw_input = reinterpret_cast::CType*>( + input->buffers[1]->data()); + auto raw_sums = reinterpret_cast::CType*>( + sums->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t i = begin, end = begin + length; i < end; ++i) { + raw_sums[group_ids[i]] += raw_input[i]; + } + }); + }; + out_type = TypeTraits::type_singleton(); + return Status::OK(); + } - // maybe a batch of group_ids should include the min/max group id - auto max_group = *std::max_element(group_ids, group_ids + batch.length); - if (max_group >= sums.size()) { - sums.resize(max_group + 1, 0.0); + Status Visit(const BooleanType&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums) { + auto raw_input = input->buffers[1]->data(); + auto raw_sums = reinterpret_cast(sums->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t i = begin, end = begin + length; i < end; ++i) { + raw_sums[group_ids[i]] += BitUtil::GetBit(raw_input, i); + } + }); + }; + out_type = boolean(); + return Status::OK(); } - DCHECK_EQ(batch[0].type()->id(), Type::DOUBLE); - auto input = batch[0].array_as(); + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Summing data of type ", type); + } - for (int64_t i = 0; i < input->length(); ++i) { - if (input->IsNull(i)) continue; - sums[group_ids[i]] += input->Value(i); + Status Visit(const DataType& type) { + return Status::NotImplemented("Summing data of type ", type); } - } - void Finalize(KernelContext* ctx, Datum* out) override { - KERNEL_ASSIGN_OR_RAISE(auto sums_buf, ctx, - ctx->Allocate(sizeof(double) * sums.size())); - std::copy(sums.begin(), sums.end(), - reinterpret_cast(sums_buf->mutable_data())); - *out = std::make_shared(sums.size(), std::move(sums_buf)); + ConsumeImpl consume_impl; + std::shared_ptr out_type; + }; + + static std::unique_ptr Make(KernelContext* ctx, + const DataType& input_type, + const FunctionOptions* options) { + auto out = ::arrow::internal::make_unique(); + ctx->SetStatus(ctx->Allocate(0).Value(&out->sums)); + if (ctx->HasError()) return nullptr; + + GetConsumeImpl get_consume_impl; + ctx->SetStatus(VisitTypeInline(input_type, &get_consume_impl)); + + out->consume_impl = std::move(get_consume_impl.consume_impl); + out->out_type = std::move(get_consume_impl.out_type); + return out; } - std::vector sums; -}; + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups() * kSumSize; + KERNEL_RETURN_IF_ERROR(ctx, sums->Resize(new_num_groups * kSumSize)); + auto new_size = num_groups() * kSumSize; + std::memset(sums->mutable_data() + old_size, 0, new_size - old_size); + } -Result> GroupedAggregator::Make( - std::string function, const FunctionOptions* options) { - if (function == "count") { - return ::arrow::internal::make_unique(options); + void Consume(KernelContext* ctx, const ExecBatch& batch, + const uint32_t* group_ids) override { + MaybeResize(ctx, batch.length, group_ids); + if (ctx->HasError()) return; + consume_impl(batch[0].array(), group_ids, sums.get()); } - if (function == "sum") { - return ::arrow::internal::make_unique(options); + + void Finalize(KernelContext* ctx, Datum* out) override { + auto length = num_groups(); + *out = ArrayData::Make(std::move(out_type), length, + {/*null_bitmap=*/nullptr, std::move(sums)}); } - return Status::NotImplemented("Grouped aggregate ", function); -} + + int64_t num_groups() const override { return sums->size() * kSumSize; } + + std::shared_ptr sums; + std::shared_ptr out_type; + ConsumeImpl consume_impl; +}; std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique( @@ -345,17 +431,16 @@ struct GroupByImpl : public ScalarAggregator { void MergeFrom(KernelContext* ctx, KernelState&& src) override { // TODO(michalursa) merge two hash tables + ctx->SetStatus(Status::NotImplemented("merging grouped aggregations")); } void Finalize(KernelContext* ctx, Datum* out) override { - FieldVector out_fields(aggregators.size() + 1); ArrayDataVector out_columns(aggregators.size() + 1); for (size_t i = 0; i < aggregators.size(); ++i) { Datum aggregand; aggregators[i]->Finalize(ctx, &aggregand); if (ctx->HasError()) return; out_columns[i] = aggregand.array(); - out_fields[i] = field(options.aggregates[i].name, aggregand.type()); } int64_t length = keys_.size(); @@ -365,21 +450,37 @@ struct GroupByImpl : public ScalarAggregator { auto key = std::make_shared(length, std::move(key_buf)); out_columns.back() = key->data(); - out_fields.back() = field(options.key_names[0], key->type()); - *out = ArrayData::Make(struct_(std::move(out_fields)), key->length(), - {/*null_bitmap=*/nullptr}, std::move(out_columns)); + *out = ArrayData::Make(std::move(out_type), key->length(), {/*null_bitmap=*/nullptr}, + std::move(out_columns)); } std::map map_; std::vector keys_; + std::shared_ptr out_type; GroupByOptions options; std::vector> aggregators; }; +template +std::unique_ptr MakeAggregator(KernelContext* ctx, + const std::string& function_name, + const DataType& input_type, + const FunctionOptions* options) { + if (options == nullptr) { + if (auto function = ctx->exec_context() + ->func_registry() + ->GetFunction(function_name) + .ValueOr(nullptr)) { + options = function->default_options(); + } + } + + return Aggregator::Make(ctx, input_type, options); +} + std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArgs& args) { - // TODO(michalursa) do construction of group by implementation auto impl = ::arrow::internal::make_unique(); impl->options = *checked_cast(args.options); const auto& aggregates = impl->options.aggregates; @@ -389,11 +490,26 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg return nullptr; } + FieldVector out_fields(args.inputs.size()); + impl->aggregators.resize(aggregates.size()); for (size_t i = 0; i < aggregates.size(); ++i) { - ctx->SetStatus(GroupedAggregator::Make(aggregates[i].function, aggregates[i].options) - .Value(&impl->aggregators[i])); + const std::string& function = aggregates[i].function; + const FunctionOptions* options = aggregates[i].options; + const auto& input_type = args.inputs[i].type; + + if (function == "count") { + impl->aggregators[i] = + MakeAggregator(ctx, function, *input_type, options); + } else if (function == "sum") { + impl->aggregators[i] = + MakeAggregator(ctx, function, *input_type, options); + } else { + ctx->SetStatus(Status::NotImplemented("Grouped aggregate ", function)); + } if (ctx->HasError()) return nullptr; + + out_fields[i] = field(aggregates[i].name, input_type); } size_t n_keys = args.inputs.size() - aggregates.size(); @@ -408,6 +524,9 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg return nullptr; } + out_fields.back() = field(impl->options.key_names[0], int64()); + impl->out_type = struct_(std::move(out_fields)); + return impl; } @@ -557,15 +676,18 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) { // group_by func = std::make_shared("group_by", Arity::VarArgs(), &group_by_doc); - // aggregate::AddBasicAggKernels(aggregate::GroupByInit, {null()}, null(), func.get()); { - InputType any_array(ValueDescr::ARRAY); - auto sig = KernelSignature::Make({any_array}, ValueDescr::Array(int64()), true); + auto sig = KernelSignature::Make( + {ValueDescr::ARRAY}, + OutputType([](KernelContext* ctx, const std::vector&) { + return Result{ + checked_cast(ctx->state())->out_type}; + }), + /*is_varargs=*/true); AddAggKernel(std::move(sig), aggregate::GroupByInit, func.get(), SimdLevel::NONE, true); } DCHECK_OK(registry->AddFunction(std::move(func))); - // TODO(michalursa) add Kernels to the function named "group_by" } } // namespace internal diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 1d8a4b88b41..af4a40c2ad7 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -137,6 +137,78 @@ TEST(GroupBy, CountOnly) { } } +TEST(GroupBy, CountAndSum) { + auto key = ArrayFromJSON(int64(), + "[1, 2, 1," + "3, 2, 3]"); + auto aggregand = ArrayFromJSON(float32(), + "[1.0, 0.0, null, " + "3.25, 0.125, -0.25]"); + + ASSERT_EQ(key->length(), aggregand->length()); + + GroupByOptions options; + CountOptions count_options; + count_options.count_mode = CountOptions::COUNT_NON_NULL; + options.aggregates = { + GroupByOptions::Aggregate{"count", &count_options, "f32 counted"}, + GroupByOptions::Aggregate{"sum", nullptr, "f32 summed"}, + GroupByOptions::Aggregate{"sum", nullptr, "i64 summed"}, + }; + options.key_names = {"i64 key"}; + + ASSERT_OK_AND_ASSIGN(Datum boxed, + CallFunction("group_by", + { + // NB: passing the same aggregand twice + aggregand, + aggregand, + // NB: passing the key column also as an aggregand + key, + key, + }, + &options)); + + auto aggregated_and_grouped = boxed.array_as(); + + auto f32_counted = checked_pointer_cast( + aggregated_and_grouped->GetFieldByName("f32 counted")); + + // NB: summing a float32 array results in float64 sums + auto f32_summed = checked_pointer_cast( + aggregated_and_grouped->GetFieldByName("f32 summed")); + + auto i64_summed = checked_pointer_cast( + aggregated_and_grouped->GetFieldByName("i64 summed")); + + auto i64_key = + checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); + + ASSERT_EQ(i64_key->length(), 3); + + for (int64_t i = 0; i < i64_key->length(); ++i) { + int64_t key = i64_key->Value(i); + + if (key == 1) { + ASSERT_EQ(f32_counted->Value(i), 1); + } else { + ASSERT_EQ(f32_counted->Value(i), 2); + } + + ASSERT_EQ(i64_summed->Value(i), key * 2); + + if (key == 1) { + ASSERT_EQ(f32_summed->Value(i), 1.0); + } + if (key == 2) { + ASSERT_EQ(f32_summed->Value(i), 0.125); + } + if (key == 3) { + ASSERT_EQ(f32_summed->Value(i), 3.0); + } + } +} + // // Sum // From 4fe0613544e93d6461ea3155ff2d1bcabc8cdd12 Mon Sep 17 00:00:00 2001 From: michalursa Date: Fri, 19 Feb 2021 02:33:32 -0800 Subject: [PATCH 04/49] Add support for multiple key columns in group by (no testing yet) --- .../arrow/compute/kernels/aggregate_basic.cc | 588 +++++++++++++++++- 1 file changed, 558 insertions(+), 30 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 6b57c2bf0a2..70d28f9ca2d 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -390,6 +390,471 @@ std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) } struct GroupByImpl : public ScalarAggregator { + + using AddLengthImpl = + std::function&, int32_t*)>; + + struct GetAddLengthImpl { + static constexpr int32_t null_extra_byte = 1; + + static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, int32_t* lengths) { + for (int64_t i = 0; i < num_repeats; ++i) { + lengths[i] += fixed_length + null_extra_byte; + } + } + + static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { + using offset_type = typename StringType::offset_type; + constexpr int32_t length_extra_bytes = sizeof(offset_type); + auto offset = data->offset; + const auto offsets = data->GetValues(1); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + lengths[i] += null_extra_byte + length_extra_bytes; + } else { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - offsets[offset + i]; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - offsets[offset + i]; + } + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; + add_length_impl = [num_bytes](const std::shared_ptr& data, int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); + }; + return Status::OK(); + } + + Status Visit(const StringType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + add_length_impl = [num_bytes](const std::shared_ptr& data, int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); + }; + return Status::OK(); + } + + AddLengthImpl add_length_impl; + }; + + using EncodeNextImpl = std::function&, uint8_t**)>; + + struct GetEncodeNextImpl { + + template + static void EncodeSmallFixed(const std::shared_ptr& data, uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + uint64_t null_multiplier = is_null ? 0 : 1; + if (NumBits == 1) { + encoded_ptr[0] = static_cast(null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + encoded_ptr += 8; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + if (NumBits == 1) { + encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 8; + } + } + } + } + + static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + if (is_null) { + memset(encoded_ptr, 0, num_bytes); + } else { + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + } + encoded_ptr += num_bytes; + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + encoded_ptr += num_bytes; + } + } + } + + static void EncodeVarLength(const std::shared_ptr& data, uint8_t** encoded_bytes) { + using offset_type = typename StringType::offset_type; + auto offset = data->offset; + const auto offsets = data->GetValues(1); + auto raw_input = data->buffers[2]->data(); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + encoded_ptr[0] = 1; + encoded_ptr++; + reinterpret_cast(encoded_ptr)[0] = 0; + encoded_ptr += sizeof(offset_type); + } else { + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bits = bit_width(input_type.id()); + switch (num_bits) { + case 1: + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeSmallFixed<1>(data, encoded_bytes); + }; + break; + case 8: + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeSmallFixed<8>(data, encoded_bytes); + }; + break; + case 16: + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeSmallFixed<16>(data, encoded_bytes); + }; + break; + case 32: + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeSmallFixed<32>(data, encoded_bytes); + }; + break; + case 64: + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeSmallFixed<64>(data, encoded_bytes); + }; + break; + } + return Status::OK(); + } + + Status Visit(const StringType&) { + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + encode_next_impl = [num_bytes](const std::shared_ptr& data, uint8_t** encoded_bytes) { + EncodeBigFixed(num_bytes, data, encoded_bytes); + }; + return Status::OK(); + } + + EncodeNextImpl encode_next_impl; + }; + + using DecodeNextImpl = std::function*)>; + + struct GetDecodeNextImpl { + + static void DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* null_buf, int32_t &null_count) { + // Do we have nulls? + null_count = 0; + for (size_t i = 0; i < length; ++i) { + null_count += encoded_bytes[i][0]; + } + null_buf = NULLPTR; + if (null_count > 0) { + KERNEL_ASSIGN_OR_RAISE(*null_buf, ctx, ctx->Allocate((length + 7) / 8)); + uint8_t* nulls = (*null_buf)->mutable_data(); + memset(nulls, 0, (length + 7) / 8); + for (size_t i = 0; i < length; ++i) { + if (encoded_bytes[i][0]) { + BitUtil::SetBit(nulls, i); + } + encoded_bytes[i] += 1; + } + } else { + for (size_t i = 0; i < length; ++i) { + encoded_bytes[i] += 1; + } + } + } + + template + static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); + + uint8_t* raw_output = key_buf->mutable_data(); + for (size_t i = 0; i < length; ++i) { + auto &encoded_ptr = encoded_bytes[i]; + if (NumBits == 1) { + BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + raw_output[i] = encoded_ptr[0]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 8; + } + } + + switch (output_type) { + case Type::BOOL: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::UINT8: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::INT8: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::UINT16: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::INT16: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::UINT32: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::INT32: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::UINT64: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + break; + case Type::INT64: + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + // *out = std::make_shared(length, key_buf, null_buf); + break; + } + } + + static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); + auto raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); + encoded_bytes[i] += num_bytes; + } + + *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, null_count); + } + + static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + + using offset_type = typename StringType::offset_type; + + int32_t length_sum = 0; + for (int32_t i = 0; i < length; ++i) { + length_sum += reinterpret_cast(encoded_bytes)[0]; + } + + KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, ctx->Allocate(sizeof(offset_type) * (1 + length))); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); + + auto raw_offsets = offset_buf->mutable_data(); + auto raw_keys = key_buf->mutable_data(); + int32_t current_offset = 0; + for (int32_t i = 0; i < length; ++i) { + offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_offsets)[i] = current_offset; + encoded_bytes[i] += sizeof(offset_type); + memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); + encoded_bytes[i] += key_length; + current_offset += key_length; + } + reinterpret_cast(raw_offsets)[length] = current_offset; + + if (is_string) { + *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, null_count, 0); + } else { + *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, null_count, 0); + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bits = bit_width(input_type.id()); + auto type_id = input_type.id(); + switch (num_bits) { + case 1: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 8: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 16: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 32: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 64: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); + }; + break; + } + return Status::OK(); + } + + Status Visit(const StringType&) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeVarLength(ctx, true, length, encoded_bytes, out); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeVarLength(ctx, false, length, encoded_bytes, out); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); + }; + return Status::OK(); + } + + DecodeNextImpl decode_next_impl; + }; + void Consume(KernelContext* ctx, const ExecBatch& batch) override { ArrayDataVector aggregands, keys; @@ -401,30 +866,51 @@ struct GroupByImpl : public ScalarAggregator { keys.push_back(batch[i++].array()); } - auto key64 = batch[aggregators.size()].array_as(); - if (key64->null_count() != 0) { - ctx->SetStatus(Status::NotImplemented("nulls in key column")); - return; + offsets_batch_.clear(); + offsets_batch_.resize(batch.length + 1); + offsets_batch_[0] = 0; + memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); + for (size_t i = 0; i < keys.size(); ++i) { + add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); + } + int32_t total_length = 0; + for (int64_t i = 0; i < batch.length; ++i) { + total_length += offsets_batch_[i]; + offsets_batch_[i + 1] = total_length; } - const int64_t* key64_raw = key64->raw_values(); + key_bytes_batch_.clear(); + key_bytes_batch_.resize(total_length); + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; + } + for (size_t i = 0; i < keys.size(); ++i) { + encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); + } - std::vector group_ids(batch.length); + group_ids_batch_.clear(); + group_ids_batch_.resize(batch.length); for (int64_t i = 0; i < batch.length; ++i) { - uint64_t key = key64_raw[i]; + int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; + std::string key(reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), key_length); auto iter = map_.find(key); if (iter == map_.end()) { - group_ids[i] = static_cast(keys_.size()); - keys_.push_back(key); - map_.insert(std::make_pair(key, group_ids[i])); + group_ids_batch_[i] = n_groups++; + int32_t next_key_offset = static_cast(key_bytes_.size()); + key_bytes_.resize(next_key_offset + key_length); + offsets_.push_back(next_key_offset + key_length); + memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); + map_.insert(std::make_pair(key, group_ids_batch_[i])); } else { - group_ids[i] = iter->second; + group_ids_batch_[i] = iter->second; } } for (size_t i = 0; i < aggregators.size(); ++i) { ExecBatch aggregand_batch{{aggregands[i]}, batch.length}; - aggregators[i]->Consume(ctx, aggregand_batch, group_ids.data()); + aggregators[i]->Consume(ctx, aggregand_batch, group_ids_batch_.data()); if (ctx->HasError()) return; } } @@ -435,7 +921,8 @@ struct GroupByImpl : public ScalarAggregator { } void Finalize(KernelContext* ctx, Datum* out) override { - ArrayDataVector out_columns(aggregators.size() + 1); + size_t n_keys = decode_next_impl.size(); + ArrayDataVector out_columns(aggregators.size() + n_keys); for (size_t i = 0; i < aggregators.size(); ++i) { Datum aggregand; aggregators[i]->Finalize(ctx, &aggregand); @@ -443,24 +930,39 @@ struct GroupByImpl : public ScalarAggregator { out_columns[i] = aggregand.array(); } - int64_t length = keys_.size(); - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(sizeof(int64_t) * length)); - std::copy(keys_.begin(), keys_.end(), - reinterpret_cast(key_buf->mutable_data())); - auto key = std::make_shared(length, std::move(key_buf)); + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(n_groups); + for (int64_t i = 0; i < n_groups; ++i) { + key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; + } - out_columns.back() = key->data(); + int64_t length = n_groups; + for (size_t i = 0; i < n_keys; ++i) { + std::shared_ptr key_array; + decode_next_impl[i].decode_next_impl(ctx, static_cast(length), key_buf_ptrs_.data(), &key_array); + out_columns[aggregators.size() + i] = std::move(key_array); + } - *out = ArrayData::Make(std::move(out_type), key->length(), {/*null_bitmap=*/nullptr}, + *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, std::move(out_columns)); } + std::vector offsets_batch_; + std::vector key_bytes_batch_; + std::vector key_buf_ptrs_; + std::vector group_ids_batch_; - std::map map_; - std::vector keys_; + std::map map_; + std::vector offsets_; + std::vector key_bytes_; + uint32_t n_groups; std::shared_ptr out_type; GroupByOptions options; std::vector> aggregators; + + std::vector add_length_impl; + std::vector encode_next_impl; + std::vector decode_next_impl; }; template @@ -485,6 +987,9 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg impl->options = *checked_cast(args.options); const auto& aggregates = impl->options.aggregates; + impl->n_groups = 0; + impl->offsets_.push_back(0); + if (aggregates.size() > args.inputs.size()) { ctx->SetStatus(Status::Invalid("more aggegates than inputs!")); return nullptr; @@ -513,18 +1018,41 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg } size_t n_keys = args.inputs.size() - aggregates.size(); - if (n_keys != 1) { - ctx->SetStatus(Status::NotImplemented("more than one key")); - return nullptr; + for (size_t i = 0; i < n_keys; ++i) { + const auto& key_type = args.inputs[aggregates.size() + i].type; + switch (key_type->id()) + { + // Supported types of keys + case Type::BOOL: + case Type::UINT8: + case Type::INT8: + case Type::UINT16: + case Type::INT16: + case Type::UINT32: + case Type::INT32: + case Type::UINT64: + case Type::INT64: + case Type::STRING: + case Type::BINARY: + case Type::FIXED_SIZE_BINARY: + break; + default: + ctx->SetStatus(Status::NotImplemented("Key of type", key_type->ToString())); + return nullptr; + } + out_fields[aggregates.size() + i] = field(impl->options.key_names[i], key_type); } - if (args.inputs.back().type->id() != Type::INT64) { - ctx->SetStatus( - Status::NotImplemented("key of type", args.inputs.back().type->ToString())); - return nullptr; + impl->add_length_impl.resize(n_keys); + impl->encode_next_impl.resize(n_keys); + impl->decode_next_impl.resize(n_keys); + for (size_t i = 0; i < n_keys; ++i) { + const auto& key_type = args.inputs[aggregates.size() + i].type; + ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->add_length_impl[i])); + ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->encode_next_impl[i])); + ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->decode_next_impl[i])); } - out_fields.back() = field(impl->options.key_names[0], int64()); impl->out_type = struct_(std::move(out_fields)); return impl; From c72a178a1f8e885cf529ef357afbbf1b1765b461 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 19 Feb 2021 14:36:44 -0500 Subject: [PATCH 05/49] add randomized testing for group_by --- .../arrow/compute/kernels/aggregate_test.cc | 305 +++++++++++++++++- 1 file changed, 297 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index af4a40c2ad7..34a1db54b95 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -27,24 +27,275 @@ #include "arrow/array.h" #include "arrow/chunked_array.h" #include "arrow/compute/api_aggregate.h" +#include "arrow/compute/api_scalar.h" +#include "arrow/compute/api_vector.h" +#include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/type.h" #include "arrow/type_traits.h" #include "arrow/util/bitmap_reader.h" #include "arrow/util/checked_cast.h" +#include "arrow/util/int_util_internal.h" #include "arrow/testing/gtest_common.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/random.h" +#include "arrow/util/logging.h" namespace arrow { +using internal::BitmapReader; using internal::checked_cast; using internal::checked_pointer_cast; namespace compute { +// Copy-pasta from partition.cc +// +// In the finished product this will only be a test helper for group_by +// and partition.cc will rely on a no-aggregate call to group_by. +namespace group_helpers { +namespace { + +// Transform an array of counts to offsets which will divide a ListArray +// into an equal number of slices with corresponding lengths. +Result> CountsToOffsets(std::shared_ptr counts) { + TypedBufferBuilder offset_builder; + RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); + + int32_t current_offset = 0; + offset_builder.UnsafeAppend(current_offset); + + for (int64_t i = 0; i < counts->length(); ++i) { + DCHECK_NE(counts->Value(i), 0); + current_offset += static_cast(counts->Value(i)); + offset_builder.UnsafeAppend(current_offset); + } + + std::shared_ptr offsets; + RETURN_NOT_OK(offset_builder.Finish(&offsets)); + return offsets; +} + +class StructDictionary { + public: + struct Encoded { + std::shared_ptr indices; + std::shared_ptr dictionary; + }; + + static Result Encode(const ArrayVector& columns) { + Encoded out{nullptr, std::make_shared()}; + + for (const auto& column : columns) { + if (column->null_count() != 0) { + return Status::NotImplemented("Grouping on a field with nulls"); + } + + RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices)); + } + + return out; + } + + Result> Decode(std::shared_ptr fused_indices, + FieldVector fields) { + std::vector builders(dictionaries_.size()); + for (Int32Builder& b : builders) { + RETURN_NOT_OK(b.Resize(fused_indices->length())); + } + + std::vector codes(dictionaries_.size()); + for (int64_t i = 0; i < fused_indices->length(); ++i) { + Expand(fused_indices->Value(i), codes.data()); + + auto builder_it = builders.begin(); + for (int32_t index : codes) { + builder_it++->UnsafeAppend(index); + } + } + + ArrayVector columns(dictionaries_.size()); + for (size_t i = 0; i < dictionaries_.size(); ++i) { + std::shared_ptr indices; + RETURN_NOT_OK(builders[i].FinishInternal(&indices)); + + ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices)); + + if (fields[i]->type()->id() == Type::DICTIONARY) { + RETURN_NOT_OK(RestoreDictionaryEncoding( + checked_pointer_cast(fields[i]->type()), &column)); + } + + columns[i] = column.make_array(); + } + + return StructArray::Make(std::move(columns), std::move(fields)); + } + + private: + Status AddOne(Datum column, std::shared_ptr* fused_indices) { + if (column.type()->id() != Type::DICTIONARY) { + ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(std::move(column))); + } + + auto dict_column = column.array_as(); + dictionaries_.push_back(dict_column->dictionary()); + ARROW_ASSIGN_OR_RAISE(auto indices, compute::Cast(*dict_column->indices(), int32())); + + if (*fused_indices == nullptr) { + *fused_indices = checked_pointer_cast(std::move(indices)); + return IncreaseSize(); + } + + // It's useful to think about the case where each of dictionaries_ has size 10. + // In this case the decimal digit in the ones place is the code in dictionaries_[0], + // the tens place corresponds to the code in dictionaries_[1], etc. + // The incumbent indices must be shifted to the hundreds place so as not to collide. + ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices, + compute::Multiply(indices, MakeScalar(size_))); + + ARROW_ASSIGN_OR_RAISE(new_fused_indices, + compute::Add(new_fused_indices, *fused_indices)); + + *fused_indices = checked_pointer_cast(new_fused_indices.make_array()); + return IncreaseSize(); + } + + // expand a fused code into component dict codes, order is in order of addition + void Expand(int32_t fused_code, int32_t* codes) { + for (size_t i = 0; i < dictionaries_.size(); ++i) { + auto dictionary_size = static_cast(dictionaries_[i]->length()); + codes[i] = fused_code % dictionary_size; + fused_code /= dictionary_size; + } + } + + Status RestoreDictionaryEncoding(std::shared_ptr expected_type, + Datum* column) { + DCHECK_NE(column->type()->id(), Type::DICTIONARY); + ARROW_ASSIGN_OR_RAISE(*column, compute::DictionaryEncode(std::move(*column))); + + if (expected_type->index_type()->id() == Type::INT32) { + // dictionary_encode has already yielded the expected index_type + return Status::OK(); + } + + // cast the indices to the expected index type + auto dictionary = std::move(column->mutable_array()->dictionary); + column->mutable_array()->type = int32(); + + ARROW_ASSIGN_OR_RAISE(*column, + compute::Cast(std::move(*column), expected_type->index_type())); + + column->mutable_array()->dictionary = std::move(dictionary); + column->mutable_array()->type = expected_type; + return Status::OK(); + } + + Status IncreaseSize() { + auto factor = static_cast(dictionaries_.back()->length()); + + if (arrow::internal::MultiplyWithOverflow(size_, factor, &size_)) { + return Status::CapacityError("Max groups exceeded"); + } + return Status::OK(); + } + + int32_t size_ = 1; + ArrayVector dictionaries_; +}; + +Result> MakeGroupings(const StructArray& keys) { + if (keys.num_fields() == 0) { + return Status::Invalid("Grouping with no keys"); + } + + if (keys.null_count() != 0) { + return Status::Invalid("Grouping with null keys"); + } + + ARROW_ASSIGN_OR_RAISE(auto fused, StructDictionary::Encode(keys.fields())); + + ARROW_ASSIGN_OR_RAISE(auto sort_indices, compute::SortIndices(*fused.indices)); + ARROW_ASSIGN_OR_RAISE(Datum sorted, compute::Take(fused.indices, *sort_indices)); + fused.indices = checked_pointer_cast(sorted.make_array()); + + ARROW_ASSIGN_OR_RAISE(auto fused_counts_and_values, + compute::ValueCounts(fused.indices)); + fused.indices.reset(); + + auto unique_fused_indices = + checked_pointer_cast(fused_counts_and_values->GetFieldByName("values")); + ARROW_ASSIGN_OR_RAISE( + auto unique_rows, + fused.dictionary->Decode(std::move(unique_fused_indices), keys.type()->fields())); + + auto counts = + checked_pointer_cast(fused_counts_and_values->GetFieldByName("counts")); + ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); + + auto grouped_sort_indices = + std::make_shared(list(sort_indices->type()), unique_rows->length(), + std::move(offsets), std::move(sort_indices)); + + return StructArray::Make( + ArrayVector{std::move(unique_rows), std::move(grouped_sort_indices)}, + std::vector{"values", "groupings"}); +} + +Result> ApplyGroupings(const ListArray& groupings, + const Array& array) { + ARROW_ASSIGN_OR_RAISE(Datum sorted, + compute::Take(array, groupings.data()->child_data[0])); + + return std::make_shared(list(array.type()), groupings.length(), + groupings.value_offsets(), sorted.make_array()); +} + +Result> NaiveGroupBy(GroupByOptions options, + ArrayVector aggregands, ArrayVector keys) { + ARROW_ASSIGN_OR_RAISE(auto keys_struct, StructArray::Make(keys, options.key_names)); + ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); + + auto groupings = + checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); + + auto keys_unique = + checked_pointer_cast(groupings_and_values->GetFieldByName("values")); + + int64_t n_groups = groupings->length(); + + std::vector out(n_groups); + + auto aggregate_spec = options.aggregates.begin(); + for (const auto& aggregand : aggregands) { + ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, ApplyGroupings(*groupings, *aggregand)); + + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ARROW_ASSIGN_OR_RAISE(auto grouped_aggregate, + CallFunction(aggregate_spec->function, + {grouped_aggregand->value_slice(i_group)})); + out[i_group].push_back(grouped_aggregate.scalar()); + } + + ++aggregate_spec; + } + + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ARROW_ASSIGN_OR_RAISE(auto keys_for_group, keys_unique->GetScalar(i_group)); + for (const auto& key : checked_cast(*keys_for_group).value) { + out[i_group].push_back(key); + } + } + + return out; +} + +} // namespace +} // namespace group_helpers + // TODO(michalursa) add tests TEST(GroupBy, SumOnly) { auto key = ArrayFromJSON(int64(), @@ -209,6 +460,47 @@ TEST(GroupBy, CountAndSum) { } } +TEST(GroupBy, RandomArraySum) { + auto rand = random::RandomArrayGenerator(0xdeadbeef); + GroupByOptions options; + options.aggregates = { + GroupByOptions::Aggregate{"sum", nullptr, "f32 summed"}, + }; + + options.key_names = {"i64 key"}; + for (size_t i = 3; i < 14; i += 2) { + for (auto null_probability : {0.0, 0.001, 0.1, 0.5, 0.999, 1.0}) { + int64_t length = 1UL << i; + auto summand = rand.Float32(length, -100, 100, null_probability); + auto key = rand.Int64(length, 0, 12); + + ASSERT_OK_AND_ASSIGN(auto expected, + group_helpers::NaiveGroupBy(options, {summand}, {key})); + auto n_groups = static_cast(expected.size()); + + ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", + { + summand, + key, + }, + &options)); + auto actual = boxed.array_as(); + ASSERT_EQ(actual->length(), n_groups); + + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + const auto& expected_for_group = expected[i_group]; + auto actual_for_group = + checked_pointer_cast(*actual->GetScalar(i_group))->value; + + ASSERT_EQ(expected_for_group.size(), actual_for_group.size()); + for (size_t i = 0; i < expected_for_group.size(); ++i) { + AssertScalarsEqual(*expected_for_group[i], *actual_for_group[i]); + } + } + } + } +} + // // Sum // @@ -229,8 +521,7 @@ static SumResult NaiveSumPartial(const Array& array) { const auto values = array_numeric.raw_values(); if (array.null_count() != 0) { - internal::BitmapReader reader(array.null_bitmap_data(), array.offset(), - array.length()); + BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); for (int64_t i = 0; i < array.length(); i++) { if (reader.IsSet()) { result.first += values[i]; @@ -810,8 +1101,7 @@ static enable_if_integer> NaiveMinMax( T min = std::numeric_limits::max(); T max = std::numeric_limits::min(); if (array.null_count() != 0) { // Some values are null - internal::BitmapReader reader(array.null_bitmap_data(), array.offset(), - array.length()); + BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); for (int64_t i = 0; i < array.length(); i++) { if (reader.IsSet()) { min = std::min(min, values[i]); @@ -850,8 +1140,7 @@ static enable_if_floating_point> NaiveMinMax( T min = std::numeric_limits::infinity(); T max = -std::numeric_limits::infinity(); if (array.null_count() != 0) { // Some values are null - internal::BitmapReader reader(array.null_bitmap_data(), array.offset(), - array.length()); + BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); for (int64_t i = 0; i < array.length(); i++) { if (reader.IsSet()) { min = std::fmin(min, values[i]); @@ -1194,7 +1483,7 @@ ModeResult NaiveMode(const Array& array) { const auto& array_numeric = reinterpret_cast(array); const auto values = array_numeric.raw_values(); - internal::BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); + BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); for (int64_t i = 0; i < array.length(); ++i) { if (reader.IsSet()) { ++value_counts[values[i]]; @@ -1445,7 +1734,7 @@ void KahanSum(double& sum, double& adjust, double addend) { template std::pair WelfordVar(const ArrayType& array) { const auto values = array.raw_values(); - internal::BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); + BitmapReader reader(array.null_bitmap_data(), array.offset(), array.length()); double count = 0, mean = 0, m2 = 0; double mean_adjust = 0, m2_adjust = 0; for (int64_t i = 0; i < array.length(); ++i) { From 3085d0479d13da0594bc2a939ef833d864259464 Mon Sep 17 00:00:00 2001 From: michalursa Date: Sun, 21 Feb 2021 19:26:51 -0800 Subject: [PATCH 06/49] Fixing bugs in group_by. Current tests should be passing now. --- .../arrow/compute/kernels/aggregate_basic.cc | 21 ++++---- .../arrow/compute/kernels/aggregate_test.cc | 54 +++++++++++++++++-- 2 files changed, 63 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 70d28f9ca2d..d0f854355b1 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -168,7 +168,7 @@ struct GroupedCountImpl : public GroupedAggregator { *out = std::make_shared(length, std::move(counts)); } - int64_t num_groups() const override { return counts->size() * sizeof(int64_t); } + int64_t num_groups() const override { return counts->size() / sizeof(int64_t); } CountOptions options; std::shared_ptr counts; @@ -270,7 +270,7 @@ struct GroupedSumImpl : public GroupedAggregator { {/*null_bitmap=*/nullptr, std::move(sums)}); } - int64_t num_groups() const override { return sums->size() * kSumSize; } + int64_t num_groups() const override { return sums->size() / kSumSize; } std::shared_ptr sums; std::shared_ptr out_type; @@ -660,22 +660,21 @@ struct GroupByImpl : public ScalarAggregator { static void DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* null_buf, int32_t &null_count) { // Do we have nulls? null_count = 0; - for (size_t i = 0; i < length; ++i) { + for (int32_t i = 0; i < length; ++i) { null_count += encoded_bytes[i][0]; } - null_buf = NULLPTR; if (null_count > 0) { - KERNEL_ASSIGN_OR_RAISE(*null_buf, ctx, ctx->Allocate((length + 7) / 8)); + ctx->SetStatus(ctx->Allocate((length + 7) / 8).Value(null_buf)); uint8_t* nulls = (*null_buf)->mutable_data(); memset(nulls, 0, (length + 7) / 8); - for (size_t i = 0; i < length; ++i) { + for (int32_t i = 0; i < length; ++i) { if (encoded_bytes[i][0]) { BitUtil::SetBit(nulls, i); } encoded_bytes[i] += 1; } } else { - for (size_t i = 0; i < length; ++i) { + for (int32_t i = 0; i < length; ++i) { encoded_bytes[i] += 1; } } @@ -690,7 +689,7 @@ struct GroupByImpl : public ScalarAggregator { KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); uint8_t* raw_output = key_buf->mutable_data(); - for (size_t i = 0; i < length; ++i) { + for (int32_t i = 0; i < length; ++i) { auto &encoded_ptr = encoded_bytes[i]; if (NumBits == 1) { BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); @@ -743,6 +742,8 @@ struct GroupByImpl : public ScalarAggregator { *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); // *out = std::make_shared(length, key_buf, null_buf); break; + default: + break; } } @@ -875,9 +876,11 @@ struct GroupByImpl : public ScalarAggregator { } int32_t total_length = 0; for (int64_t i = 0; i < batch.length; ++i) { + auto total_length_before = total_length; total_length += offsets_batch_[i]; - offsets_batch_[i + 1] = total_length; + offsets_batch_[i] = total_length_before; } + offsets_batch_[batch.length] = total_length; key_bytes_batch_.clear(); key_bytes_batch_.resize(total_length); diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 34a1db54b95..01e71481ab6 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -343,6 +343,37 @@ TEST(GroupBy, SumOnly) { } } +TEST(GroupBy, StringKey) { + auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); + auto aggregand = ArrayFromJSON(int64(), + "[10, 5, 4, 2, 12, 9]"); + GroupByOptions options; + options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr, "sum"}}; + options.key_names = {"key"}; + ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", {aggregand, key}, &options)); + auto aggregated_and_grouped = boxed.array_as(); + auto result_sum = checked_pointer_cast(aggregated_and_grouped->GetFieldByName("sum")); + auto result_key = checked_pointer_cast(aggregated_and_grouped->GetFieldByName("key")); + ASSERT_EQ(result_key->length(), 4); + for (int64_t i = 0; i < result_key->length(); ++i) { + int32_t key_length; + const uint8_t* key_chars = result_key->GetValue(i, &key_length); + std::string key_str((char*)key_chars, key_length); + if (key_str.compare("alfa") == 0) { + ASSERT_EQ(result_sum->Value(i), 10); + } + if (key_str.compare("beta") == 0) { + ASSERT_EQ(result_sum->Value(i), 14); + } + if (key_str.compare("gamma") == 0) { + ASSERT_EQ(result_sum->Value(i), 6); + } + if (key_str.compare("") == 0) { + ASSERT_EQ(result_sum->Value(i), 12); + } + } +} + TEST(GroupBy, CountOnly) { auto key = ArrayFromJSON(int64(), "[1, 2, 1," @@ -487,15 +518,32 @@ TEST(GroupBy, RandomArraySum) { auto actual = boxed.array_as(); ASSERT_EQ(actual->length(), n_groups); + std::vector> vexpected; + std::vector> vactual; + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { const auto& expected_for_group = expected[i_group]; auto actual_for_group = checked_pointer_cast(*actual->GetScalar(i_group))->value; ASSERT_EQ(expected_for_group.size(), actual_for_group.size()); - for (size_t i = 0; i < expected_for_group.size(); ++i) { - AssertScalarsEqual(*expected_for_group[i], *actual_for_group[i]); - } + ASSERT_EQ(expected_for_group.size(), 2); + + double expected_sum = ((DoubleScalar*)expected_for_group[0].get())->value; + int64_t expected_key = ((Int64Scalar*)expected_for_group[1].get())->value; + double actual_sum = ((DoubleScalar*)actual_for_group[0].get())->value; + int64_t actual_key = ((Int64Scalar*)actual_for_group[1].get())->value; + + vexpected.push_back(std::make_pair(expected_key, expected_sum)); + vactual.push_back(std::make_pair(actual_key, actual_sum)); + } + + std::sort(vexpected.begin(), vexpected.end()); + std::sort(vactual.begin(), vactual.end()); + + for (size_t i = 0; i < vexpected.size(); ++i) { + ASSERT_EQ(vexpected[i].first, vactual[i].first); + ASSERT_EQ(vexpected[i].second, vactual[i].second); } } } From e7fee76c900159f40aacc79af1d336fc6cded3ab Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Feb 2021 12:34:41 -0500 Subject: [PATCH 07/49] formatting --- .../arrow/compute/kernels/aggregate_basic.cc | 207 +++++++++++------- .../arrow/compute/kernels/aggregate_test.cc | 9 +- 2 files changed, 136 insertions(+), 80 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index d0f854355b1..8fe2279533b 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -390,14 +390,13 @@ std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) } struct GroupByImpl : public ScalarAggregator { - - using AddLengthImpl = - std::function&, int32_t*)>; + using AddLengthImpl = std::function&, int32_t*)>; struct GetAddLengthImpl { static constexpr int32_t null_extra_byte = 1; - static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, int32_t* lengths) { + static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, + int32_t* lengths) { for (int64_t i = 0; i < num_repeats; ++i) { lengths[i] += fixed_length + null_extra_byte; } @@ -415,12 +414,14 @@ struct GroupByImpl : public ScalarAggregator { if (is_null) { lengths[i] += null_extra_byte + length_extra_bytes; } else { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - offsets[offset + i]; + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; } } } else { for (int64_t i = 0; i < data->length; ++i) { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - offsets[offset + i]; + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; } } } @@ -428,7 +429,8 @@ struct GroupByImpl : public ScalarAggregator { template Status Visit(const T& input_type) { int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; - add_length_impl = [num_bytes](const std::shared_ptr& data, int32_t* lengths) { + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { AddFixedLength(num_bytes, data->length, lengths); }; return Status::OK(); @@ -450,7 +452,8 @@ struct GroupByImpl : public ScalarAggregator { Status Visit(const FixedSizeBinaryType& type) { int32_t num_bytes = type.byte_width(); - add_length_impl = [num_bytes](const std::shared_ptr& data, int32_t* lengths) { + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { AddFixedLength(num_bytes, data->length, lengths); }; return Status::OK(); @@ -459,53 +462,60 @@ struct GroupByImpl : public ScalarAggregator { AddLengthImpl add_length_impl; }; - using EncodeNextImpl = std::function&, uint8_t**)>; + using EncodeNextImpl = + std::function&, uint8_t**)>; struct GetEncodeNextImpl { - template - static void EncodeSmallFixed(const std::shared_ptr& data, uint8_t** encoded_bytes) { + static void EncodeSmallFixed(const std::shared_ptr& data, + uint8_t** encoded_bytes) { auto raw_input = data->buffers[1]->data(); auto offset = data->offset; if (data->MayHaveNulls()) { const uint8_t* nulls = data->buffers[0]->data(); for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; bool is_null = !BitUtil::GetBit(nulls, offset + i); encoded_ptr[0] = is_null ? 1 : 0; encoded_ptr += 1; uint64_t null_multiplier = is_null ? 0 : 1; if (NumBits == 1) { - encoded_ptr[0] = static_cast(null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); + encoded_ptr[0] = static_cast( + null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); encoded_ptr += 1; } if (NumBits == 8) { - encoded_ptr[0] = static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + encoded_ptr[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); encoded_ptr += 1; } if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); encoded_ptr += 2; } if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); encoded_ptr += 4; } if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast(raw_input)[offset + i]); + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); encoded_ptr += 8; } } } else { for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; encoded_ptr[0] = 0; encoded_ptr += 1; if (NumBits == 1) { - encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); + encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); encoded_ptr += 1; } if (NumBits == 8) { @@ -513,28 +523,32 @@ struct GroupByImpl : public ScalarAggregator { encoded_ptr += 1; } if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; encoded_ptr += 2; } if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; encoded_ptr += 4; } if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = reinterpret_cast(raw_input)[offset + i]; + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; encoded_ptr += 8; - } + } } } } - static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, uint8_t** encoded_bytes) { + static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, + uint8_t** encoded_bytes) { auto raw_input = data->buffers[1]->data(); auto offset = data->offset; if (data->MayHaveNulls()) { const uint8_t* nulls = data->buffers[0]->data(); for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; bool is_null = !BitUtil::GetBit(nulls, offset + i); encoded_ptr[0] = is_null ? 1 : 0; encoded_ptr += 1; @@ -547,7 +561,7 @@ struct GroupByImpl : public ScalarAggregator { } } else { for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; encoded_ptr[0] = 0; encoded_ptr += 1; memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); @@ -556,7 +570,8 @@ struct GroupByImpl : public ScalarAggregator { } } - static void EncodeVarLength(const std::shared_ptr& data, uint8_t** encoded_bytes) { + static void EncodeVarLength(const std::shared_ptr& data, + uint8_t** encoded_bytes) { using offset_type = typename StringType::offset_type; auto offset = data->offset; const auto offsets = data->GetValues(1); @@ -564,7 +579,7 @@ struct GroupByImpl : public ScalarAggregator { if (data->MayHaveNulls()) { const uint8_t* nulls = data->buffers[0]->data(); for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; bool is_null = !BitUtil::GetBit(nulls, offset + i); if (is_null) { encoded_ptr[0] = 1; @@ -573,7 +588,7 @@ struct GroupByImpl : public ScalarAggregator { encoded_ptr += sizeof(offset_type); } else { encoded_ptr[0] = 0; - encoded_ptr++; + encoded_ptr++; size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; reinterpret_cast(encoded_ptr)[0] = num_bytes; encoded_ptr += sizeof(offset_type); @@ -583,9 +598,9 @@ struct GroupByImpl : public ScalarAggregator { } } else { for (int64_t i = 0; i < data->length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; encoded_ptr[0] = 0; - encoded_ptr++; + encoded_ptr++; size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; reinterpret_cast(encoded_ptr)[0] = num_bytes; encoded_ptr += sizeof(offset_type); @@ -600,27 +615,32 @@ struct GroupByImpl : public ScalarAggregator { int32_t num_bits = bit_width(input_type.id()); switch (num_bits) { case 1: - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeSmallFixed<1>(data, encoded_bytes); }; break; case 8: - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeSmallFixed<8>(data, encoded_bytes); }; break; case 16: - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeSmallFixed<16>(data, encoded_bytes); }; break; case 32: - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeSmallFixed<32>(data, encoded_bytes); }; break; case 64: - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeSmallFixed<64>(data, encoded_bytes); }; break; @@ -629,39 +649,44 @@ struct GroupByImpl : public ScalarAggregator { } Status Visit(const StringType&) { - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeVarLength(data, encoded_bytes); }; return Status::OK(); } Status Visit(const BinaryType&) { - encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeVarLength(data, encoded_bytes); }; - return Status::OK(); + return Status::OK(); } Status Visit(const FixedSizeBinaryType& type) { int32_t num_bytes = type.byte_width(); - encode_next_impl = [num_bytes](const std::shared_ptr& data, uint8_t** encoded_bytes) { + encode_next_impl = [num_bytes](const std::shared_ptr& data, + uint8_t** encoded_bytes) { EncodeBigFixed(num_bytes, data, encoded_bytes); }; - return Status::OK(); + return Status::OK(); } EncodeNextImpl encode_next_impl; }; - using DecodeNextImpl = std::function*)>; + using DecodeNextImpl = std::function*)>; struct GetDecodeNextImpl { - - static void DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* null_buf, int32_t &null_count) { + static void DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, + int32_t& null_count) { // Do we have nulls? null_count = 0; for (int32_t i = 0; i < length; ++i) { - null_count += encoded_bytes[i][0]; + null_count += encoded_bytes[i][0]; } if (null_count > 0) { ctx->SetStatus(ctx->Allocate((length + 7) / 8).Value(null_buf)); @@ -681,16 +706,20 @@ struct GroupByImpl : public ScalarAggregator { } template - static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, + int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); + KERNEL_ASSIGN_OR_RAISE( + auto key_buf, ctx, + ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); uint8_t* raw_output = key_buf->mutable_data(); for (int32_t i = 0; i < length; ++i) { - auto &encoded_ptr = encoded_bytes[i]; + auto& encoded_ptr = encoded_bytes[i]; if (NumBits == 1) { BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); encoded_ptr += 1; @@ -700,15 +729,18 @@ struct GroupByImpl : public ScalarAggregator { encoded_ptr += 1; } if (NumBits == 16) { - reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; encoded_ptr += 2; } if (NumBits == 32) { - reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; encoded_ptr += 4; } if (NumBits == 64) { - reinterpret_cast(raw_output)[i] = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; encoded_ptr += 8; } } @@ -747,7 +779,8 @@ struct GroupByImpl : public ScalarAggregator { } } - static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, + uint8_t** encoded_bytes, std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); @@ -759,10 +792,13 @@ struct GroupByImpl : public ScalarAggregator { encoded_bytes[i] += num_bytes; } - *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, null_count); + *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, + null_count); } - static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); @@ -774,7 +810,8 @@ struct GroupByImpl : public ScalarAggregator { length_sum += reinterpret_cast(encoded_bytes)[0]; } - KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, ctx->Allocate(sizeof(offset_type) * (1 + length))); + KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, + ctx->Allocate(sizeof(offset_type) * (1 + length))); KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); auto raw_offsets = offset_buf->mutable_data(); @@ -791,9 +828,11 @@ struct GroupByImpl : public ScalarAggregator { reinterpret_cast(raw_offsets)[length] = current_offset; if (is_string) { - *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, null_count, 0); + *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); } else { - *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, null_count, 0); + *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); } } @@ -803,54 +842,68 @@ struct GroupByImpl : public ScalarAggregator { auto type_id = input_type.id(); switch (num_bits) { case 1: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); }; break; case 8: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); }; break; case 16: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); }; break; case 32: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); }; break; case 64: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); }; break; } - return Status::OK(); + return Status::OK(); } Status Visit(const StringType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeVarLength(ctx, true, length, encoded_bytes, out); }; - return Status::OK(); + return Status::OK(); } Status Visit(const BinaryType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeVarLength(ctx, false, length, encoded_bytes, out); }; - return Status::OK(); + return Status::OK(); } Status Visit(const FixedSizeBinaryType& type) { int32_t num_bytes = type.byte_width(); - decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { + decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); }; - return Status::OK(); + return Status::OK(); } DecodeNextImpl decode_next_impl; @@ -897,7 +950,9 @@ struct GroupByImpl : public ScalarAggregator { group_ids_batch_.resize(batch.length); for (int64_t i = 0; i < batch.length; ++i) { int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; - std::string key(reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), key_length); + std::string key( + reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), + key_length); auto iter = map_.find(key); if (iter == map_.end()) { group_ids_batch_[i] = n_groups++; @@ -942,7 +997,8 @@ struct GroupByImpl : public ScalarAggregator { int64_t length = n_groups; for (size_t i = 0; i < n_keys; ++i) { std::shared_ptr key_array; - decode_next_impl[i].decode_next_impl(ctx, static_cast(length), key_buf_ptrs_.data(), &key_array); + decode_next_impl[i].decode_next_impl(ctx, static_cast(length), + key_buf_ptrs_.data(), &key_array); out_columns[aggregators.size() + i] = std::move(key_array); } @@ -1023,8 +1079,7 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg size_t n_keys = args.inputs.size() - aggregates.size(); for (size_t i = 0; i < n_keys; ++i) { const auto& key_type = args.inputs[aggregates.size() + i].type; - switch (key_type->id()) - { + switch (key_type->id()) { // Supported types of keys case Type::BOOL: case Type::UINT8: diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 01e71481ab6..1860c5194b5 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -345,15 +345,16 @@ TEST(GroupBy, SumOnly) { TEST(GroupBy, StringKey) { auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - auto aggregand = ArrayFromJSON(int64(), - "[10, 5, 4, 2, 12, 9]"); + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); GroupByOptions options; options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr, "sum"}}; options.key_names = {"key"}; ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", {aggregand, key}, &options)); auto aggregated_and_grouped = boxed.array_as(); - auto result_sum = checked_pointer_cast(aggregated_and_grouped->GetFieldByName("sum")); - auto result_key = checked_pointer_cast(aggregated_and_grouped->GetFieldByName("key")); + auto result_sum = + checked_pointer_cast(aggregated_and_grouped->GetFieldByName("sum")); + auto result_key = + checked_pointer_cast(aggregated_and_grouped->GetFieldByName("key")); ASSERT_EQ(result_key->length(), 4); for (int64_t i = 0; i < result_key->length(); ++i) { int32_t key_length; From 879b6fd7ccb43d4e8e5aef8cf1b6a4bfa3ec7baf Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Feb 2021 13:39:12 -0500 Subject: [PATCH 08/49] add simple group_by benchmark --- .../compute/kernels/aggregate_benchmark.cc | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index c90dd03c06e..74038743ca1 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -300,6 +300,56 @@ BENCHMARK_TEMPLATE(ReferenceSum, SumBitmapVectorizeUnroll) ->Apply(BenchmarkSetArgs); #endif // ARROW_WITH_BENCHMARKS_REFERENCE +// +// GroupBy +// + +static void BenchmarkGroupBy(benchmark::State& state, + std::vector aggregates, + std::vector aggregands, std::vector keys) { + std::vector arguments; + for (const Datum& aggregand : aggregands) { + arguments.push_back(aggregand); + } + + for (const Datum& key : keys) { + arguments.push_back(key); + } + + GroupByOptions options; + options.aggregates = aggregates; + options.key_names.resize(keys.size(), "ignored"); + + for (auto _ : state) { + ABORT_NOT_OK(CallFunction("group_by", arguments, &options).status()); + } +} + +#define GROUP_BY_BENCHMARK(Name, Impl) \ + static void Name(benchmark::State& state) { \ + RegressionArgs args(state, false); \ + auto rng = random::RandomArrayGenerator(1923); \ + (Impl)(); \ + } \ + BENCHMARK(Name)->Apply([](benchmark::internal::Benchmark* bench) { \ + BenchmarkSetArgsWithSizes(bench, {1 * 1024 * 1024}); \ + }) + +GROUP_BY_BENCHMARK(SumDoublesGroupedByStrings, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.StringWithRepeats(args.size, + /*unique=*/16, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"sum", NULLPTR, "summed f64"}}, {summand}, {key}); +}); + // // Sum // From 6377dd9db66ac1d17cc28f0bb86b2f99cad432eb Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Feb 2021 15:19:47 -0500 Subject: [PATCH 09/49] add group_by benchmark keyed on integers --- .../arrow/compute/kernels/aggregate_benchmark.cc | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index 74038743ca1..a55673eba9a 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -335,7 +335,7 @@ static void BenchmarkGroupBy(benchmark::State& state, BenchmarkSetArgsWithSizes(bench, {1 * 1024 * 1024}); \ }) -GROUP_BY_BENCHMARK(SumDoublesGroupedByStrings, [&] { +GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { auto summand = rng.Float64(args.size, /*min=*/0.0, /*max=*/1.0e14, @@ -350,6 +350,20 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedByStrings, [&] { BenchmarkGroupBy(state, {{"sum", NULLPTR, "summed f64"}}, {summand}, {key}); }); +GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.Int64(args.size, + /*min=*/0, + /*max=*/15); + + BenchmarkGroupBy(state, {{"sum", NULLPTR, "summed f64"}}, {summand}, {key}); +}); + // // Sum // From 04e4f76fa2236f73cd99ed86ed6f0ba40db8f8d3 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Feb 2021 16:04:10 -0500 Subject: [PATCH 10/49] store key bytes -> group id mapping with unordered_map --- cpp/src/arrow/compute/kernels/aggregate_basic.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 8fe2279533b..6ab074f3750 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include +#include #include "arrow/compute/api_aggregate.h" #include "arrow/compute/kernels/aggregate_basic_internal.h" @@ -1010,7 +1010,7 @@ struct GroupByImpl : public ScalarAggregator { std::vector key_buf_ptrs_; std::vector group_ids_batch_; - std::map map_; + std::unordered_map map_; std::vector offsets_; std::vector key_bytes_; uint32_t n_groups; From 2b7b76efdad4f8e1b7cef7cc03397b986127d482 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 25 Feb 2021 16:54:57 -0500 Subject: [PATCH 11/49] repair null bitmap materialization, simplify testing against random data --- cpp/src/arrow/compute/api_aggregate.cc | 10 + cpp/src/arrow/compute/api_aggregate.h | 37 +- .../arrow/compute/kernels/aggregate_basic.cc | 153 +++--- .../arrow/compute/kernels/aggregate_test.cc | 434 +++++++++--------- cpp/src/arrow/dataset/expression.cc | 2 +- cpp/src/arrow/scalar.cc | 2 +- cpp/src/arrow/scalar.h | 8 +- 7 files changed, 349 insertions(+), 297 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.cc b/cpp/src/arrow/compute/api_aggregate.cc index 5afa1048960..98768cf3bfc 100644 --- a/cpp/src/arrow/compute/api_aggregate.cc +++ b/cpp/src/arrow/compute/api_aggregate.cc @@ -73,5 +73,15 @@ Result TDigest(const Datum& value, const TDigestOptions& options, return CallFunction("tdigest", {value}, &options, ctx); } +Result GroupBy(const std::vector& aggregands, + const std::vector& keys, const GroupByOptions& options, + ExecContext* ctx) { + std::vector args = aggregands; + for (const Datum& key : keys) { + args.push_back(key); + } + return CallFunction("group_by", args, &options, ctx); +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index ab0b3ae1941..f8d9c91fbeb 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -149,7 +149,7 @@ struct ARROW_EXPORT TDigestOptions : public FunctionOptions { uint32_t buffer_size; }; -// TODO(michalursa) add docstring +/// \brief Configure a grouped aggregation struct ARROW_EXPORT GroupByOptions : public FunctionOptions { struct Aggregate { /// the name of the aggregation function @@ -161,6 +161,12 @@ struct ARROW_EXPORT GroupByOptions : public FunctionOptions { /// the name of the resulting column in output std::string name; }; + + GroupByOptions() = default; + + GroupByOptions(std::vector aggregates, std::vector key_names) + : aggregates(std::move(aggregates)), key_names(std::move(key_names)) {} + std::vector aggregates; /// the names of key columns @@ -324,5 +330,34 @@ Result TDigest(const Datum& value, const TDigestOptions& options = TDigestOptions::Defaults(), ExecContext* ctx = NULLPTR); +/// \brief Calculate multiple aggregations grouped on multiple keys +/// +/// \param[in] aggregands datums to which aggregations will be applied +/// \param[in] keys datums which will be used to group the aggregations +/// \param[in] options GroupByOptions, encapsulating the names and options of aggregate +/// functions to be applied and the field names for results in the output. +/// \return a StructArray with len(aggregands) + len(keys) fields. The first +/// len(aggregands) fields are the results of the aggregations for the group +/// specified by keys in the final len(keys) fields. +/// +/// For example: +/// GroupByOptions options = { +/// .aggregates = { +/// {"sum", nullptr, "sum result"}, +/// {"mean", nullptr, "mean result"}, +/// }, +/// .key_names = {"str key", "date key"}, +/// }; +/// assert(*GroupBy({[2, 5, 8], [1.5, 2.0, 3.0]}, +/// {["a", "b", "a"], [today, today, today]}, +/// options).Equals([ +/// {"sum result": 10, "mean result": 2.25, "str key": "a", "date key": today}, +/// {"sum result": 5, "mean result": 2.0, "str key": "b", "date key": today}, +/// ])) +ARROW_EXPORT +Result GroupBy(const std::vector& aggregands, + const std::vector& keys, const GroupByOptions& options, + ExecContext* ctx = NULLPTR); + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 6ab074f3750..731be201ae7 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -119,6 +119,8 @@ struct GroupedAggregator { Resize(ctx, new_size); } } + + virtual std::shared_ptr out_type() const = 0; }; struct GroupedCountImpl : public GroupedAggregator { @@ -126,17 +128,17 @@ struct GroupedCountImpl : public GroupedAggregator { const DataType& input_type, const FunctionOptions* options) { auto out = ::arrow::internal::make_unique(); - out->options = checked_cast(*options); - ctx->SetStatus(ctx->Allocate(0).Value(&out->counts)); + out->options_ = checked_cast(*options); + ctx->SetStatus(ctx->Allocate(0).Value(&out->counts_)); return out; } void Resize(KernelContext* ctx, int64_t new_num_groups) override { auto old_size = num_groups(); - KERNEL_RETURN_IF_ERROR(ctx, counts->TypedResize(new_num_groups)); + KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); auto new_size = num_groups(); - auto raw_counts = reinterpret_cast(counts->mutable_data()); + auto raw_counts = reinterpret_cast(counts_->mutable_data()); for (auto i = old_size; i < new_size; ++i) { raw_counts[i] = 0; } @@ -147,17 +149,17 @@ struct GroupedCountImpl : public GroupedAggregator { MaybeResize(ctx, batch.length, group_ids); if (ctx->HasError()) return; - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - if (options.count_mode == CountOptions::COUNT_NON_NULL) { - auto input = batch[0].make_array(); + auto raw_counts = reinterpret_cast(counts_->mutable_data()); + auto input = batch[0].make_array(); + if (options_.count_mode == CountOptions::COUNT_NON_NULL) { for (int64_t i = 0; i < input->length(); ++i) { if (input->IsNull(i)) continue; raw_counts[group_ids[i]]++; } } else { for (int64_t i = 0; i < batch.length; ++i) { + if (input->IsValid(i)) continue; raw_counts[group_ids[i]]++; } } @@ -165,13 +167,15 @@ struct GroupedCountImpl : public GroupedAggregator { void Finalize(KernelContext* ctx, Datum* out) override { auto length = num_groups(); - *out = std::make_shared(length, std::move(counts)); + *out = std::make_shared(length, std::move(counts_)); } - int64_t num_groups() const override { return counts->size() / sizeof(int64_t); } + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - CountOptions options; - std::shared_ptr counts; + std::shared_ptr out_type() const override { return int64(); } + + CountOptions options_; + std::shared_ptr counts_; }; struct GroupedSumImpl : public GroupedAggregator { @@ -179,25 +183,27 @@ struct GroupedSumImpl : public GroupedAggregator { // we always have 64 bits per group in the sums buffer. static constexpr size_t kSumSize = sizeof(int64_t); - using ConsumeImpl = - std::function&, const uint32_t*, Buffer*)>; + using ConsumeImpl = std::function&, + const uint32_t*, Buffer*, Buffer*)>; struct GetConsumeImpl { template ::Type> Status Visit(const T&) { consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums) { + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { auto raw_input = reinterpret_cast::CType*>( input->buffers[1]->data()); auto raw_sums = reinterpret_cast::CType*>( sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); arrow::internal::VisitSetBitRunsVoid( input->buffers[0], input->offset, input->length, [&](int64_t begin, int64_t length) { for (int64_t i = begin, end = begin + length; i < end; ++i) { raw_sums[group_ids[i]] += raw_input[i]; + raw_counts[group_ids[i]] += 1; } }); }; @@ -207,15 +213,17 @@ struct GroupedSumImpl : public GroupedAggregator { Status Visit(const BooleanType&) { consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums) { + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { auto raw_input = input->buffers[1]->data(); auto raw_sums = reinterpret_cast(sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); arrow::internal::VisitSetBitRunsVoid( input->buffers[0], input->offset, input->length, [&](int64_t begin, int64_t length) { for (int64_t i = begin, end = begin + length; i < end; ++i) { raw_sums[group_ids[i]] += BitUtil::GetBit(raw_input, i); + raw_counts[group_ids[i]] += 1; } }); }; @@ -239,42 +247,64 @@ struct GroupedSumImpl : public GroupedAggregator { const DataType& input_type, const FunctionOptions* options) { auto out = ::arrow::internal::make_unique(); - ctx->SetStatus(ctx->Allocate(0).Value(&out->sums)); + + ctx->SetStatus(ctx->Allocate(0).Value(&out->sums_)); + if (ctx->HasError()) return nullptr; + + ctx->SetStatus(ctx->Allocate(0).Value(&out->counts_)); if (ctx->HasError()) return nullptr; GetConsumeImpl get_consume_impl; ctx->SetStatus(VisitTypeInline(input_type, &get_consume_impl)); - out->consume_impl = std::move(get_consume_impl.consume_impl); - out->out_type = std::move(get_consume_impl.out_type); + out->consume_impl_ = std::move(get_consume_impl.consume_impl); + out->out_type_ = std::move(get_consume_impl.out_type); return out; } void Resize(KernelContext* ctx, int64_t new_num_groups) override { auto old_size = num_groups() * kSumSize; - KERNEL_RETURN_IF_ERROR(ctx, sums->Resize(new_num_groups * kSumSize)); + KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); + KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); auto new_size = num_groups() * kSumSize; - std::memset(sums->mutable_data() + old_size, 0, new_size - old_size); + std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); + std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); } void Consume(KernelContext* ctx, const ExecBatch& batch, const uint32_t* group_ids) override { MaybeResize(ctx, batch.length, group_ids); if (ctx->HasError()) return; - consume_impl(batch[0].array(), group_ids, sums.get()); + consume_impl_(batch[0].array(), group_ids, sums_.get(), counts_.get()); } void Finalize(KernelContext* ctx, Datum* out) override { - auto length = num_groups(); - *out = ArrayData::Make(std::move(out_type), length, - {/*null_bitmap=*/nullptr, std::move(sums)}); + std::shared_ptr null_bitmap; + int64_t null_count = 0; + + for (int64_t i = 0; i < num_groups(); ++i) { + if (reinterpret_cast(counts_->data())[i] > 0) continue; + + if (null_bitmap == nullptr) { + KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); + BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); + } + + null_count += 1; + BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); + } + + *out = ArrayData::Make(std::move(out_type_), num_groups(), + {std::move(null_bitmap), std::move(sums_)}, null_count); } - int64_t num_groups() const override { return sums->size() / kSumSize; } + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - std::shared_ptr sums; - std::shared_ptr out_type; - ConsumeImpl consume_impl; + std::shared_ptr out_type() const override { return out_type_; } + + std::shared_ptr sums_, counts_; + std::shared_ptr out_type_; + ConsumeImpl consume_impl_; }; std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { @@ -409,6 +439,7 @@ struct GroupByImpl : public ScalarAggregator { const auto offsets = data->GetValues(1); if (data->MayHaveNulls()) { const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { bool is_null = !BitUtil::GetBit(nulls, offset + i); if (is_null) { @@ -680,20 +711,20 @@ struct GroupByImpl : public ScalarAggregator { std::shared_ptr*)>; struct GetDecodeNextImpl { - static void DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, - int32_t& null_count) { + static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, + int32_t* null_count) { // Do we have nulls? - null_count = 0; + *null_count = 0; for (int32_t i = 0; i < length; ++i) { - null_count += encoded_bytes[i][0]; + *null_count += encoded_bytes[i][0]; } - if (null_count > 0) { - ctx->SetStatus(ctx->Allocate((length + 7) / 8).Value(null_buf)); + if (*null_count > 0) { + ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); uint8_t* nulls = (*null_buf)->mutable_data(); - memset(nulls, 0, (length + 7) / 8); + memset(nulls, 0, (*null_buf)->size()); for (int32_t i = 0; i < length; ++i) { - if (encoded_bytes[i][0]) { + if (!encoded_bytes[i][0]) { BitUtil::SetBit(nulls, i); } encoded_bytes[i] += 1; @@ -703,6 +734,7 @@ struct GroupByImpl : public ScalarAggregator { encoded_bytes[i] += 1; } } + return Status ::OK(); } template @@ -711,7 +743,8 @@ struct GroupByImpl : public ScalarAggregator { std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; - DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); KERNEL_ASSIGN_OR_RAISE( auto key_buf, ctx, @@ -745,45 +778,16 @@ struct GroupByImpl : public ScalarAggregator { } } - switch (output_type) { - case Type::BOOL: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::UINT8: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::INT8: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::UINT16: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::INT16: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::UINT32: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::INT32: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::UINT64: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - break; - case Type::INT64: - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - // *out = std::make_shared(length, key_buf, null_buf); - break; - default: - break; - } + DCHECK(is_integer(output_type) || output_type == Type::BOOL); + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); } static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; - DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); auto raw_output = key_buf->mutable_data(); @@ -801,7 +805,8 @@ struct GroupByImpl : public ScalarAggregator { std::shared_ptr* out) { std::shared_ptr null_buf; int32_t null_count; - DecodeNulls(ctx, length, encoded_bytes, &null_buf, null_count); + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); using offset_type = typename StringType::offset_type; @@ -1073,7 +1078,7 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg } if (ctx->HasError()) return nullptr; - out_fields[i] = field(aggregates[i].name, input_type); + out_fields[i] = field(aggregates[i].name, impl->aggregators[i]->out_type()); } size_t n_keys = args.inputs.size() - aggregates.size(); diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 1860c5194b5..185fdfc454a 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -254,8 +254,53 @@ Result> ApplyGroupings(const ListArray& groupings, groupings.value_offsets(), sorted.make_array()); } -Result> NaiveGroupBy(GroupByOptions options, - ArrayVector aggregands, ArrayVector keys) { +struct ScalarsToScalarsMap { + struct Eq { + bool operator()(const ScalarVector& l, const ScalarVector& r) const { + return l.size() == r.size() && + std::equal(l.begin(), l.end(), r.begin(), + [](const std::shared_ptr& l, + const std::shared_ptr& r) { return l->Equals(r); }); + } + }; + + struct Hash { + size_t operator()(const ScalarVector& vec) const { + size_t hash = 0; + for (const auto& scalar : vec) { + hash ^= scalar->hash(); + } + return hash; + } + }; + + static std::string ToString(const ScalarVector& vec) { + std::string out = "["; + int i = 0; + for (const auto& scalar : vec) { + if (i++ != 0) { + out += ","; + } + out += scalar->ToString(); + } + out += "]"; + return out; + } + + std::string ToString() const { + std::string out = "{\n"; + for (const auto& key_value : map) { + out += " " + ToString(key_value.first) + ": " + ToString(key_value.second) + "\n"; + } + out += "}\n"; + return out; + } + + std::unordered_map map; +}; + +Result NaiveGroupBy(GroupByOptions options, ArrayVector aggregands, + ArrayVector keys) { ARROW_ASSIGN_OR_RAISE(auto keys_struct, StructArray::Make(keys, options.key_names)); ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); @@ -267,7 +312,7 @@ Result> NaiveGroupBy(GroupByOptions options, int64_t n_groups = groupings->length(); - std::vector out(n_groups); + std::vector grouped_aggregates(n_groups); auto aggregate_spec = options.aggregates.begin(); for (const auto& aggregand : aggregands) { @@ -277,219 +322,215 @@ Result> NaiveGroupBy(GroupByOptions options, ARROW_ASSIGN_OR_RAISE(auto grouped_aggregate, CallFunction(aggregate_spec->function, {grouped_aggregand->value_slice(i_group)})); - out[i_group].push_back(grouped_aggregate.scalar()); + grouped_aggregates[i_group].push_back(grouped_aggregate.scalar()); } ++aggregate_spec; } + ScalarsToScalarsMap out; + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { ARROW_ASSIGN_OR_RAISE(auto keys_for_group, keys_unique->GetScalar(i_group)); - for (const auto& key : checked_cast(*keys_for_group).value) { - out[i_group].push_back(key); - } + out.map.emplace(checked_cast(*keys_for_group).value, + std::move(grouped_aggregates[i_group])); } return out; } -} // namespace -} // namespace group_helpers - -// TODO(michalursa) add tests -TEST(GroupBy, SumOnly) { - auto key = ArrayFromJSON(int64(), - "[1, 2, 1," - "3, 2, 3]"); - auto aggregand = ArrayFromJSON(float64(), - "[1.0, 0.0, null, " - "3.25, 0.125, -0.25]"); - - ASSERT_EQ(key->length(), aggregand->length()); - - GroupByOptions options; - options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr, "f64 summed"}}; - options.key_names = {"i64 key"}; +void ValidateGroupBy(GroupByOptions options, ArrayVector aggregands, ArrayVector keys) { + ASSERT_OK_AND_ASSIGN(auto expected, + group_helpers::NaiveGroupBy(options, aggregands, keys)); - ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", - { - aggregand, - key, - }, - &options)); + std::vector arguments; + for (const auto& aggregand : aggregands) { + arguments.emplace_back(aggregand->data()); + } + for (const auto& key : keys) { + arguments.emplace_back(key->data()); + } - auto aggregated_and_grouped = boxed.array_as(); + const int n_aggregands = static_cast(aggregands.size()); + const int n_keys = static_cast(keys.size()); - auto f64_summed = checked_pointer_cast( - aggregated_and_grouped->GetFieldByName("f64 summed")); + ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, + CallFunction("group_by", arguments, &options).Map([](Datum d) { + return d.array_as(); + })); - auto i64_key = - checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); + ScalarsToScalarsMap actual; + for (int64_t i_group = 0; i_group < aggregated_and_grouped->length(); ++i_group) { + ScalarVector aggregated_i(n_aggregands), keys_i(n_keys); - ASSERT_EQ(i64_key->length(), 3); + for (int i_field = 0; i_field < aggregated_and_grouped->num_fields(); ++i_field) { + if (i_field < n_aggregands) { + ASSERT_OK_AND_ASSIGN(aggregated_i[i_field], + aggregated_and_grouped->field(i_field)->GetScalar(i_group)); + } else { + ASSERT_OK_AND_ASSIGN(keys_i[i_field - n_aggregands], + aggregated_and_grouped->field(i_field)->GetScalar(i_group)); + } + } - for (int64_t i = 0; i < i64_key->length(); ++i) { - int64_t key = i64_key->Value(i); + actual.map.emplace(std::move(keys_i), std::move(aggregated_i)); + } - if (key == 1) { - ASSERT_EQ(f64_summed->Value(i), 1.0); - } - if (key == 2) { - ASSERT_EQ(f64_summed->Value(i), 0.125); - } - if (key == 3) { - ASSERT_EQ(f64_summed->Value(i), 3.0); + ScalarsToScalarsMap unexpected; + for (const auto& pair : actual.map) { + if (expected.map.find(pair.first) == expected.map.end()) { + unexpected.map.insert(pair); } } -} + EXPECT_TRUE(unexpected.map.empty()) << " unexpected groups: " << unexpected.ToString(); -TEST(GroupBy, StringKey) { - auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); - GroupByOptions options; - options.aggregates = {GroupByOptions::Aggregate{"sum", nullptr, "sum"}}; - options.key_names = {"key"}; - ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", {aggregand, key}, &options)); - auto aggregated_and_grouped = boxed.array_as(); - auto result_sum = - checked_pointer_cast(aggregated_and_grouped->GetFieldByName("sum")); - auto result_key = - checked_pointer_cast(aggregated_and_grouped->GetFieldByName("key")); - ASSERT_EQ(result_key->length(), 4); - for (int64_t i = 0; i < result_key->length(); ++i) { - int32_t key_length; - const uint8_t* key_chars = result_key->GetValue(i, &key_length); - std::string key_str((char*)key_chars, key_length); - if (key_str.compare("alfa") == 0) { - ASSERT_EQ(result_sum->Value(i), 10); - } - if (key_str.compare("beta") == 0) { - ASSERT_EQ(result_sum->Value(i), 14); - } - if (key_str.compare("gamma") == 0) { - ASSERT_EQ(result_sum->Value(i), 6); - } - if (key_str.compare("") == 0) { - ASSERT_EQ(result_sum->Value(i), 12); + ScalarsToScalarsMap missing; + for (const auto& pair : expected.map) { + if (actual.map.find(pair.first) == actual.map.end()) { + missing.map.insert(pair); } } -} + EXPECT_TRUE(missing.map.empty()) << " missing groups: " << missing.ToString(); -TEST(GroupBy, CountOnly) { - auto key = ArrayFromJSON(int64(), - "[1, 2, 1," - "3, 2, 3]"); - auto aggregand = ArrayFromJSON(float64(), - "[1.0, 0.0, null, " - "3.25, 0.125, -0.25]"); + for (const auto& pair : expected.map) { + auto it = actual.map.find(pair.first); + if (it == actual.map.end()) continue; - ASSERT_EQ(key->length(), aggregand->length()); + const auto& expected_aggregated = pair.second; + const auto& actual_aggregated = it->second; - GroupByOptions options; - CountOptions count_options; - count_options.count_mode = CountOptions::COUNT_NON_NULL; - options.aggregates = { - GroupByOptions::Aggregate{"count", &count_options, "f64 counted"}}; - options.key_names = {"i64 key"}; - - ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", - { - aggregand, - key, - }, - &options)); - - auto aggregated_and_grouped = boxed.array_as(); - - auto f64_counted = checked_pointer_cast( - aggregated_and_grouped->GetFieldByName("f64 counted")); - - auto i64_key = - checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); + static ScalarsToScalarsMap::Eq eq; + EXPECT_TRUE(eq(expected_aggregated, actual_aggregated)) + << " group " << ScalarsToScalarsMap::ToString(pair.first) + << "\n had: " << ScalarsToScalarsMap::ToString(actual_aggregated) + << "\n expected: " << ScalarsToScalarsMap::ToString(expected_aggregated); + } +} - ASSERT_EQ(i64_key->length(), 3); +} // namespace +} // namespace group_helpers - for (int64_t i = 0; i < i64_key->length(); ++i) { - int64_t key = i64_key->Value(i); +TEST(GroupBy, SumOnly) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - if (key == 1) { - ASSERT_EQ(f64_counted->Value(i), 1); - } else { - ASSERT_EQ(f64_counted->Value(i), 2); - } - } + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + GroupBy({aggregand}, {key}, + GroupByOptions(/*aggregates=*/{{"sum", nullptr, "f64 summed"}}, + /*key_names=*/{"i64 key"}))); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("f64 summed", float64()), + field("i64 key", int64()), + }), + R"([ + [4.25, 1], + [-0.125, 2], + [null, 3], + [0.75, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); } TEST(GroupBy, CountAndSum) { - auto key = ArrayFromJSON(int64(), - "[1, 2, 1," - "3, 2, 3]"); - auto aggregand = ArrayFromJSON(float32(), - "[1.0, 0.0, null, " - "3.25, 0.125, -0.25]"); - - ASSERT_EQ(key->length(), aggregand->length()); + auto aggregand = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); - GroupByOptions options; CountOptions count_options; count_options.count_mode = CountOptions::COUNT_NON_NULL; - options.aggregates = { - GroupByOptions::Aggregate{"count", &count_options, "f32 counted"}, - GroupByOptions::Aggregate{"sum", nullptr, "f32 summed"}, - GroupByOptions::Aggregate{"sum", nullptr, "i64 summed"}, - }; - options.key_names = {"i64 key"}; - ASSERT_OK_AND_ASSIGN(Datum boxed, - CallFunction("group_by", - { - // NB: passing the same aggregand twice - aggregand, - aggregand, - // NB: passing the key column also as an aggregand - key, - key, - }, - &options)); - - auto aggregated_and_grouped = boxed.array_as(); - - auto f32_counted = checked_pointer_cast( - aggregated_and_grouped->GetFieldByName("f32 counted")); - - // NB: summing a float32 array results in float64 sums - auto f32_summed = checked_pointer_cast( - aggregated_and_grouped->GetFieldByName("f32 summed")); - - auto i64_summed = checked_pointer_cast( - aggregated_and_grouped->GetFieldByName("i64 summed")); - - auto i64_key = - checked_pointer_cast(aggregated_and_grouped->GetFieldByName("i64 key")); - - ASSERT_EQ(i64_key->length(), 3); - - for (int64_t i = 0; i < i64_key->length(); ++i) { - int64_t key = i64_key->Value(i); + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + // NB: passing an aggregand twice or also using it as a key is legal + GroupBy({aggregand, aggregand, key}, {key}, + GroupByOptions(/*aggregates=*/ + { + {"count", &count_options, "f32 counted"}, + {"sum", nullptr, "f32 summed"}, + {"sum", nullptr, "i64 summed"}, + }, + /*key_names=*/{"i64 key"}))); + + AssertDatumsEqual( + ArrayFromJSON(struct_({ + field("f32 counted", int64()), + // NB: summing a float32 array results in float64 sums + field("f32 summed", float64()), + field("i64 summed", int64()), + field("i64 key", int64()), + }), + R"([ + [1, 1.0, 2, 1], + [2, 0.125, 4, 2], + [2, 3.0, 6, 3], + [1, 0.75, null, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} - if (key == 1) { - ASSERT_EQ(f32_counted->Value(i), 1); - } else { - ASSERT_EQ(f32_counted->Value(i), 2); - } +TEST(GroupBy, StringKey) { + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); + auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - ASSERT_EQ(i64_summed->Value(i), key * 2); + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + GroupBy({aggregand}, {key}, + GroupByOptions(/*aggregates=*/{{"sum", nullptr, "i64 summed"}}, + /*key_names=*/{"str key"}))); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("i64 summed", int64()), + field("str key", utf8()), + }), + R"([ + [10, "alfa"], + [14, "beta"], + [6, "gamma"], + [12, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, MultipleKeys) { + auto aggregand = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); + auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); + auto str_key = + ArrayFromJSON(utf8(), R"(["beta", "beta", "gamma", "gamma", null, "beta"])"); - if (key == 1) { - ASSERT_EQ(f32_summed->Value(i), 1.0); - } - if (key == 2) { - ASSERT_EQ(f32_summed->Value(i), 0.125); - } - if (key == 3) { - ASSERT_EQ(f32_summed->Value(i), 3.0); - } - } + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + GroupBy({aggregand}, {int_key, str_key}, + GroupByOptions(/*aggregates=*/{{"sum", nullptr, "f32 summed"}}, + /*key_names=*/{"int key", "str key"}))); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("f32 summed", float64()), + field("int key", int32()), + field("str key", utf8()), + }), + R"([ + [0.125, 0, "beta"], + [2.5, 1, "beta"], + [-0.75, 0, "gamma"], + [8, 1, "gamma"], + [1.0, 0, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); + auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); + + group_helpers::ValidateGroupBy( + GroupByOptions(/*aggregates=*/{{"sum", nullptr, "i64 summed"}}, + /*key_names=*/{"str key"}), + {aggregand}, {key}); } TEST(GroupBy, RandomArraySum) { @@ -506,46 +547,7 @@ TEST(GroupBy, RandomArraySum) { auto summand = rand.Float32(length, -100, 100, null_probability); auto key = rand.Int64(length, 0, 12); - ASSERT_OK_AND_ASSIGN(auto expected, - group_helpers::NaiveGroupBy(options, {summand}, {key})); - auto n_groups = static_cast(expected.size()); - - ASSERT_OK_AND_ASSIGN(Datum boxed, CallFunction("group_by", - { - summand, - key, - }, - &options)); - auto actual = boxed.array_as(); - ASSERT_EQ(actual->length(), n_groups); - - std::vector> vexpected; - std::vector> vactual; - - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - const auto& expected_for_group = expected[i_group]; - auto actual_for_group = - checked_pointer_cast(*actual->GetScalar(i_group))->value; - - ASSERT_EQ(expected_for_group.size(), actual_for_group.size()); - ASSERT_EQ(expected_for_group.size(), 2); - - double expected_sum = ((DoubleScalar*)expected_for_group[0].get())->value; - int64_t expected_key = ((Int64Scalar*)expected_for_group[1].get())->value; - double actual_sum = ((DoubleScalar*)actual_for_group[0].get())->value; - int64_t actual_key = ((Int64Scalar*)actual_for_group[1].get())->value; - - vexpected.push_back(std::make_pair(expected_key, expected_sum)); - vactual.push_back(std::make_pair(actual_key, actual_sum)); - } - - std::sort(vexpected.begin(), vexpected.end()); - std::sort(vactual.begin(), vactual.end()); - - for (size_t i = 0; i < vexpected.size(); ++i) { - ASSERT_EQ(vexpected[i].first, vactual[i].first); - ASSERT_EQ(vexpected[i].second, vactual[i].second); - } + group_helpers::ValidateGroupBy(options, {summand}, {key}); } } } diff --git a/cpp/src/arrow/dataset/expression.cc b/cpp/src/arrow/dataset/expression.cc index 6e71aa17e74..627477b3038 100644 --- a/cpp/src/arrow/dataset/expression.cc +++ b/cpp/src/arrow/dataset/expression.cc @@ -284,7 +284,7 @@ bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.i size_t Expression::hash() const { if (auto lit = literal()) { if (lit->is_scalar()) { - return Scalar::Hash::hash(*lit->scalar()); + return lit->scalar()->hash(); } return 0; } diff --git a/cpp/src/arrow/scalar.cc b/cpp/src/arrow/scalar.cc index 399eac675f4..56a36114e49 100644 --- a/cpp/src/arrow/scalar.cc +++ b/cpp/src/arrow/scalar.cc @@ -145,7 +145,7 @@ struct ScalarHashImpl { size_t hash_; }; -size_t Scalar::Hash::hash(const Scalar& scalar) { return ScalarHashImpl(scalar).hash_; } +size_t Scalar::hash() const { return ScalarHashImpl(*this).hash_; } StringScalar::StringScalar(std::string s) : StringScalar(Buffer::FromString(std::move(s))) {} diff --git a/cpp/src/arrow/scalar.h b/cpp/src/arrow/scalar.h index e84e3fab900..24744859686 100644 --- a/cpp/src/arrow/scalar.h +++ b/cpp/src/arrow/scalar.h @@ -69,15 +69,15 @@ struct ARROW_EXPORT Scalar : public util::EqualityComparable { const EqualOptions& options = EqualOptions::Defaults()) const; struct ARROW_EXPORT Hash { - size_t operator()(const Scalar& scalar) const { return hash(scalar); } + size_t operator()(const Scalar& scalar) const { return scalar.hash(); } size_t operator()(const std::shared_ptr& scalar) const { - return hash(*scalar); + return scalar->hash(); } - - static size_t hash(const Scalar& scalar); }; + size_t hash() const; + std::string ToString() const; static Result> Parse(const std::shared_ptr& type, From 3b82be6b147e4ed9bf9da0206e4c0d6f40725cc7 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 26 Feb 2021 12:11:15 -0500 Subject: [PATCH 12/49] simplify ValidateGroupBy further --- .../arrow/compute/kernels/aggregate_test.cc | 215 ++++++++---------- 1 file changed, 101 insertions(+), 114 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 185fdfc454a..bf8a735b8e5 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -32,6 +32,7 @@ #include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/test_util.h" +#include "arrow/compute/registry.h" #include "arrow/type.h" #include "arrow/type_traits.h" #include "arrow/util/bitmap_reader.h" @@ -254,157 +255,143 @@ Result> ApplyGroupings(const ListArray& groupings, groupings.value_offsets(), sorted.make_array()); } -struct ScalarsToScalarsMap { - struct Eq { - bool operator()(const ScalarVector& l, const ScalarVector& r) const { - return l.size() == r.size() && - std::equal(l.begin(), l.end(), r.begin(), - [](const std::shared_ptr& l, - const std::shared_ptr& r) { return l->Equals(r); }); +struct ScalarVectorToArray { + template ::BuilderType, + typename ScalarType = typename TypeTraits::ScalarType> + Status UseBuilder(const AppendScalar& append) { + BuilderType builder(type(), default_memory_pool()); + for (const auto& s : scalars_) { + if (s->is_valid) { + RETURN_NOT_OK(append(checked_cast(*s), &builder)); + } else { + RETURN_NOT_OK(builder.AppendNull()); + } } - }; + return builder.FinishInternal(&data_); + } - struct Hash { - size_t operator()(const ScalarVector& vec) const { - size_t hash = 0; - for (const auto& scalar : vec) { - hash ^= scalar->hash(); - } - return hash; + struct AppendValue { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + return builder->Append(s.value); } }; - static std::string ToString(const ScalarVector& vec) { - std::string out = "["; - int i = 0; - for (const auto& scalar : vec) { - if (i++ != 0) { - out += ","; - } - out += scalar->ToString(); + struct AppendBuffer { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + const Buffer& buffer = *s.value; + return builder->Append(util::string_view{buffer}); } - out += "]"; - return out; + }; + + template + enable_if_primitive_ctype Visit(const T&) { + return UseBuilder(AppendValue{}); } - std::string ToString() const { - std::string out = "{\n"; - for (const auto& key_value : map) { - out += " " + ToString(key_value.first) + ": " + ToString(key_value.second) + "\n"; + template + enable_if_has_string_view Visit(const T&) { + return UseBuilder(AppendBuffer{}); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("ScalarVectorToArray for type ", type); + } + + Result Convert(ScalarVector scalars) && { + if (scalars.size() == 0) { + return Status::NotImplemented("ScalarVectorToArray with no scalars"); } - out += "}\n"; - return out; + scalars_ = std::move(scalars); + RETURN_NOT_OK(VisitTypeInline(*type(), this)); + return Datum(std::move(data_)); } - std::unordered_map map; + const std::shared_ptr& type() { return scalars_[0]->type; } + + ScalarVector scalars_; + std::shared_ptr data_; }; -Result NaiveGroupBy(GroupByOptions options, ArrayVector aggregands, - ArrayVector keys) { - ARROW_ASSIGN_OR_RAISE(auto keys_struct, StructArray::Make(keys, options.key_names)); +Result NaiveGroupBy(std::vector aggregands, std::vector keys, + GroupByOptions options) { + ArrayVector keys_arrays; + for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); + ARROW_ASSIGN_OR_RAISE(auto keys_struct, + StructArray::Make(std::move(keys_arrays), options.key_names)); + ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); auto groupings = checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); - auto keys_unique = - checked_pointer_cast(groupings_and_values->GetFieldByName("values")); - int64_t n_groups = groupings->length(); - std::vector grouped_aggregates(n_groups); + std::vector out_names; + ArrayVector out_columns; - auto aggregate_spec = options.aggregates.begin(); - for (const auto& aggregand : aggregands) { - ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, ApplyGroupings(*groupings, *aggregand)); + for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { + const Datum& aggregand = aggregands[i_agg]; + const std::string& function = options.aggregates[i_agg].function; + out_names.push_back(options.aggregates[i_agg].name); - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE(auto grouped_aggregate, - CallFunction(aggregate_spec->function, - {grouped_aggregand->value_slice(i_group)})); - grouped_aggregates[i_group].push_back(grouped_aggregate.scalar()); - } + ScalarVector aggregated_scalars; - ++aggregate_spec; - } + if (n_groups > 0) { + ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, + ApplyGroupings(*groupings, *aggregand.make_array())); - ScalarsToScalarsMap out; + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ARROW_ASSIGN_OR_RAISE( + Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); + aggregated_scalars.push_back(d.scalar()); + } + } else { + DCHECK_EQ(aggregand.length(), 0); + ARROW_ASSIGN_OR_RAISE(Datum d, CallFunction(function, {aggregand})); + aggregated_scalars.push_back(d.scalar()); + } - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE(auto keys_for_group, keys_unique->GetScalar(i_group)); - out.map.emplace(checked_cast(*keys_for_group).value, - std::move(grouped_aggregates[i_group])); + ARROW_ASSIGN_OR_RAISE(Datum aggregated_column, + ScalarVectorToArray{}.Convert(std::move(aggregated_scalars))); + out_columns.push_back(aggregated_column.make_array()); } - return out; -} - -void ValidateGroupBy(GroupByOptions options, ArrayVector aggregands, ArrayVector keys) { - ASSERT_OK_AND_ASSIGN(auto expected, - group_helpers::NaiveGroupBy(options, aggregands, keys)); - - std::vector arguments; - for (const auto& aggregand : aggregands) { - arguments.emplace_back(aggregand->data()); - } - for (const auto& key : keys) { - arguments.emplace_back(key->data()); + keys_struct = + checked_pointer_cast(groupings_and_values->GetFieldByName("values")); + for (size_t i_key = 0; i_key < aggregands.size(); ++i_key) { + out_names.push_back(options.key_names[i_key]); + out_columns.push_back(keys_struct->field(i_key)); } - const int n_aggregands = static_cast(aggregands.size()); - const int n_keys = static_cast(keys.size()); + return StructArray::Make(std::move(out_columns), std::move(out_names)); +} - ASSERT_OK_AND_ASSIGN(auto aggregated_and_grouped, - CallFunction("group_by", arguments, &options).Map([](Datum d) { - return d.array_as(); - })); +void ValidateGroupBy(GroupByOptions options, std::vector aggregands, + std::vector keys) { + ASSERT_OK_AND_ASSIGN(Datum expected, + group_helpers::NaiveGroupBy(aggregands, keys, options)); - ScalarsToScalarsMap actual; - for (int64_t i_group = 0; i_group < aggregated_and_grouped->length(); ++i_group) { - ScalarVector aggregated_i(n_aggregands), keys_i(n_keys); + ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, options)); - for (int i_field = 0; i_field < aggregated_and_grouped->num_fields(); ++i_field) { - if (i_field < n_aggregands) { - ASSERT_OK_AND_ASSIGN(aggregated_i[i_field], - aggregated_and_grouped->field(i_field)->GetScalar(i_group)); - } else { - ASSERT_OK_AND_ASSIGN(keys_i[i_field - n_aggregands], - aggregated_and_grouped->field(i_field)->GetScalar(i_group)); - } - } + // Ordering of groups is not important, so sort by key columns to ensure the comparison + // doesn't fail spuriously - actual.map.emplace(std::move(keys_i), std::move(aggregated_i)); + SortOptions sort_options; + for (const std::string& key_name : options.key_names) { + sort_options.sort_keys.emplace_back(key_name); } - ScalarsToScalarsMap unexpected; - for (const auto& pair : actual.map) { - if (expected.map.find(pair.first) == expected.map.end()) { - unexpected.map.insert(pair); - } + for (Datum* out : {&expected, &actual}) { + ASSERT_OK_AND_ASSIGN(Datum batch, + RecordBatch::FromStructArray(out->array_as())); + ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(batch, sort_options)); + ASSERT_OK_AND_ASSIGN(*out, Take(*out, sort_indices, TakeOptions::NoBoundsCheck())); } - EXPECT_TRUE(unexpected.map.empty()) << " unexpected groups: " << unexpected.ToString(); - ScalarsToScalarsMap missing; - for (const auto& pair : expected.map) { - if (actual.map.find(pair.first) == actual.map.end()) { - missing.map.insert(pair); - } - } - EXPECT_TRUE(missing.map.empty()) << " missing groups: " << missing.ToString(); - - for (const auto& pair : expected.map) { - auto it = actual.map.find(pair.first); - if (it == actual.map.end()) continue; - - const auto& expected_aggregated = pair.second; - const auto& actual_aggregated = it->second; - - static ScalarsToScalarsMap::Eq eq; - EXPECT_TRUE(eq(expected_aggregated, actual_aggregated)) - << " group " << ScalarsToScalarsMap::ToString(pair.first) - << "\n had: " << ScalarsToScalarsMap::ToString(actual_aggregated) - << "\n expected: " << ScalarsToScalarsMap::ToString(expected_aggregated); - } + AssertDatumsEqual(expected, actual, /*verbose=*/true); } } // namespace From 0d7144276abd8e53ad66969d2eda220c490c2c0b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 1 Mar 2021 16:37:41 -0500 Subject: [PATCH 13/49] update compute.rst with group_by --- docs/source/cpp/compute.rst | 63 +++++++++++++++++++++++++++++++------ 1 file changed, 54 insertions(+), 9 deletions(-) diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst index 00016330075..5224a91492e 100644 --- a/docs/source/cpp/compute.rst +++ b/docs/source/cpp/compute.rst @@ -86,20 +86,20 @@ Many compute functions are also available directly as concrete APIs, here Some functions accept or require an options structure that determines the exact semantics of the function:: - MinMaxOptions options; - options.null_handling = MinMaxOptions::EMIT_NULL; + MinMaxOptions min_max_options; + min_max_options.null_handling = MinMaxOptions::EMIT_NULL; std::shared_ptr array = ...; arrow::Datum min_max_datum; - ARROW_ASSIGN_OR_RAISE(min_max_datum, - arrow::compute::CallFunction("min_max", {array}, &options)); + ARROW_ASSIGN_OR_RAISE(min_max, + arrow::compute::CallFunction("min_max", {array}, + &min_max_options)); // Unpack struct scalar result (a two-field {"min", "max"} scalar) - const auto& min_max_scalar = \ - static_cast(*min_max_datum.scalar()); - const auto min_value = min_max_scalar.value[0]; - const auto max_value = min_max_scalar.value[1]; + const auto& min_max_scalar = min_max.scalar_as(); + std::shared_ptr min_value = min_max_scalar.value[0]; + std::shared_ptr max_value = min_max_scalar.value[1]; .. seealso:: :doc:`Compute API reference ` @@ -212,6 +212,8 @@ Aggregations +--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ | variance | Unary | Numeric | Scalar Float64 | :struct:`VarianceOptions` | +--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ +| group_by | VarArgs | Any | Array Struct | :struct:`GroupByOptions` | ++--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ Notes: @@ -228,6 +230,48 @@ Notes: * \(4) Output is Int64, UInt64 or Float64, depending on the input type. +Grouped Aggregations +~~~~~~~~~~~~~~~~~~~~ + +Aggregations can be grouped by one or more keys using the ``group_by`` +function. :member:`GroupByOptions::aggregates` is a vector specifying which +aggregations will be performed: each element is a +:struct:`GroupByOptions::Aggregate` containing the name of an aggregate +function and a pointer to a :class:`FunctionOptions`. The first arguments to +``group_by`` are interpreted as the corresponding aggregands and the remainder +will be used as grouping keys. The output will be an array with the same +number of fields where each slot contains the aggregation result and keys +for a group:: + + GroupByOptions options{ + {"sum", nullptr}, // first argument will be summed + {"min_max", + &min_max_options}, // second argument's extrema will be found + }; + + std::shared_ptr needs_sum = ...; + std::shared_ptr needs_min_max = ...; + std::shared_ptr key_0 = ...; + std::shared_ptr key_1 = ...; + + ARROW_ASSIGN_OR_RAISE(arrow::Datum out, + arrow::compute::CallFunction("group_by", + { + needs_sum, + needs_min_max, + key_0, + key_1, + }, + &options)); + + // Unpack struct array result (a four-field array) + auto out_array = out.array_as(); + std::shared_ptr sums = out_array->field(0); + std::shared_ptr mins_and_maxes = out_array->field(1); + std::shared_ptr group_key_0 = out_array->field(2); + std::shared_ptr group_key_1 = out_array->field(3); + + Element-wise ("scalar") functions --------------------------------- @@ -717,7 +761,8 @@ Associative transforms * \(2) Duplicates are removed from the output while the original order is maintained. -* \(3) Output is a ``{"values": input type, "counts": Int64}`` Struct. +* \(3) Output is a ``{ + "values" : input type, "counts" : Int64}`` Struct. Each output element corresponds to a unique value in the input, along with the number of times this value has appeared. From 9bf51e6b67f8cf562fe401180686aa41223dbf58 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 1 Mar 2021 17:11:28 -0500 Subject: [PATCH 14/49] remove named output fields --- cpp/src/arrow/compute/api_aggregate.h | 12 +-- .../arrow/compute/kernels/aggregate_basic.cc | 6 +- .../arrow/compute/kernels/aggregate_test.cc | 100 ++++++++---------- 3 files changed, 54 insertions(+), 64 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index f8d9c91fbeb..e92a7adf768 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -157,20 +157,16 @@ struct ARROW_EXPORT GroupByOptions : public FunctionOptions { /// options for the aggregation function const FunctionOptions* options; - - /// the name of the resulting column in output - std::string name; }; GroupByOptions() = default; - GroupByOptions(std::vector aggregates, std::vector key_names) - : aggregates(std::move(aggregates)), key_names(std::move(key_names)) {} + GroupByOptions(std::initializer_list aggregates) : aggregates(aggregates) {} - std::vector aggregates; + explicit GroupByOptions(std::vector aggregates) + : aggregates(std::move(aggregates)) {} - /// the names of key columns - std::vector key_names; + std::vector aggregates; }; /// @} diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 731be201ae7..5cc1404dedb 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -961,7 +961,7 @@ struct GroupByImpl : public ScalarAggregator { auto iter = map_.find(key); if (iter == map_.end()) { group_ids_batch_[i] = n_groups++; - int32_t next_key_offset = static_cast(key_bytes_.size()); + auto next_key_offset = static_cast(key_bytes_.size()); key_bytes_.resize(next_key_offset + key_length); offsets_.push_back(next_key_offset + key_length); memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); @@ -1078,7 +1078,7 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg } if (ctx->HasError()) return nullptr; - out_fields[i] = field(aggregates[i].name, impl->aggregators[i]->out_type()); + out_fields[i] = field("", impl->aggregators[i]->out_type()); } size_t n_keys = args.inputs.size() - aggregates.size(); @@ -1103,7 +1103,7 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg ctx->SetStatus(Status::NotImplemented("Key of type", key_type->ToString())); return nullptr; } - out_fields[aggregates.size() + i] = field(impl->options.key_names[i], key_type); + out_fields[aggregates.size() + i] = field("", key_type); } impl->add_length_impl.resize(n_keys); diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index bf8a735b8e5..1cfd24f21c0 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -319,8 +319,9 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key GroupByOptions options) { ArrayVector keys_arrays; for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); + std::vector key_names(keys_arrays.size(), ""); ARROW_ASSIGN_OR_RAISE(auto keys_struct, - StructArray::Make(std::move(keys_arrays), options.key_names)); + StructArray::Make(std::move(keys_arrays), std::move(key_names))); ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); @@ -329,13 +330,11 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key int64_t n_groups = groupings->length(); - std::vector out_names; ArrayVector out_columns; for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { const Datum& aggregand = aggregands[i_agg]; const std::string& function = options.aggregates[i_agg].function; - out_names.push_back(options.aggregates[i_agg].name); ScalarVector aggregated_scalars; @@ -362,10 +361,10 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key keys_struct = checked_pointer_cast(groupings_and_values->GetFieldByName("values")); for (size_t i_key = 0; i_key < aggregands.size(); ++i_key) { - out_names.push_back(options.key_names[i_key]); out_columns.push_back(keys_struct->field(i_key)); } + std::vector out_names(out_columns.size(), ""); return StructArray::Make(std::move(out_columns), std::move(out_names)); } @@ -379,15 +378,22 @@ void ValidateGroupBy(GroupByOptions options, std::vector aggregands, // Ordering of groups is not important, so sort by key columns to ensure the comparison // doesn't fail spuriously - SortOptions sort_options; - for (const std::string& key_name : options.key_names) { - sort_options.sort_keys.emplace_back(key_name); - } - for (Datum* out : {&expected, &actual}) { - ASSERT_OK_AND_ASSIGN(Datum batch, - RecordBatch::FromStructArray(out->array_as())); - ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(batch, sort_options)); + auto out_columns = out->array_as()->fields(); + + SortOptions sort_options; + FieldVector key_fields; + ArrayVector key_columns; + for (size_t i = 0; i < keys.size(); ++i) { + auto name = std::to_string(i); + sort_options.sort_keys.emplace_back(name); + key_fields.push_back(field(name, out_columns[0]->type())); + key_columns.push_back(out_columns[0]); + } + auto key_batch = RecordBatch::Make(schema(std::move(key_fields)), out->length(), + std::move(key_columns)); + + ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(key_batch, sort_options)); ASSERT_OK_AND_ASSIGN(*out, Take(*out, sort_indices, TakeOptions::NoBoundsCheck())); } @@ -401,15 +407,14 @@ TEST(GroupBy, SumOnly) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - GroupBy({aggregand}, {key}, - GroupByOptions(/*aggregates=*/{{"sum", nullptr, "f64 summed"}}, - /*key_names=*/{"i64 key"}))); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"sum", nullptr}, + })); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("f64 summed", float64()), - field("i64 key", int64()), + field("", float64()), + field("", int64()), }), R"([ [4.25, 1], @@ -432,21 +437,19 @@ TEST(GroupBy, CountAndSum) { Datum aggregated_and_grouped, // NB: passing an aggregand twice or also using it as a key is legal GroupBy({aggregand, aggregand, key}, {key}, - GroupByOptions(/*aggregates=*/ - { - {"count", &count_options, "f32 counted"}, - {"sum", nullptr, "f32 summed"}, - {"sum", nullptr, "i64 summed"}, - }, - /*key_names=*/{"i64 key"}))); + GroupByOptions{ + {"count", &count_options}, + {"sum", nullptr}, + {"sum", nullptr}, + })); AssertDatumsEqual( ArrayFromJSON(struct_({ - field("f32 counted", int64()), + field("", int64()), // NB: summing a float32 array results in float64 sums - field("f32 summed", float64()), - field("i64 summed", int64()), - field("i64 key", int64()), + field("", float64()), + field("", int64()), + field("", int64()), }), R"([ [1, 1.0, 2, 1], @@ -462,15 +465,12 @@ TEST(GroupBy, StringKey) { auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - GroupBy({aggregand}, {key}, - GroupByOptions(/*aggregates=*/{{"sum", nullptr, "i64 summed"}}, - /*key_names=*/{"str key"}))); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupBy({aggregand}, {key}, GroupByOptions{{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("i64 summed", int64()), - field("str key", utf8()), + field("", int64()), + field("", utf8()), }), R"([ [10, "alfa"], @@ -490,14 +490,12 @@ TEST(GroupBy, MultipleKeys) { ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - GroupBy({aggregand}, {int_key, str_key}, - GroupByOptions(/*aggregates=*/{{"sum", nullptr, "f32 summed"}}, - /*key_names=*/{"int key", "str key"}))); + GroupBy({aggregand}, {int_key, str_key}, GroupByOptions{{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("f32 summed", float64()), - field("int key", int32()), - field("str key", utf8()), + field("", float64()), + field("", int32()), + field("", utf8()), }), R"([ [0.125, 0, "beta"], @@ -514,27 +512,23 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - group_helpers::ValidateGroupBy( - GroupByOptions(/*aggregates=*/{{"sum", nullptr, "i64 summed"}}, - /*key_names=*/{"str key"}), - {aggregand}, {key}); + group_helpers::ValidateGroupBy(GroupByOptions{{"sum", nullptr}}, {aggregand}, {key}); } TEST(GroupBy, RandomArraySum) { auto rand = random::RandomArrayGenerator(0xdeadbeef); - GroupByOptions options; - options.aggregates = { - GroupByOptions::Aggregate{"sum", nullptr, "f32 summed"}, - }; - options.key_names = {"i64 key"}; for (size_t i = 3; i < 14; i += 2) { for (auto null_probability : {0.0, 0.001, 0.1, 0.5, 0.999, 1.0}) { int64_t length = 1UL << i; auto summand = rand.Float32(length, -100, 100, null_probability); auto key = rand.Int64(length, 0, 12); - group_helpers::ValidateGroupBy(options, {summand}, {key}); + group_helpers::ValidateGroupBy( + GroupByOptions{ + {"sum", nullptr}, + }, + {summand}, {key}); } } } From 8ec135ea942d33724467691140b0ffc7f07f25f6 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 2 Mar 2021 11:57:17 -0500 Subject: [PATCH 15/49] reference ARROW-11840, add FunctionDoc --- cpp/src/arrow/compute/exec.cc | 20 ++++++++++--------- .../arrow/compute/kernels/aggregate_basic.cc | 10 +++++++--- .../arrow/compute/kernels/aggregate_test.cc | 1 - docs/source/cpp/compute.rst | 1 + 4 files changed, 19 insertions(+), 13 deletions(-) diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index 60e77fd40d7..d014828c1a6 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -838,6 +838,13 @@ class ScalarAggExecutor : public KernelExecutorImpl { private: Status Consume(const ExecBatch& batch) { + if (kernel_->nomerge) { + kernel_->consume(kernel_ctx_, batch); + ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + return Status::OK(); + } + + // FIXME(ARROW-11840) don't merge *any* aggegates for every batch auto batch_state = kernel_->init(kernel_ctx_, {kernel_, *input_descrs_, options_}); ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); @@ -850,16 +857,11 @@ class ScalarAggExecutor : public KernelExecutorImpl { KernelContext batch_ctx(exec_context()); batch_ctx.SetState(batch_state.get()); - if (kernel_->nomerge) { - kernel_->consume(kernel_ctx_, batch); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); - } else { - kernel_->consume(&batch_ctx, batch); - ARROW_CTX_RETURN_IF_ERROR(&batch_ctx); + kernel_->consume(&batch_ctx, batch); + ARROW_CTX_RETURN_IF_ERROR(&batch_ctx); - kernel_->merge(kernel_ctx_, std::move(*batch_state), state()); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); - } + kernel_->merge(kernel_ctx_, std::move(*batch_state), state()); + ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); return Status::OK(); } diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 5cc1404dedb..afe067ae1d3 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -979,7 +979,7 @@ struct GroupByImpl : public ScalarAggregator { } void MergeFrom(KernelContext* ctx, KernelState&& src) override { - // TODO(michalursa) merge two hash tables + // TODO(ARROW-11840) merge two hash tables ctx->SetStatus(Status::NotImplemented("merging grouped aggregations")); } @@ -1178,8 +1178,12 @@ const FunctionDoc all_doc{ ("Null values are ignored."), {"array"}}; -// TODO(michalursa) add FunctionDoc for group_by -const FunctionDoc group_by_doc{"", (""), {}}; +const FunctionDoc group_by_doc{ + ("Compute aggregations on input arrays, grouped by key columns."), + ("Leading arguments are passed to the corresponding aggregation function\n" + "named in GroupByOptions, remaining inputs are used as keys for grouping."), + {"*args"}, + "GroupByOptions"}; } // namespace diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 1cfd24f21c0..46dad6965a8 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -431,7 +431,6 @@ TEST(GroupBy, CountAndSum) { auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); CountOptions count_options; - count_options.count_mode = CountOptions::COUNT_NON_NULL; ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst index 5224a91492e..2c1543075f5 100644 --- a/docs/source/cpp/compute.rst +++ b/docs/source/cpp/compute.rst @@ -271,6 +271,7 @@ for a group:: std::shared_ptr group_key_0 = out_array->field(2); std::shared_ptr group_key_1 = out_array->field(3); +The output fields' names are empty. Element-wise ("scalar") functions --------------------------------- From 3e95d6e579751a3279e7adffd44b0d8e09a57e05 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 2 Mar 2021 16:23:30 -0500 Subject: [PATCH 16/49] add support for grouped min_max --- .../arrow/compute/kernels/aggregate_basic.cc | 237 +++++++++++++++--- .../arrow/compute/kernels/aggregate_test.cc | 30 ++- docs/source/cpp/compute.rst | 3 +- 3 files changed, 226 insertions(+), 44 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index afe067ae1d3..03208bebe3c 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -98,7 +98,7 @@ struct CountImpl : public ScalarAggregator { struct GroupedAggregator { virtual ~GroupedAggregator() = default; - virtual void Consume(KernelContext*, const ExecBatch& batch, + virtual void Consume(KernelContext*, const Datum& aggregand, const uint32_t* group_ids) = 0; virtual void Finalize(KernelContext* ctx, Datum* out) = 0; @@ -125,7 +125,7 @@ struct GroupedAggregator { struct GroupedCountImpl : public GroupedAggregator { static std::unique_ptr Make(KernelContext* ctx, - const DataType& input_type, + const std::shared_ptr&, const FunctionOptions* options) { auto out = ::arrow::internal::make_unique(); out->options_ = checked_cast(*options); @@ -144,25 +144,32 @@ struct GroupedCountImpl : public GroupedAggregator { } } - void Consume(KernelContext* ctx, const ExecBatch& batch, + void Consume(KernelContext* ctx, const Datum& aggregand, const uint32_t* group_ids) override { - MaybeResize(ctx, batch.length, group_ids); + MaybeResize(ctx, aggregand.length(), group_ids); if (ctx->HasError()) return; auto raw_counts = reinterpret_cast(counts_->mutable_data()); - auto input = batch[0].make_array(); - if (options_.count_mode == CountOptions::COUNT_NON_NULL) { - for (int64_t i = 0; i < input->length(); ++i) { - if (input->IsNull(i)) continue; - raw_counts[group_ids[i]]++; - } - } else { - for (int64_t i = 0; i < batch.length; ++i) { - if (input->IsValid(i)) continue; - raw_counts[group_ids[i]]++; + const auto& input = aggregand.array(); + + if (options_.count_mode == CountOptions::COUNT_NULL) { + for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { + auto g = group_ids[i]; + raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); } + return; } + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_counts[g] += 1; + } + }); } void Finalize(KernelContext* ctx, Datum* out) override { @@ -201,9 +208,11 @@ struct GroupedSumImpl : public GroupedAggregator { arrow::internal::VisitSetBitRunsVoid( input->buffers[0], input->offset, input->length, [&](int64_t begin, int64_t length) { - for (int64_t i = begin, end = begin + length; i < end; ++i) { - raw_sums[group_ids[i]] += raw_input[i]; - raw_counts[group_ids[i]] += 1; + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_sums[g] += raw_input[input_i]; + raw_counts[g] += 1; } }); }; @@ -221,9 +230,11 @@ struct GroupedSumImpl : public GroupedAggregator { arrow::internal::VisitSetBitRunsVoid( input->buffers[0], input->offset, input->length, [&](int64_t begin, int64_t length) { - for (int64_t i = begin, end = begin + length; i < end; ++i) { - raw_sums[group_ids[i]] += BitUtil::GetBit(raw_input, i); - raw_counts[group_ids[i]] += 1; + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i) { + auto g = group_ids[i]; + raw_sums[g] += BitUtil::GetBit(raw_input, input_i); + raw_counts[g] += 1; } }); }; @@ -244,7 +255,7 @@ struct GroupedSumImpl : public GroupedAggregator { }; static std::unique_ptr Make(KernelContext* ctx, - const DataType& input_type, + const std::shared_ptr& input_type, const FunctionOptions* options) { auto out = ::arrow::internal::make_unique(); @@ -255,7 +266,7 @@ struct GroupedSumImpl : public GroupedAggregator { if (ctx->HasError()) return nullptr; GetConsumeImpl get_consume_impl; - ctx->SetStatus(VisitTypeInline(input_type, &get_consume_impl)); + ctx->SetStatus(VisitTypeInline(*input_type, &get_consume_impl)); out->consume_impl_ = std::move(get_consume_impl.consume_impl); out->out_type_ = std::move(get_consume_impl.out_type); @@ -271,11 +282,11 @@ struct GroupedSumImpl : public GroupedAggregator { std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); } - void Consume(KernelContext* ctx, const ExecBatch& batch, + void Consume(KernelContext* ctx, const Datum& aggregand, const uint32_t* group_ids) override { - MaybeResize(ctx, batch.length, group_ids); + MaybeResize(ctx, aggregand.length(), group_ids); if (ctx->HasError()) return; - consume_impl_(batch[0].array(), group_ids, sums_.get(), counts_.get()); + consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); } void Finalize(KernelContext* ctx, Datum* out) override { @@ -307,6 +318,154 @@ struct GroupedSumImpl : public GroupedAggregator { ConsumeImpl consume_impl_; }; +struct GroupedMinMaxImpl : public GroupedAggregator { + using ConsumeImpl = std::function&, + const uint32_t*, BufferVector*)>; + + using ResizeImpl = std::function; + + struct GetImpl { + template ::CType> + enable_if_number Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, BufferVector* buffers) { + auto raw_inputs = reinterpret_cast(input->buffers[1]->data()); + + auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); + auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); + + auto raw_has_nulls = buffers->at(2)->mutable_data(); + auto raw_has_values = buffers->at(3)->mutable_data(); + + for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { + auto g = group_ids[i]; + bool is_valid = BitUtil::GetBit(input->buffers[0]->data(), input_i); + if (is_valid) { + raw_maxes[g] = std::max(raw_maxes[g], raw_inputs[input_i]); + raw_mins[g] = std::min(raw_mins[g], raw_inputs[input_i]); + BitUtil::SetBit(raw_has_values, g); + } else { + BitUtil::SetBit(raw_has_nulls, g); + } + } + }; + + for (auto pair : + {std::make_pair(&resize_min_impl, std::numeric_limits::max()), + std::make_pair(&resize_max_impl, std::numeric_limits::min())}) { + *pair.first = [pair](Buffer* vals, int64_t new_num_groups) { + int64_t old_num_groups = vals->size() / sizeof(CType); + + int64_t new_size = new_num_groups * sizeof(CType); + RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); + + auto raw_vals = reinterpret_cast(vals->mutable_data()); + for (int64_t i = old_num_groups; i != new_num_groups; ++i) { + raw_vals[i] = pair.second; + } + return Status::OK(); + }; + } + + return Status::OK(); + } + + Status Visit(const BooleanType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + ConsumeImpl consume_impl; + ResizeImpl resize_min_impl, resize_max_impl; + }; + + static std::unique_ptr Make( + KernelContext* ctx, const std::shared_ptr& input_type, + const FunctionOptions* options) { + auto out = ::arrow::internal::make_unique(); + out->options_ = *checked_cast(options); + out->type_ = input_type; + + out->buffers_.resize(4); + for (auto& buf : out->buffers_) { + ctx->SetStatus(ctx->Allocate(0).Value(&buf)); + if (ctx->HasError()) return nullptr; + } + + GetImpl get_impl; + ctx->SetStatus(VisitTypeInline(*input_type, &get_impl)); + + out->consume_impl_ = std::move(get_impl.consume_impl); + out->resize_min_impl_ = std::move(get_impl.resize_min_impl); + out->resize_max_impl_ = std::move(get_impl.resize_max_impl); + return out; + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_num_groups = num_groups_; + num_groups_ = new_num_groups; + + KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); + KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); + + for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { + KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( + BitUtil::BytesForBits(new_num_groups))); + BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); + } + } + + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + consume_impl_(aggregand.array(), group_ids, &buffers_); + } + + void Finalize(KernelContext* ctx, Datum* out) override { + // aggregation for group is valid if there was at least one value in that group + std::shared_ptr null_bitmap = std::move(buffers_[3]); + + if (options_.null_handling == MinMaxOptions::EMIT_NULL) { + // ... and there were no nulls in that group + for (int64_t i = 0; i < num_groups(); ++i) { + if (BitUtil::GetBit(buffers_[2]->data(), i)) { + BitUtil::ClearBit(null_bitmap->mutable_data(), i); + } + } + } + + auto mins = + ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); + + auto maxes = ArrayData::Make(type_, num_groups(), + {std::move(null_bitmap), std::move(buffers_[1])}); + + *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, + {std::move(mins), std::move(maxes)}); + } + + int64_t num_groups() const override { return num_groups_; } + + std::shared_ptr out_type() const override { + return struct_({field("min", type_), field("max", type_)}); + } + + int64_t num_groups_; + BufferVector buffers_; + std::shared_ptr type_; + ConsumeImpl consume_impl_; + ResizeImpl resize_min_impl_, resize_max_impl_; + MinMaxOptions options_; +}; + std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique( static_cast(*args.options)); @@ -972,8 +1131,7 @@ struct GroupByImpl : public ScalarAggregator { } for (size_t i = 0; i < aggregators.size(); ++i) { - ExecBatch aggregand_batch{{aggregands[i]}, batch.length}; - aggregators[i]->Consume(ctx, aggregand_batch, group_ids_batch_.data()); + aggregators[i]->Consume(ctx, aggregands[i], group_ids_batch_.data()); if (ctx->HasError()) return; } } @@ -1032,7 +1190,7 @@ struct GroupByImpl : public ScalarAggregator { template std::unique_ptr MakeAggregator(KernelContext* ctx, const std::string& function_name, - const DataType& input_type, + const std::shared_ptr& input_type, const FunctionOptions* options) { if (options == nullptr) { if (auto function = ctx->exec_context() @@ -1069,10 +1227,13 @@ std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArg if (function == "count") { impl->aggregators[i] = - MakeAggregator(ctx, function, *input_type, options); + MakeAggregator(ctx, function, input_type, options); } else if (function == "sum") { impl->aggregators[i] = - MakeAggregator(ctx, function, *input_type, options); + MakeAggregator(ctx, function, input_type, options); + } else if (function == "min_max") { + impl->aggregators[i] = + MakeAggregator(ctx, function, input_type, options); } else { ctx->SetStatus(Status::NotImplemented("Grouped aggregate ", function)); } @@ -1168,18 +1329,16 @@ const FunctionDoc min_max_doc{"Compute the minimum and maximum values of a numer {"array"}, "MinMaxOptions"}; -const FunctionDoc any_doc{ - "Test whether any element in a boolean array evaluates to true.", - ("Null values are ignored."), - {"array"}}; +const FunctionDoc any_doc{"Test whether any element in a boolean array evaluates to true", + ("Null values are ignored."), + {"array"}}; -const FunctionDoc all_doc{ - "Test whether all elements in a boolean array evaluate to true.", - ("Null values are ignored."), - {"array"}}; +const FunctionDoc all_doc{"Test whether all elements in a boolean array evaluate to true", + ("Null values are ignored."), + {"array"}}; const FunctionDoc group_by_doc{ - ("Compute aggregations on input arrays, grouped by key columns."), + ("Compute aggregations on input arrays, grouped by key columns"), ("Leading arguments are passed to the corresponding aggregation function\n" "named in GroupByOptions, remaining inputs are used as keys for grouping."), {"*args"}, diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 46dad6965a8..b8ba23909e1 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -426,6 +426,32 @@ TEST(GroupBy, SumOnly) { /*verbose=*/true); } +TEST(GroupBy, MinMaxOnly) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"min_max", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("", int64()), + }), + R"([ + [{"min": 1.0, "max": 3.25}, 1], + [{"min": -0.25, "max": 0.125}, 2], + [{"min": null, "max": null}, 3], + [{"min": 0.75, "max": 0.75}, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + TEST(GroupBy, CountAndSum) { auto aggregand = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); @@ -974,9 +1000,7 @@ class TestPrimitiveMinMaxKernel : public ::testing::Test { void AssertMinMaxIsNull(const Datum& array, const MinMaxOptions& options) { ASSERT_OK_AND_ASSIGN(Datum out, MinMax(array, options)); - - const StructScalar& value = out.scalar_as(); - for (const auto& val : value.value) { + for (const auto& val : out.scalar_as().value) { ASSERT_FALSE(val->is_valid); } } diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst index 2c1543075f5..3878aa928f3 100644 --- a/docs/source/cpp/compute.rst +++ b/docs/source/cpp/compute.rst @@ -762,8 +762,7 @@ Associative transforms * \(2) Duplicates are removed from the output while the original order is maintained. -* \(3) Output is a ``{ - "values" : input type, "counts" : Int64}`` Struct. +* \(3) Output is a ``{"values": input type, "counts": Int64}`` Struct. Each output element corresponds to a unique value in the input, along with the number of times this value has appeared. From 4694a11c5847ae22a04ca07cf9fae21bcfac4d12 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 2 Mar 2021 16:37:08 -0500 Subject: [PATCH 17/49] remove speculative empty case for now --- .../arrow/compute/kernels/aggregate_test.cc | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index b8ba23909e1..e7872d171f8 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -338,18 +338,12 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key ScalarVector aggregated_scalars; - if (n_groups > 0) { - ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, - ApplyGroupings(*groupings, *aggregand.make_array())); - - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE( - Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); - aggregated_scalars.push_back(d.scalar()); - } - } else { - DCHECK_EQ(aggregand.length(), 0); - ARROW_ASSIGN_OR_RAISE(Datum d, CallFunction(function, {aggregand})); + ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, + ApplyGroupings(*groupings, *aggregand.make_array())); + + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ARROW_ASSIGN_OR_RAISE( + Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); aggregated_scalars.push_back(d.scalar()); } From 6a105c3ee7c25d3f83ba1e3fb8015c4d47aa3af7 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 11 Mar 2021 11:48:05 -0500 Subject: [PATCH 18/49] first pass at review comments --- cpp/src/arrow/compute/exec_internal.h | 5 ++ .../arrow/compute/kernels/aggregate_basic.cc | 82 +++++++++++-------- .../compute/kernels/aggregate_benchmark.cc | 5 +- 3 files changed, 53 insertions(+), 39 deletions(-) diff --git a/cpp/src/arrow/compute/exec_internal.h b/cpp/src/arrow/compute/exec_internal.h index a74e5c8d8fa..55daa243cd3 100644 --- a/cpp/src/arrow/compute/exec_internal.h +++ b/cpp/src/arrow/compute/exec_internal.h @@ -106,6 +106,11 @@ class ARROW_EXPORT KernelExecutor { public: virtual ~KernelExecutor() = default; + /// The Kernel's `init` method must be called and any KernelState set in the + /// KernelContext *before* KernelExecutor::Init is called. This is to facilitate + /// the case where init may be expensive and does not need to be called again for + /// each execution of the kernel, for example the same lookup table can be re-used + /// for all scanned batches in a dataset filter. virtual Status Init(KernelContext*, KernelInitArgs) = 0; /// XXX: Better configurability for listener diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 03208bebe3c..edec65c923d 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -22,6 +22,7 @@ #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/common.h" #include "arrow/util/bit_run_reader.h" +#include "arrow/util/bitmap_ops.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" @@ -238,7 +239,7 @@ struct GroupedSumImpl : public GroupedAggregator { } }); }; - out_type = boolean(); + out_type = uint64(); return Status::OK(); } @@ -313,6 +314,8 @@ struct GroupedSumImpl : public GroupedAggregator { std::shared_ptr out_type() const override { return out_type_; } + // NB: counts are used here instead of a simple "has_values_" bitmap since + // we expect to reuse this kernel to handle Mean std::shared_ptr sums_, counts_; std::shared_ptr out_type_; ConsumeImpl consume_impl_; @@ -329,44 +332,24 @@ struct GroupedMinMaxImpl : public GroupedAggregator { enable_if_number Visit(const T&) { consume_impl = [](const std::shared_ptr& input, const uint32_t* group_ids, BufferVector* buffers) { - auto raw_inputs = reinterpret_cast(input->buffers[1]->data()); - auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); auto raw_has_nulls = buffers->at(2)->mutable_data(); auto raw_has_values = buffers->at(3)->mutable_data(); - for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { - auto g = group_ids[i]; - bool is_valid = BitUtil::GetBit(input->buffers[0]->data(), input_i); - if (is_valid) { - raw_maxes[g] = std::max(raw_maxes[g], raw_inputs[input_i]); - raw_mins[g] = std::min(raw_mins[g], raw_inputs[input_i]); - BitUtil::SetBit(raw_has_values, g); - } else { - BitUtil::SetBit(raw_has_nulls, g); - } - } + auto g = group_ids; + VisitArrayDataInline( + *input, + [&](CType val) { + raw_maxes[*g] = std::max(raw_maxes[*g], val); + raw_mins[*g] = std::min(raw_mins[*g], val); + BitUtil::SetBit(raw_has_values, *g++); + }, + [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); }; - for (auto pair : - {std::make_pair(&resize_min_impl, std::numeric_limits::max()), - std::make_pair(&resize_max_impl, std::numeric_limits::min())}) { - *pair.first = [pair](Buffer* vals, int64_t new_num_groups) { - int64_t old_num_groups = vals->size() / sizeof(CType); - - int64_t new_size = new_num_groups * sizeof(CType); - RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); - - auto raw_vals = reinterpret_cast(vals->mutable_data()); - for (int64_t i = old_num_groups; i != new_num_groups; ++i) { - raw_vals[i] = pair.second; - } - return Status::OK(); - }; - } - + GetResizeImpls(); return Status::OK(); } @@ -382,6 +365,36 @@ struct GroupedMinMaxImpl : public GroupedAggregator { return Status::NotImplemented("Grouped MinMax data of type ", type); } + template + ResizeImpl MakeResizeImpl(CType anti_extreme) { + // resize a min or max buffer, storing the correct anti extreme + return [anti_extreme](Buffer* vals, int64_t new_num_groups) { + int64_t old_num_groups = vals->size() / sizeof(CType); + + int64_t new_size = new_num_groups * sizeof(CType); + RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); + + auto raw_vals = reinterpret_cast(vals->mutable_data()); + for (int64_t i = old_num_groups; i != new_num_groups; ++i) { + raw_vals[i] = anti_extreme; + } + return Status::OK(); + }; + } + + template ::CType> + enable_if_floating_point GetResizeImpls() { + auto inf = std::numeric_limits::infinity(); + resize_min_impl = MakeResizeImpl(inf); + resize_max_impl = MakeResizeImpl(-inf); + } + + template ::CType> + enable_if_integer GetResizeImpls() { + resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); + resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); + } + ConsumeImpl consume_impl; ResizeImpl resize_min_impl, resize_max_impl; }; @@ -435,11 +448,8 @@ struct GroupedMinMaxImpl : public GroupedAggregator { if (options_.null_handling == MinMaxOptions::EMIT_NULL) { // ... and there were no nulls in that group - for (int64_t i = 0; i < num_groups(); ++i) { - if (BitUtil::GetBit(buffers_[2]->data(), i)) { - BitUtil::ClearBit(null_bitmap->mutable_data(), i); - } - } + arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, + num_groups(), 0, null_bitmap->mutable_data()); } auto mins = diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index a55673eba9a..76db483be96 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -318,7 +318,6 @@ static void BenchmarkGroupBy(benchmark::State& state, GroupByOptions options; options.aggregates = aggregates; - options.key_names.resize(keys.size(), "ignored"); for (auto _ : state) { ABORT_NOT_OK(CallFunction("group_by", arguments, &options).status()); @@ -347,7 +346,7 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { /*min_length=*/3, /*max_length=*/32); - BenchmarkGroupBy(state, {{"sum", NULLPTR, "summed f64"}}, {summand}, {key}); + BenchmarkGroupBy(state, {{"sum", NULLPTR}}, {summand}, {key}); }); GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { @@ -361,7 +360,7 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { /*min=*/0, /*max=*/15); - BenchmarkGroupBy(state, {{"sum", NULLPTR, "summed f64"}}, {summand}, {key}); + BenchmarkGroupBy(state, {{"sum", NULLPTR}}, {summand}, {key}); }); // From 9ece6146f27195f73e52bddaba1b230bd57c71e6 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 11 Mar 2021 14:22:04 -0500 Subject: [PATCH 19/49] remove group_by from function registry --- cpp/src/arrow/CMakeLists.txt | 1 + cpp/src/arrow/compute/api_aggregate.cc | 10 - cpp/src/arrow/compute/api_aggregate.h | 49 - cpp/src/arrow/compute/exec.cc | 6 - cpp/src/arrow/compute/kernel.h | 11 +- cpp/src/arrow/compute/kernels/CMakeLists.txt | 6 +- .../arrow/compute/kernels/aggregate_basic.cc | 1116 +---------------- .../compute/kernels/aggregate_internal.h | 2 +- .../arrow/compute/kernels/aggregate_test.cc | 501 +------- .../arrow/compute/kernels/hash_aggregate.cc | 1115 ++++++++++++++++ .../compute/kernels/hash_aggregate_internal.h | 89 ++ .../compute/kernels/hash_aggregate_test.cc | 558 +++++++++ docs/source/cpp/compute.rst | 53 +- 13 files changed, 1780 insertions(+), 1737 deletions(-) create mode 100644 cpp/src/arrow/compute/kernels/hash_aggregate.cc create mode 100644 cpp/src/arrow/compute/kernels/hash_aggregate_internal.h create mode 100644 cpp/src/arrow/compute/kernels/hash_aggregate_test.cc diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 04756aaf8e9..df72dcc5b6b 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -373,6 +373,7 @@ if(ARROW_COMPUTE) compute/kernels/aggregate_tdigest.cc compute/kernels/aggregate_var_std.cc compute/kernels/codegen_internal.cc + compute/kernels/hash_aggregate.cc compute/kernels/scalar_arithmetic.cc compute/kernels/scalar_boolean.cc compute/kernels/scalar_cast_boolean.cc diff --git a/cpp/src/arrow/compute/api_aggregate.cc b/cpp/src/arrow/compute/api_aggregate.cc index 98768cf3bfc..5afa1048960 100644 --- a/cpp/src/arrow/compute/api_aggregate.cc +++ b/cpp/src/arrow/compute/api_aggregate.cc @@ -73,15 +73,5 @@ Result TDigest(const Datum& value, const TDigestOptions& options, return CallFunction("tdigest", {value}, &options, ctx); } -Result GroupBy(const std::vector& aggregands, - const std::vector& keys, const GroupByOptions& options, - ExecContext* ctx) { - std::vector args = aggregands; - for (const Datum& key : keys) { - args.push_back(key); - } - return CallFunction("group_by", args, &options, ctx); -} - } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index e92a7adf768..eef1587bb73 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -149,26 +149,6 @@ struct ARROW_EXPORT TDigestOptions : public FunctionOptions { uint32_t buffer_size; }; -/// \brief Configure a grouped aggregation -struct ARROW_EXPORT GroupByOptions : public FunctionOptions { - struct Aggregate { - /// the name of the aggregation function - std::string function; - - /// options for the aggregation function - const FunctionOptions* options; - }; - - GroupByOptions() = default; - - GroupByOptions(std::initializer_list aggregates) : aggregates(aggregates) {} - - explicit GroupByOptions(std::vector aggregates) - : aggregates(std::move(aggregates)) {} - - std::vector aggregates; -}; - /// @} /// \brief Count non-null (or null) values in an array. @@ -326,34 +306,5 @@ Result TDigest(const Datum& value, const TDigestOptions& options = TDigestOptions::Defaults(), ExecContext* ctx = NULLPTR); -/// \brief Calculate multiple aggregations grouped on multiple keys -/// -/// \param[in] aggregands datums to which aggregations will be applied -/// \param[in] keys datums which will be used to group the aggregations -/// \param[in] options GroupByOptions, encapsulating the names and options of aggregate -/// functions to be applied and the field names for results in the output. -/// \return a StructArray with len(aggregands) + len(keys) fields. The first -/// len(aggregands) fields are the results of the aggregations for the group -/// specified by keys in the final len(keys) fields. -/// -/// For example: -/// GroupByOptions options = { -/// .aggregates = { -/// {"sum", nullptr, "sum result"}, -/// {"mean", nullptr, "mean result"}, -/// }, -/// .key_names = {"str key", "date key"}, -/// }; -/// assert(*GroupBy({[2, 5, 8], [1.5, 2.0, 3.0]}, -/// {["a", "b", "a"], [today, today, today]}, -/// options).Equals([ -/// {"sum result": 10, "mean result": 2.25, "str key": "a", "date key": today}, -/// {"sum result": 5, "mean result": 2.0, "str key": "b", "date key": today}, -/// ])) -ARROW_EXPORT -Result GroupBy(const std::vector& aggregands, - const std::vector& keys, const GroupByOptions& options, - ExecContext* ctx = NULLPTR); - } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index d014828c1a6..f76a311cf90 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -838,12 +838,6 @@ class ScalarAggExecutor : public KernelExecutorImpl { private: Status Consume(const ExecBatch& batch) { - if (kernel_->nomerge) { - kernel_->consume(kernel_ctx_, batch); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); - return Status::OK(); - } - // FIXME(ARROW-11840) don't merge *any* aggegates for every batch auto batch_state = kernel_->init(kernel_ctx_, {kernel_, *input_descrs_, options_}); ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index 40fe996c52b..c8f9cacfb34 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -684,24 +684,21 @@ struct ScalarAggregateKernel : public Kernel { ScalarAggregateKernel(std::shared_ptr sig, KernelInit init, ScalarAggregateConsume consume, ScalarAggregateMerge merge, - ScalarAggregateFinalize finalize, bool nomerge = false) + ScalarAggregateFinalize finalize) : Kernel(std::move(sig), init), consume(std::move(consume)), merge(std::move(merge)), - finalize(std::move(finalize)), - nomerge(nomerge) {} + finalize(std::move(finalize)) {} ScalarAggregateKernel(std::vector in_types, OutputType out_type, KernelInit init, ScalarAggregateConsume consume, - ScalarAggregateMerge merge, ScalarAggregateFinalize finalize, - bool nomerge = false) + ScalarAggregateMerge merge, ScalarAggregateFinalize finalize) : ScalarAggregateKernel(KernelSignature::Make(std::move(in_types), out_type), init, - consume, merge, finalize, nomerge) {} + consume, merge, finalize) {} ScalarAggregateConsume consume; ScalarAggregateMerge merge; ScalarAggregateFinalize finalize; - bool nomerge; }; } // namespace compute diff --git a/cpp/src/arrow/compute/kernels/CMakeLists.txt b/cpp/src/arrow/compute/kernels/CMakeLists.txt index 577b250da87..5e223a1f906 100644 --- a/cpp/src/arrow/compute/kernels/CMakeLists.txt +++ b/cpp/src/arrow/compute/kernels/CMakeLists.txt @@ -59,5 +59,9 @@ add_arrow_benchmark(vector_selection_benchmark PREFIX "arrow-compute") # Aggregates -add_arrow_compute_test(aggregate_test SOURCES aggregate_test.cc test_util.cc) +add_arrow_compute_test(aggregate_test + SOURCES + aggregate_test.cc + hash_aggregate_test.cc + test_util.cc) add_arrow_benchmark(aggregate_benchmark PREFIX "arrow-compute") diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index edec65c923d..61dc8cb403c 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -15,14 +15,10 @@ // specific language governing permissions and limitations // under the License. -#include - #include "arrow/compute/api_aggregate.h" #include "arrow/compute/kernels/aggregate_basic_internal.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/common.h" -#include "arrow/util/bit_run_reader.h" -#include "arrow/util/bitmap_ops.h" #include "arrow/util/cpu_info.h" #include "arrow/util/make_unique.h" @@ -46,10 +42,9 @@ void AggregateFinalize(KernelContext* ctx, Datum* out) { } // namespace void AddAggKernel(std::shared_ptr sig, KernelInit init, - ScalarAggregateFunction* func, SimdLevel::type simd_level, - bool nomerge) { + ScalarAggregateFunction* func, SimdLevel::type simd_level) { ScalarAggregateKernel kernel(std::move(sig), init, AggregateConsume, AggregateMerge, - AggregateFinalize, nomerge); + AggregateFinalize); // Set the simd level kernel.simd_level = simd_level; DCHECK_OK(func->AddKernel(kernel)); @@ -96,386 +91,6 @@ struct CountImpl : public ScalarAggregator { int64_t nulls = 0; }; -struct GroupedAggregator { - virtual ~GroupedAggregator() = default; - - virtual void Consume(KernelContext*, const Datum& aggregand, - const uint32_t* group_ids) = 0; - - virtual void Finalize(KernelContext* ctx, Datum* out) = 0; - - virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; - - virtual int64_t num_groups() const = 0; - - void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { - if (length == 0) return; - - // maybe a batch of group_ids should include the min/max group id - int64_t max_group = *std::max_element(group_ids, group_ids + length); - auto old_size = num_groups(); - - if (max_group >= old_size) { - auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); - Resize(ctx, new_size); - } - } - - virtual std::shared_ptr out_type() const = 0; -}; - -struct GroupedCountImpl : public GroupedAggregator { - static std::unique_ptr Make(KernelContext* ctx, - const std::shared_ptr&, - const FunctionOptions* options) { - auto out = ::arrow::internal::make_unique(); - out->options_ = checked_cast(*options); - ctx->SetStatus(ctx->Allocate(0).Value(&out->counts_)); - return out; - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups(); - KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); - auto new_size = num_groups(); - - auto raw_counts = reinterpret_cast(counts_->mutable_data()); - for (auto i = old_size; i < new_size; ++i) { - raw_counts[i] = 0; - } - } - - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - - auto raw_counts = reinterpret_cast(counts_->mutable_data()); - - const auto& input = aggregand.array(); - - if (options_.count_mode == CountOptions::COUNT_NULL) { - for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { - auto g = group_ids[i]; - raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); - } - return; - } - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_counts[g] += 1; - } - }); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - auto length = num_groups(); - *out = std::make_shared(length, std::move(counts_)); - } - - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - - std::shared_ptr out_type() const override { return int64(); } - - CountOptions options_; - std::shared_ptr counts_; -}; - -struct GroupedSumImpl : public GroupedAggregator { - // NB: whether we are accumulating into double, int64_t, or uint64_t - // we always have 64 bits per group in the sums buffer. - static constexpr size_t kSumSize = sizeof(int64_t); - - using ConsumeImpl = std::function&, - const uint32_t*, Buffer*, Buffer*)>; - - struct GetConsumeImpl { - template ::Type> - Status Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = reinterpret_cast::CType*>( - input->buffers[1]->data()); - auto raw_sums = reinterpret_cast::CType*>( - sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_sums[g] += raw_input[input_i]; - raw_counts[g] += 1; - } - }); - }; - out_type = TypeTraits::type_singleton(); - return Status::OK(); - } - - Status Visit(const BooleanType&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = input->buffers[1]->data(); - auto raw_sums = reinterpret_cast(sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i) { - auto g = group_ids[i]; - raw_sums[g] += BitUtil::GetBit(raw_input, input_i); - raw_counts[g] += 1; - } - }); - }; - out_type = uint64(); - return Status::OK(); - } - - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Summing data of type ", type); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("Summing data of type ", type); - } - - ConsumeImpl consume_impl; - std::shared_ptr out_type; - }; - - static std::unique_ptr Make(KernelContext* ctx, - const std::shared_ptr& input_type, - const FunctionOptions* options) { - auto out = ::arrow::internal::make_unique(); - - ctx->SetStatus(ctx->Allocate(0).Value(&out->sums_)); - if (ctx->HasError()) return nullptr; - - ctx->SetStatus(ctx->Allocate(0).Value(&out->counts_)); - if (ctx->HasError()) return nullptr; - - GetConsumeImpl get_consume_impl; - ctx->SetStatus(VisitTypeInline(*input_type, &get_consume_impl)); - - out->consume_impl_ = std::move(get_consume_impl.consume_impl); - out->out_type_ = std::move(get_consume_impl.out_type); - return out; - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups() * kSumSize; - KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); - KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); - auto new_size = num_groups() * kSumSize; - std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); - std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); - } - - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - std::shared_ptr null_bitmap; - int64_t null_count = 0; - - for (int64_t i = 0; i < num_groups(); ++i) { - if (reinterpret_cast(counts_->data())[i] > 0) continue; - - if (null_bitmap == nullptr) { - KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); - BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); - } - - null_count += 1; - BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); - } - - *out = ArrayData::Make(std::move(out_type_), num_groups(), - {std::move(null_bitmap), std::move(sums_)}, null_count); - } - - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - - std::shared_ptr out_type() const override { return out_type_; } - - // NB: counts are used here instead of a simple "has_values_" bitmap since - // we expect to reuse this kernel to handle Mean - std::shared_ptr sums_, counts_; - std::shared_ptr out_type_; - ConsumeImpl consume_impl_; -}; - -struct GroupedMinMaxImpl : public GroupedAggregator { - using ConsumeImpl = std::function&, - const uint32_t*, BufferVector*)>; - - using ResizeImpl = std::function; - - struct GetImpl { - template ::CType> - enable_if_number Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, BufferVector* buffers) { - auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); - auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); - - auto raw_has_nulls = buffers->at(2)->mutable_data(); - auto raw_has_values = buffers->at(3)->mutable_data(); - - auto g = group_ids; - VisitArrayDataInline( - *input, - [&](CType val) { - raw_maxes[*g] = std::max(raw_maxes[*g], val); - raw_mins[*g] = std::min(raw_mins[*g], val); - BitUtil::SetBit(raw_has_values, *g++); - }, - [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); - }; - - GetResizeImpls(); - return Status::OK(); - } - - Status Visit(const BooleanType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); - } - - template - ResizeImpl MakeResizeImpl(CType anti_extreme) { - // resize a min or max buffer, storing the correct anti extreme - return [anti_extreme](Buffer* vals, int64_t new_num_groups) { - int64_t old_num_groups = vals->size() / sizeof(CType); - - int64_t new_size = new_num_groups * sizeof(CType); - RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); - - auto raw_vals = reinterpret_cast(vals->mutable_data()); - for (int64_t i = old_num_groups; i != new_num_groups; ++i) { - raw_vals[i] = anti_extreme; - } - return Status::OK(); - }; - } - - template ::CType> - enable_if_floating_point GetResizeImpls() { - auto inf = std::numeric_limits::infinity(); - resize_min_impl = MakeResizeImpl(inf); - resize_max_impl = MakeResizeImpl(-inf); - } - - template ::CType> - enable_if_integer GetResizeImpls() { - resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); - resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); - } - - ConsumeImpl consume_impl; - ResizeImpl resize_min_impl, resize_max_impl; - }; - - static std::unique_ptr Make( - KernelContext* ctx, const std::shared_ptr& input_type, - const FunctionOptions* options) { - auto out = ::arrow::internal::make_unique(); - out->options_ = *checked_cast(options); - out->type_ = input_type; - - out->buffers_.resize(4); - for (auto& buf : out->buffers_) { - ctx->SetStatus(ctx->Allocate(0).Value(&buf)); - if (ctx->HasError()) return nullptr; - } - - GetImpl get_impl; - ctx->SetStatus(VisitTypeInline(*input_type, &get_impl)); - - out->consume_impl_ = std::move(get_impl.consume_impl); - out->resize_min_impl_ = std::move(get_impl.resize_min_impl); - out->resize_max_impl_ = std::move(get_impl.resize_max_impl); - return out; - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_num_groups = num_groups_; - num_groups_ = new_num_groups; - - KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); - KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); - - for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { - KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( - BitUtil::BytesForBits(new_num_groups))); - BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); - } - } - - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, &buffers_); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - // aggregation for group is valid if there was at least one value in that group - std::shared_ptr null_bitmap = std::move(buffers_[3]); - - if (options_.null_handling == MinMaxOptions::EMIT_NULL) { - // ... and there were no nulls in that group - arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, - num_groups(), 0, null_bitmap->mutable_data()); - } - - auto mins = - ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); - - auto maxes = ArrayData::Make(type_, num_groups(), - {std::move(null_bitmap), std::move(buffers_[1])}); - - *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, - {std::move(mins), std::move(maxes)}); - } - - int64_t num_groups() const override { return num_groups_; } - - std::shared_ptr out_type() const override { - return struct_({field("min", type_), field("max", type_)}); - } - - int64_t num_groups_; - BufferVector buffers_; - std::shared_ptr type_; - ConsumeImpl consume_impl_; - ResizeImpl resize_min_impl_, resize_max_impl_; - MinMaxOptions options_; -}; - std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique( static_cast(*args.options)); @@ -588,710 +203,6 @@ std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) return ::arrow::internal::make_unique(); } -struct GroupByImpl : public ScalarAggregator { - using AddLengthImpl = std::function&, int32_t*)>; - - struct GetAddLengthImpl { - static constexpr int32_t null_extra_byte = 1; - - static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, - int32_t* lengths) { - for (int64_t i = 0; i < num_repeats; ++i) { - lengths[i] += fixed_length + null_extra_byte; - } - } - - static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { - using offset_type = typename StringType::offset_type; - constexpr int32_t length_extra_bytes = sizeof(offset_type); - auto offset = data->offset; - const auto offsets = data->GetValues(1); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - - for (int64_t i = 0; i < data->length; ++i) { - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - lengths[i] += null_extra_byte + length_extra_bytes; - } else { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } - } - } - - template - Status Visit(const T& input_type) { - int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); - }; - return Status::OK(); - } - - Status Visit(const StringType&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); - }; - return Status::OK(); - } - - Status Visit(const BinaryType&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); - }; - return Status::OK(); - } - - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); - }; - return Status::OK(); - } - - AddLengthImpl add_length_impl; - }; - - using EncodeNextImpl = - std::function&, uint8_t**)>; - - struct GetEncodeNextImpl { - template - static void EncodeSmallFixed(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - uint64_t null_multiplier = is_null ? 0 : 1; - if (NumBits == 1) { - encoded_ptr[0] = static_cast( - null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 8; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - if (NumBits == 1) { - encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 8; - } - } - } - } - - static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - if (is_null) { - memset(encoded_ptr, 0, num_bytes); - } else { - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - } - encoded_ptr += num_bytes; - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - encoded_ptr += num_bytes; - } - } - } - - static void EncodeVarLength(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - using offset_type = typename StringType::offset_type; - auto offset = data->offset; - const auto offsets = data->GetValues(1); - auto raw_input = data->buffers[2]->data(); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - encoded_ptr[0] = 1; - encoded_ptr++; - reinterpret_cast(encoded_ptr)[0] = 0; - encoded_ptr += sizeof(offset_type); - } else { - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } - } - } - - template - Status Visit(const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - switch (num_bits) { - case 1: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<1>(data, encoded_bytes); - }; - break; - case 8: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<8>(data, encoded_bytes); - }; - break; - case 16: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<16>(data, encoded_bytes); - }; - break; - case 32: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<32>(data, encoded_bytes); - }; - break; - case 64: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<64>(data, encoded_bytes); - }; - break; - } - return Status::OK(); - } - - Status Visit(const StringType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); - }; - return Status::OK(); - } - - Status Visit(const BinaryType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); - }; - return Status::OK(); - } - - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - encode_next_impl = [num_bytes](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeBigFixed(num_bytes, data, encoded_bytes); - }; - return Status::OK(); - } - - EncodeNextImpl encode_next_impl; - }; - - using DecodeNextImpl = std::function*)>; - - struct GetDecodeNextImpl { - static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, - int32_t* null_count) { - // Do we have nulls? - *null_count = 0; - for (int32_t i = 0; i < length; ++i) { - *null_count += encoded_bytes[i][0]; - } - if (*null_count > 0) { - ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); - uint8_t* nulls = (*null_buf)->mutable_data(); - memset(nulls, 0, (*null_buf)->size()); - for (int32_t i = 0; i < length; ++i) { - if (!encoded_bytes[i][0]) { - BitUtil::SetBit(nulls, i); - } - encoded_bytes[i] += 1; - } - } else { - for (int32_t i = 0; i < length; ++i) { - encoded_bytes[i] += 1; - } - } - return Status ::OK(); - } - - template - static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, - int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - - KERNEL_ASSIGN_OR_RAISE( - auto key_buf, ctx, - ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); - - uint8_t* raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - if (NumBits == 1) { - BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - raw_output[i] = encoded_ptr[0]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 8; - } - } - - DCHECK(is_integer(output_type) || output_type == Type::BOOL); - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - } - - static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, - uint8_t** encoded_bytes, std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); - auto raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); - encoded_bytes[i] += num_bytes; - } - - *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, - null_count); - } - - static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - - using offset_type = typename StringType::offset_type; - - int32_t length_sum = 0; - for (int32_t i = 0; i < length; ++i) { - length_sum += reinterpret_cast(encoded_bytes)[0]; - } - - KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, - ctx->Allocate(sizeof(offset_type) * (1 + length))); - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); - - auto raw_offsets = offset_buf->mutable_data(); - auto raw_keys = key_buf->mutable_data(); - int32_t current_offset = 0; - for (int32_t i = 0; i < length; ++i) { - offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; - reinterpret_cast(raw_offsets)[i] = current_offset; - encoded_bytes[i] += sizeof(offset_type); - memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); - encoded_bytes[i] += key_length; - current_offset += key_length; - } - reinterpret_cast(raw_offsets)[length] = current_offset; - - if (is_string) { - *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } else { - *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } - } - - template - Status Visit(const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - auto type_id = input_type.id(); - switch (num_bits) { - case 1: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 8: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 16: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 32: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 64: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); - }; - break; - } - return Status::OK(); - } - - Status Visit(const StringType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, true, length, encoded_bytes, out); - }; - return Status::OK(); - } - - Status Visit(const BinaryType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, false, length, encoded_bytes, out); - }; - return Status::OK(); - } - - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); - }; - return Status::OK(); - } - - DecodeNextImpl decode_next_impl; - }; - - void Consume(KernelContext* ctx, const ExecBatch& batch) override { - ArrayDataVector aggregands, keys; - - size_t i; - for (i = 0; i < aggregators.size(); ++i) { - aggregands.push_back(batch[i].array()); - } - while (i < static_cast(batch.num_values())) { - keys.push_back(batch[i++].array()); - } - - offsets_batch_.clear(); - offsets_batch_.resize(batch.length + 1); - offsets_batch_[0] = 0; - memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); - for (size_t i = 0; i < keys.size(); ++i) { - add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); - } - int32_t total_length = 0; - for (int64_t i = 0; i < batch.length; ++i) { - auto total_length_before = total_length; - total_length += offsets_batch_[i]; - offsets_batch_[i] = total_length_before; - } - offsets_batch_[batch.length] = total_length; - - key_bytes_batch_.clear(); - key_bytes_batch_.resize(total_length); - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(batch.length); - for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; - } - for (size_t i = 0; i < keys.size(); ++i) { - encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); - } - - group_ids_batch_.clear(); - group_ids_batch_.resize(batch.length); - for (int64_t i = 0; i < batch.length; ++i) { - int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; - std::string key( - reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), - key_length); - auto iter = map_.find(key); - if (iter == map_.end()) { - group_ids_batch_[i] = n_groups++; - auto next_key_offset = static_cast(key_bytes_.size()); - key_bytes_.resize(next_key_offset + key_length); - offsets_.push_back(next_key_offset + key_length); - memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); - map_.insert(std::make_pair(key, group_ids_batch_[i])); - } else { - group_ids_batch_[i] = iter->second; - } - } - - for (size_t i = 0; i < aggregators.size(); ++i) { - aggregators[i]->Consume(ctx, aggregands[i], group_ids_batch_.data()); - if (ctx->HasError()) return; - } - } - - void MergeFrom(KernelContext* ctx, KernelState&& src) override { - // TODO(ARROW-11840) merge two hash tables - ctx->SetStatus(Status::NotImplemented("merging grouped aggregations")); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - size_t n_keys = decode_next_impl.size(); - ArrayDataVector out_columns(aggregators.size() + n_keys); - for (size_t i = 0; i < aggregators.size(); ++i) { - Datum aggregand; - aggregators[i]->Finalize(ctx, &aggregand); - if (ctx->HasError()) return; - out_columns[i] = aggregand.array(); - } - - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(n_groups); - for (int64_t i = 0; i < n_groups; ++i) { - key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; - } - - int64_t length = n_groups; - for (size_t i = 0; i < n_keys; ++i) { - std::shared_ptr key_array; - decode_next_impl[i].decode_next_impl(ctx, static_cast(length), - key_buf_ptrs_.data(), &key_array); - out_columns[aggregators.size() + i] = std::move(key_array); - } - - *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, - std::move(out_columns)); - } - std::vector offsets_batch_; - std::vector key_bytes_batch_; - std::vector key_buf_ptrs_; - std::vector group_ids_batch_; - - std::unordered_map map_; - std::vector offsets_; - std::vector key_bytes_; - uint32_t n_groups; - - std::shared_ptr out_type; - GroupByOptions options; - std::vector> aggregators; - - std::vector add_length_impl; - std::vector encode_next_impl; - std::vector decode_next_impl; -}; - -template -std::unique_ptr MakeAggregator(KernelContext* ctx, - const std::string& function_name, - const std::shared_ptr& input_type, - const FunctionOptions* options) { - if (options == nullptr) { - if (auto function = ctx->exec_context() - ->func_registry() - ->GetFunction(function_name) - .ValueOr(nullptr)) { - options = function->default_options(); - } - } - - return Aggregator::Make(ctx, input_type, options); -} - -std::unique_ptr GroupByInit(KernelContext* ctx, const KernelInitArgs& args) { - auto impl = ::arrow::internal::make_unique(); - impl->options = *checked_cast(args.options); - const auto& aggregates = impl->options.aggregates; - - impl->n_groups = 0; - impl->offsets_.push_back(0); - - if (aggregates.size() > args.inputs.size()) { - ctx->SetStatus(Status::Invalid("more aggegates than inputs!")); - return nullptr; - } - - FieldVector out_fields(args.inputs.size()); - - impl->aggregators.resize(aggregates.size()); - for (size_t i = 0; i < aggregates.size(); ++i) { - const std::string& function = aggregates[i].function; - const FunctionOptions* options = aggregates[i].options; - const auto& input_type = args.inputs[i].type; - - if (function == "count") { - impl->aggregators[i] = - MakeAggregator(ctx, function, input_type, options); - } else if (function == "sum") { - impl->aggregators[i] = - MakeAggregator(ctx, function, input_type, options); - } else if (function == "min_max") { - impl->aggregators[i] = - MakeAggregator(ctx, function, input_type, options); - } else { - ctx->SetStatus(Status::NotImplemented("Grouped aggregate ", function)); - } - if (ctx->HasError()) return nullptr; - - out_fields[i] = field("", impl->aggregators[i]->out_type()); - } - - size_t n_keys = args.inputs.size() - aggregates.size(); - for (size_t i = 0; i < n_keys; ++i) { - const auto& key_type = args.inputs[aggregates.size() + i].type; - switch (key_type->id()) { - // Supported types of keys - case Type::BOOL: - case Type::UINT8: - case Type::INT8: - case Type::UINT16: - case Type::INT16: - case Type::UINT32: - case Type::INT32: - case Type::UINT64: - case Type::INT64: - case Type::STRING: - case Type::BINARY: - case Type::FIXED_SIZE_BINARY: - break; - default: - ctx->SetStatus(Status::NotImplemented("Key of type", key_type->ToString())); - return nullptr; - } - out_fields[aggregates.size() + i] = field("", key_type); - } - - impl->add_length_impl.resize(n_keys); - impl->encode_next_impl.resize(n_keys); - impl->decode_next_impl.resize(n_keys); - for (size_t i = 0; i < n_keys; ++i) { - const auto& key_type = args.inputs[aggregates.size() + i].type; - ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->add_length_impl[i])); - ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->encode_next_impl[i])); - ctx->SetStatus(VisitTypeInline(*key_type.get(), &impl->decode_next_impl[i])); - } - - impl->out_type = struct_(std::move(out_fields)); - - return impl; -} - void AddBasicAggKernels(KernelInit init, const std::vector>& types, std::shared_ptr out_ty, ScalarAggregateFunction* func, @@ -1347,13 +258,6 @@ const FunctionDoc all_doc{"Test whether all elements in a boolean array evaluate ("Null values are ignored."), {"array"}}; -const FunctionDoc group_by_doc{ - ("Compute aggregations on input arrays, grouped by key columns"), - ("Leading arguments are passed to the corresponding aggregation function\n" - "named in GroupByOptions, remaining inputs are used as keys for grouping."), - {"*args"}, - "GroupByOptions"}; - } // namespace void RegisterScalarAggregateBasic(FunctionRegistry* registry) { @@ -1436,22 +340,6 @@ void RegisterScalarAggregateBasic(FunctionRegistry* registry) { func = std::make_shared("all", Arity::Unary(), &all_doc); aggregate::AddBasicAggKernels(aggregate::AllInit, {boolean()}, boolean(), func.get()); DCHECK_OK(registry->AddFunction(std::move(func))); - - // group_by - func = std::make_shared("group_by", Arity::VarArgs(), - &group_by_doc); - { - auto sig = KernelSignature::Make( - {ValueDescr::ARRAY}, - OutputType([](KernelContext* ctx, const std::vector&) { - return Result{ - checked_cast(ctx->state())->out_type}; - }), - /*is_varargs=*/true); - AddAggKernel(std::move(sig), aggregate::GroupByInit, func.get(), SimdLevel::NONE, - true); - } - DCHECK_OK(registry->AddFunction(std::move(func))); } } // namespace internal diff --git a/cpp/src/arrow/compute/kernels/aggregate_internal.h b/cpp/src/arrow/compute/kernels/aggregate_internal.h index c1757c97ffe..67337f22c5b 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/aggregate_internal.h @@ -57,7 +57,7 @@ struct ScalarAggregator : public KernelState { void AddAggKernel(std::shared_ptr sig, KernelInit init, ScalarAggregateFunction* func, - SimdLevel::type simd_level = SimdLevel::NONE, bool nomerge = false); + SimdLevel::type simd_level = SimdLevel::NONE); namespace detail { diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index e7872d171f8..1b2896b629f 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -31,6 +31,7 @@ #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" +#include "arrow/compute/kernels/hash_aggregate_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" #include "arrow/type.h" @@ -52,506 +53,6 @@ using internal::checked_pointer_cast; namespace compute { -// Copy-pasta from partition.cc -// -// In the finished product this will only be a test helper for group_by -// and partition.cc will rely on a no-aggregate call to group_by. -namespace group_helpers { -namespace { - -// Transform an array of counts to offsets which will divide a ListArray -// into an equal number of slices with corresponding lengths. -Result> CountsToOffsets(std::shared_ptr counts) { - TypedBufferBuilder offset_builder; - RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); - - int32_t current_offset = 0; - offset_builder.UnsafeAppend(current_offset); - - for (int64_t i = 0; i < counts->length(); ++i) { - DCHECK_NE(counts->Value(i), 0); - current_offset += static_cast(counts->Value(i)); - offset_builder.UnsafeAppend(current_offset); - } - - std::shared_ptr offsets; - RETURN_NOT_OK(offset_builder.Finish(&offsets)); - return offsets; -} - -class StructDictionary { - public: - struct Encoded { - std::shared_ptr indices; - std::shared_ptr dictionary; - }; - - static Result Encode(const ArrayVector& columns) { - Encoded out{nullptr, std::make_shared()}; - - for (const auto& column : columns) { - if (column->null_count() != 0) { - return Status::NotImplemented("Grouping on a field with nulls"); - } - - RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices)); - } - - return out; - } - - Result> Decode(std::shared_ptr fused_indices, - FieldVector fields) { - std::vector builders(dictionaries_.size()); - for (Int32Builder& b : builders) { - RETURN_NOT_OK(b.Resize(fused_indices->length())); - } - - std::vector codes(dictionaries_.size()); - for (int64_t i = 0; i < fused_indices->length(); ++i) { - Expand(fused_indices->Value(i), codes.data()); - - auto builder_it = builders.begin(); - for (int32_t index : codes) { - builder_it++->UnsafeAppend(index); - } - } - - ArrayVector columns(dictionaries_.size()); - for (size_t i = 0; i < dictionaries_.size(); ++i) { - std::shared_ptr indices; - RETURN_NOT_OK(builders[i].FinishInternal(&indices)); - - ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices)); - - if (fields[i]->type()->id() == Type::DICTIONARY) { - RETURN_NOT_OK(RestoreDictionaryEncoding( - checked_pointer_cast(fields[i]->type()), &column)); - } - - columns[i] = column.make_array(); - } - - return StructArray::Make(std::move(columns), std::move(fields)); - } - - private: - Status AddOne(Datum column, std::shared_ptr* fused_indices) { - if (column.type()->id() != Type::DICTIONARY) { - ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(std::move(column))); - } - - auto dict_column = column.array_as(); - dictionaries_.push_back(dict_column->dictionary()); - ARROW_ASSIGN_OR_RAISE(auto indices, compute::Cast(*dict_column->indices(), int32())); - - if (*fused_indices == nullptr) { - *fused_indices = checked_pointer_cast(std::move(indices)); - return IncreaseSize(); - } - - // It's useful to think about the case where each of dictionaries_ has size 10. - // In this case the decimal digit in the ones place is the code in dictionaries_[0], - // the tens place corresponds to the code in dictionaries_[1], etc. - // The incumbent indices must be shifted to the hundreds place so as not to collide. - ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices, - compute::Multiply(indices, MakeScalar(size_))); - - ARROW_ASSIGN_OR_RAISE(new_fused_indices, - compute::Add(new_fused_indices, *fused_indices)); - - *fused_indices = checked_pointer_cast(new_fused_indices.make_array()); - return IncreaseSize(); - } - - // expand a fused code into component dict codes, order is in order of addition - void Expand(int32_t fused_code, int32_t* codes) { - for (size_t i = 0; i < dictionaries_.size(); ++i) { - auto dictionary_size = static_cast(dictionaries_[i]->length()); - codes[i] = fused_code % dictionary_size; - fused_code /= dictionary_size; - } - } - - Status RestoreDictionaryEncoding(std::shared_ptr expected_type, - Datum* column) { - DCHECK_NE(column->type()->id(), Type::DICTIONARY); - ARROW_ASSIGN_OR_RAISE(*column, compute::DictionaryEncode(std::move(*column))); - - if (expected_type->index_type()->id() == Type::INT32) { - // dictionary_encode has already yielded the expected index_type - return Status::OK(); - } - - // cast the indices to the expected index type - auto dictionary = std::move(column->mutable_array()->dictionary); - column->mutable_array()->type = int32(); - - ARROW_ASSIGN_OR_RAISE(*column, - compute::Cast(std::move(*column), expected_type->index_type())); - - column->mutable_array()->dictionary = std::move(dictionary); - column->mutable_array()->type = expected_type; - return Status::OK(); - } - - Status IncreaseSize() { - auto factor = static_cast(dictionaries_.back()->length()); - - if (arrow::internal::MultiplyWithOverflow(size_, factor, &size_)) { - return Status::CapacityError("Max groups exceeded"); - } - return Status::OK(); - } - - int32_t size_ = 1; - ArrayVector dictionaries_; -}; - -Result> MakeGroupings(const StructArray& keys) { - if (keys.num_fields() == 0) { - return Status::Invalid("Grouping with no keys"); - } - - if (keys.null_count() != 0) { - return Status::Invalid("Grouping with null keys"); - } - - ARROW_ASSIGN_OR_RAISE(auto fused, StructDictionary::Encode(keys.fields())); - - ARROW_ASSIGN_OR_RAISE(auto sort_indices, compute::SortIndices(*fused.indices)); - ARROW_ASSIGN_OR_RAISE(Datum sorted, compute::Take(fused.indices, *sort_indices)); - fused.indices = checked_pointer_cast(sorted.make_array()); - - ARROW_ASSIGN_OR_RAISE(auto fused_counts_and_values, - compute::ValueCounts(fused.indices)); - fused.indices.reset(); - - auto unique_fused_indices = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("values")); - ARROW_ASSIGN_OR_RAISE( - auto unique_rows, - fused.dictionary->Decode(std::move(unique_fused_indices), keys.type()->fields())); - - auto counts = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("counts")); - ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); - - auto grouped_sort_indices = - std::make_shared(list(sort_indices->type()), unique_rows->length(), - std::move(offsets), std::move(sort_indices)); - - return StructArray::Make( - ArrayVector{std::move(unique_rows), std::move(grouped_sort_indices)}, - std::vector{"values", "groupings"}); -} - -Result> ApplyGroupings(const ListArray& groupings, - const Array& array) { - ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(array, groupings.data()->child_data[0])); - - return std::make_shared(list(array.type()), groupings.length(), - groupings.value_offsets(), sorted.make_array()); -} - -struct ScalarVectorToArray { - template ::BuilderType, - typename ScalarType = typename TypeTraits::ScalarType> - Status UseBuilder(const AppendScalar& append) { - BuilderType builder(type(), default_memory_pool()); - for (const auto& s : scalars_) { - if (s->is_valid) { - RETURN_NOT_OK(append(checked_cast(*s), &builder)); - } else { - RETURN_NOT_OK(builder.AppendNull()); - } - } - return builder.FinishInternal(&data_); - } - - struct AppendValue { - template - Status operator()(const ScalarType& s, BuilderType* builder) const { - return builder->Append(s.value); - } - }; - - struct AppendBuffer { - template - Status operator()(const ScalarType& s, BuilderType* builder) const { - const Buffer& buffer = *s.value; - return builder->Append(util::string_view{buffer}); - } - }; - - template - enable_if_primitive_ctype Visit(const T&) { - return UseBuilder(AppendValue{}); - } - - template - enable_if_has_string_view Visit(const T&) { - return UseBuilder(AppendBuffer{}); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("ScalarVectorToArray for type ", type); - } - - Result Convert(ScalarVector scalars) && { - if (scalars.size() == 0) { - return Status::NotImplemented("ScalarVectorToArray with no scalars"); - } - scalars_ = std::move(scalars); - RETURN_NOT_OK(VisitTypeInline(*type(), this)); - return Datum(std::move(data_)); - } - - const std::shared_ptr& type() { return scalars_[0]->type; } - - ScalarVector scalars_; - std::shared_ptr data_; -}; - -Result NaiveGroupBy(std::vector aggregands, std::vector keys, - GroupByOptions options) { - ArrayVector keys_arrays; - for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); - std::vector key_names(keys_arrays.size(), ""); - ARROW_ASSIGN_OR_RAISE(auto keys_struct, - StructArray::Make(std::move(keys_arrays), std::move(key_names))); - - ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); - - auto groupings = - checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); - - int64_t n_groups = groupings->length(); - - ArrayVector out_columns; - - for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { - const Datum& aggregand = aggregands[i_agg]; - const std::string& function = options.aggregates[i_agg].function; - - ScalarVector aggregated_scalars; - - ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, - ApplyGroupings(*groupings, *aggregand.make_array())); - - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE( - Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); - aggregated_scalars.push_back(d.scalar()); - } - - ARROW_ASSIGN_OR_RAISE(Datum aggregated_column, - ScalarVectorToArray{}.Convert(std::move(aggregated_scalars))); - out_columns.push_back(aggregated_column.make_array()); - } - - keys_struct = - checked_pointer_cast(groupings_and_values->GetFieldByName("values")); - for (size_t i_key = 0; i_key < aggregands.size(); ++i_key) { - out_columns.push_back(keys_struct->field(i_key)); - } - - std::vector out_names(out_columns.size(), ""); - return StructArray::Make(std::move(out_columns), std::move(out_names)); -} - -void ValidateGroupBy(GroupByOptions options, std::vector aggregands, - std::vector keys) { - ASSERT_OK_AND_ASSIGN(Datum expected, - group_helpers::NaiveGroupBy(aggregands, keys, options)); - - ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, options)); - - // Ordering of groups is not important, so sort by key columns to ensure the comparison - // doesn't fail spuriously - - for (Datum* out : {&expected, &actual}) { - auto out_columns = out->array_as()->fields(); - - SortOptions sort_options; - FieldVector key_fields; - ArrayVector key_columns; - for (size_t i = 0; i < keys.size(); ++i) { - auto name = std::to_string(i); - sort_options.sort_keys.emplace_back(name); - key_fields.push_back(field(name, out_columns[0]->type())); - key_columns.push_back(out_columns[0]); - } - auto key_batch = RecordBatch::Make(schema(std::move(key_fields)), out->length(), - std::move(key_columns)); - - ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(key_batch, sort_options)); - ASSERT_OK_AND_ASSIGN(*out, Take(*out, sort_indices, TakeOptions::NoBoundsCheck())); - } - - AssertDatumsEqual(expected, actual, /*verbose=*/true); -} - -} // namespace -} // namespace group_helpers - -TEST(GroupBy, SumOnly) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"sum", nullptr}, - })); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int64()), - }), - R"([ - [4.25, 1], - [-0.125, 2], - [null, 3], - [0.75, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, MinMaxOnly) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"min_max", nullptr}, - })); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", struct_({ - field("min", float64()), - field("max", float64()), - })), - field("", int64()), - }), - R"([ - [{"min": 1.0, "max": 3.25}, 1], - [{"min": -0.25, "max": 0.125}, 2], - [{"min": null, "max": null}, 3], - [{"min": 0.75, "max": 0.75}, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, CountAndSum) { - auto aggregand = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); - - CountOptions count_options; - - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - // NB: passing an aggregand twice or also using it as a key is legal - GroupBy({aggregand, aggregand, key}, {key}, - GroupByOptions{ - {"count", &count_options}, - {"sum", nullptr}, - {"sum", nullptr}, - })); - - AssertDatumsEqual( - ArrayFromJSON(struct_({ - field("", int64()), - // NB: summing a float32 array results in float64 sums - field("", float64()), - field("", int64()), - field("", int64()), - }), - R"([ - [1, 1.0, 2, 1], - [2, 0.125, 4, 2], - [2, 3.0, 6, 3], - [1, 0.75, null, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, StringKey) { - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); - auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - GroupBy({aggregand}, {key}, GroupByOptions{{"sum", nullptr}})); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", int64()), - field("", utf8()), - }), - R"([ - [10, "alfa"], - [14, "beta"], - [6, "gamma"], - [12, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, MultipleKeys) { - auto aggregand = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); - auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); - auto str_key = - ArrayFromJSON(utf8(), R"(["beta", "beta", "gamma", "gamma", null, "beta"])"); - - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - GroupBy({aggregand}, {int_key, str_key}, GroupByOptions{{"sum", nullptr}})); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int32()), - field("", utf8()), - }), - R"([ - [0.125, 0, "beta"], - [2.5, 1, "beta"], - [-0.75, 0, "gamma"], - [8, 1, "gamma"], - [1.0, 0, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); - auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - - group_helpers::ValidateGroupBy(GroupByOptions{{"sum", nullptr}}, {aggregand}, {key}); -} - -TEST(GroupBy, RandomArraySum) { - auto rand = random::RandomArrayGenerator(0xdeadbeef); - - for (size_t i = 3; i < 14; i += 2) { - for (auto null_probability : {0.0, 0.001, 0.1, 0.5, 0.999, 1.0}) { - int64_t length = 1UL << i; - auto summand = rand.Float32(length, -100, 100, null_probability); - auto key = rand.Int64(length, 0, 12); - - group_helpers::ValidateGroupBy( - GroupByOptions{ - {"sum", nullptr}, - }, - {summand}, {key}); - } - } -} - // // Sum // diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc new file mode 100644 index 00000000000..199afeed215 --- /dev/null +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -0,0 +1,1115 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/kernels/hash_aggregate_internal.h" + +#include + +#include "arrow/compute/api_aggregate.h" +#include "arrow/compute/exec_internal.h" +#include "arrow/compute/kernels/aggregate_basic_internal.h" +#include "arrow/compute/kernels/aggregate_internal.h" +#include "arrow/compute/kernels/common.h" +#include "arrow/util/bit_run_reader.h" +#include "arrow/util/bitmap_ops.h" +#include "arrow/util/cpu_info.h" +#include "arrow/util/make_unique.h" + +namespace arrow { +namespace compute { +namespace aggregate { + +// ---------------------------------------------------------------------- +// Count implementation + +struct GroupedCountImpl : public GroupedAggregator { + void Init(KernelContext* ctx, const FunctionOptions* options, + const std::shared_ptr&) override { + options_ = checked_cast(*options); + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups(); + KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); + auto new_size = num_groups(); + + auto raw_counts = reinterpret_cast(counts_->mutable_data()); + for (auto i = old_size; i < new_size; ++i) { + raw_counts[i] = 0; + } + } + + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + + auto raw_counts = reinterpret_cast(counts_->mutable_data()); + + const auto& input = aggregand.array(); + + if (options_.count_mode == CountOptions::COUNT_NULL) { + for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { + auto g = group_ids[i]; + raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); + } + return; + } + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_counts[g] += 1; + } + }); + } + + void Finalize(KernelContext* ctx, Datum* out) override { + auto length = num_groups(); + *out = std::make_shared(length, std::move(counts_)); + } + + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } + + std::shared_ptr out_type() const override { return int64(); } + + CountOptions options_; + std::shared_ptr counts_; +}; + +// ---------------------------------------------------------------------- +// Sum implementation + +struct GroupedSumImpl : public GroupedAggregator { + // NB: whether we are accumulating into double, int64_t, or uint64_t + // we always have 64 bits per group in the sums buffer. + static constexpr size_t kSumSize = sizeof(int64_t); + + using ConsumeImpl = std::function&, + const uint32_t*, Buffer*, Buffer*)>; + + struct GetConsumeImpl { + template ::Type> + Status Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { + auto raw_input = reinterpret_cast::CType*>( + input->buffers[1]->data()); + auto raw_sums = reinterpret_cast::CType*>( + sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_sums[g] += raw_input[input_i]; + raw_counts[g] += 1; + } + }); + }; + out_type = TypeTraits::type_singleton(); + return Status::OK(); + } + + Status Visit(const BooleanType&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { + auto raw_input = input->buffers[1]->data(); + auto raw_sums = reinterpret_cast(sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i) { + auto g = group_ids[i]; + raw_sums[g] += BitUtil::GetBit(raw_input, input_i); + raw_counts[g] += 1; + } + }); + }; + out_type = uint64(); + return Status::OK(); + } + + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Summing data of type ", type); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("Summing data of type ", type); + } + + ConsumeImpl consume_impl; + std::shared_ptr out_type; + }; + + void Init(KernelContext* ctx, const FunctionOptions*, + const std::shared_ptr& input_type) override { + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&sums_)); + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); + + GetConsumeImpl get_consume_impl; + KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_consume_impl)); + + consume_impl_ = std::move(get_consume_impl.consume_impl); + out_type_ = std::move(get_consume_impl.out_type); + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups() * kSumSize; + KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); + KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); + auto new_size = num_groups() * kSumSize; + std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); + std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); + } + + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); + } + + void Finalize(KernelContext* ctx, Datum* out) override { + std::shared_ptr null_bitmap; + int64_t null_count = 0; + + for (int64_t i = 0; i < num_groups(); ++i) { + if (reinterpret_cast(counts_->data())[i] > 0) continue; + + if (null_bitmap == nullptr) { + KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); + BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); + } + + null_count += 1; + BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); + } + + *out = ArrayData::Make(std::move(out_type_), num_groups(), + {std::move(null_bitmap), std::move(sums_)}, null_count); + } + + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } + + std::shared_ptr out_type() const override { return out_type_; } + + // NB: counts are used here instead of a simple "has_values_" bitmap since + // we expect to reuse this kernel to handle Mean + std::shared_ptr sums_, counts_; + std::shared_ptr out_type_; + ConsumeImpl consume_impl_; +}; + +// ---------------------------------------------------------------------- +// MinMax implementation + +struct GroupedMinMaxImpl : public GroupedAggregator { + using ConsumeImpl = std::function&, + const uint32_t*, BufferVector*)>; + + using ResizeImpl = std::function; + + struct GetImpl { + template ::CType> + enable_if_number Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, BufferVector* buffers) { + auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); + auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); + + auto raw_has_nulls = buffers->at(2)->mutable_data(); + auto raw_has_values = buffers->at(3)->mutable_data(); + + auto g = group_ids; + VisitArrayDataInline( + *input, + [&](CType val) { + raw_maxes[*g] = std::max(raw_maxes[*g], val); + raw_mins[*g] = std::min(raw_mins[*g], val); + BitUtil::SetBit(raw_has_values, *g++); + }, + [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); + }; + + GetResizeImpls(); + return Status::OK(); + } + + Status Visit(const BooleanType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); + } + + template + ResizeImpl MakeResizeImpl(CType anti_extreme) { + // resize a min or max buffer, storing the correct anti extreme + return [anti_extreme](Buffer* vals, int64_t new_num_groups) { + int64_t old_num_groups = vals->size() / sizeof(CType); + + int64_t new_size = new_num_groups * sizeof(CType); + RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); + + auto raw_vals = reinterpret_cast(vals->mutable_data()); + for (int64_t i = old_num_groups; i != new_num_groups; ++i) { + raw_vals[i] = anti_extreme; + } + return Status::OK(); + }; + } + + template ::CType> + enable_if_floating_point GetResizeImpls() { + auto inf = std::numeric_limits::infinity(); + resize_min_impl = MakeResizeImpl(inf); + resize_max_impl = MakeResizeImpl(-inf); + } + + template ::CType> + enable_if_integer GetResizeImpls() { + resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); + resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); + } + + ConsumeImpl consume_impl; + ResizeImpl resize_min_impl, resize_max_impl; + }; + + void Init(KernelContext* ctx, const FunctionOptions* options, + const std::shared_ptr& input_type) override { + options_ = *checked_cast(options); + type_ = input_type; + + buffers_.resize(4); + for (auto& buf : buffers_) { + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&buf)); + } + + GetImpl get_impl; + KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_impl)); + + consume_impl_ = std::move(get_impl.consume_impl); + resize_min_impl_ = std::move(get_impl.resize_min_impl); + resize_max_impl_ = std::move(get_impl.resize_max_impl); + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_num_groups = num_groups_; + num_groups_ = new_num_groups; + + KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); + KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); + + for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { + KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( + BitUtil::BytesForBits(new_num_groups))); + BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); + } + } + + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + consume_impl_(aggregand.array(), group_ids, &buffers_); + } + + void Finalize(KernelContext* ctx, Datum* out) override { + // aggregation for group is valid if there was at least one value in that group + std::shared_ptr null_bitmap = std::move(buffers_[3]); + + if (options_.null_handling == MinMaxOptions::EMIT_NULL) { + // ... and there were no nulls in that group + arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, + num_groups(), 0, null_bitmap->mutable_data()); + } + + auto mins = + ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); + + auto maxes = ArrayData::Make(type_, num_groups(), + {std::move(null_bitmap), std::move(buffers_[1])}); + + *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, + {std::move(mins), std::move(maxes)}); + } + + int64_t num_groups() const override { return num_groups_; } + + std::shared_ptr out_type() const override { + return struct_({field("min", type_), field("max", type_)}); + } + + int64_t num_groups_; + BufferVector buffers_; + std::shared_ptr type_; + ConsumeImpl consume_impl_; + ResizeImpl resize_min_impl_, resize_max_impl_; + MinMaxOptions options_; +}; + +struct GroupByImpl { + using AddLengthImpl = std::function&, int32_t*)>; + + struct GetAddLengthImpl { + static constexpr int32_t null_extra_byte = 1; + + static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, + int32_t* lengths) { + for (int64_t i = 0; i < num_repeats; ++i) { + lengths[i] += fixed_length + null_extra_byte; + } + } + + static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { + using offset_type = typename StringType::offset_type; + constexpr int32_t length_extra_bytes = sizeof(offset_type); + auto offset = data->offset; + const auto offsets = data->GetValues(1); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + + for (int64_t i = 0; i < data->length; ++i) { + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + lengths[i] += null_extra_byte + length_extra_bytes; + } else { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; + } + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); + }; + return Status::OK(); + } + + Status Visit(const StringType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); + }; + return Status::OK(); + } + + AddLengthImpl add_length_impl; + }; + + using EncodeNextImpl = + std::function&, uint8_t**)>; + + struct GetEncodeNextImpl { + template + static void EncodeSmallFixed(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + uint64_t null_multiplier = is_null ? 0 : 1; + if (NumBits == 1) { + encoded_ptr[0] = static_cast( + null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 8; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + if (NumBits == 1) { + encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 8; + } + } + } + } + + static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, + uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + if (is_null) { + memset(encoded_ptr, 0, num_bytes); + } else { + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + } + encoded_ptr += num_bytes; + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + encoded_ptr += num_bytes; + } + } + } + + static void EncodeVarLength(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + using offset_type = typename StringType::offset_type; + auto offset = data->offset; + const auto offsets = data->GetValues(1); + auto raw_input = data->buffers[2]->data(); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + encoded_ptr[0] = 1; + encoded_ptr++; + reinterpret_cast(encoded_ptr)[0] = 0; + encoded_ptr += sizeof(offset_type); + } else { + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bits = bit_width(input_type.id()); + switch (num_bits) { + case 1: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<1>(data, encoded_bytes); + }; + break; + case 8: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<8>(data, encoded_bytes); + }; + break; + case 16: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<16>(data, encoded_bytes); + }; + break; + case 32: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<32>(data, encoded_bytes); + }; + break; + case 64: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<64>(data, encoded_bytes); + }; + break; + } + return Status::OK(); + } + + Status Visit(const StringType&) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + encode_next_impl = [num_bytes](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeBigFixed(num_bytes, data, encoded_bytes); + }; + return Status::OK(); + } + + EncodeNextImpl encode_next_impl; + }; + + using DecodeNextImpl = std::function*)>; + + struct GetDecodeNextImpl { + static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, + int32_t* null_count) { + // Do we have nulls? + *null_count = 0; + for (int32_t i = 0; i < length; ++i) { + *null_count += encoded_bytes[i][0]; + } + if (*null_count > 0) { + ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); + uint8_t* nulls = (*null_buf)->mutable_data(); + memset(nulls, 0, (*null_buf)->size()); + for (int32_t i = 0; i < length; ++i) { + if (!encoded_bytes[i][0]) { + BitUtil::SetBit(nulls, i); + } + encoded_bytes[i] += 1; + } + } else { + for (int32_t i = 0; i < length; ++i) { + encoded_bytes[i] += 1; + } + } + return Status ::OK(); + } + + template + static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, + int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + + KERNEL_ASSIGN_OR_RAISE( + auto key_buf, ctx, + ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); + + uint8_t* raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + if (NumBits == 1) { + BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + raw_output[i] = encoded_ptr[0]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 8; + } + } + + DCHECK(is_integer(output_type) || output_type == Type::BOOL); + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + } + + static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, + uint8_t** encoded_bytes, std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); + auto raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); + encoded_bytes[i] += num_bytes; + } + + *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, + null_count); + } + + static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + + using offset_type = typename StringType::offset_type; + + int32_t length_sum = 0; + for (int32_t i = 0; i < length; ++i) { + length_sum += reinterpret_cast(encoded_bytes)[0]; + } + + KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, + ctx->Allocate(sizeof(offset_type) * (1 + length))); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); + + auto raw_offsets = offset_buf->mutable_data(); + auto raw_keys = key_buf->mutable_data(); + int32_t current_offset = 0; + for (int32_t i = 0; i < length; ++i) { + offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_offsets)[i] = current_offset; + encoded_bytes[i] += sizeof(offset_type); + memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); + encoded_bytes[i] += key_length; + current_offset += key_length; + } + reinterpret_cast(raw_offsets)[length] = current_offset; + + if (is_string) { + *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); + } else { + *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); + } + } + + template + Status Visit(const T& input_type) { + int32_t num_bits = bit_width(input_type.id()); + auto type_id = input_type.id(); + switch (num_bits) { + case 1: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 8: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 16: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 32: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 64: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); + }; + break; + } + return Status::OK(); + } + + Status Visit(const StringType&) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeVarLength(ctx, true, length, encoded_bytes, out); + }; + return Status::OK(); + } + + Status Visit(const BinaryType&) { + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeVarLength(ctx, false, length, encoded_bytes, out); + }; + return Status::OK(); + } + + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); + }; + return Status::OK(); + } + + DecodeNextImpl decode_next_impl; + }; + + void Consume(KernelContext* ctx, const ExecBatch& batch) { + ArrayDataVector aggregands, keys; + + size_t i; + for (i = 0; i < aggregators.size(); ++i) { + aggregands.push_back(batch[i].array()); + } + while (i < static_cast(batch.num_values())) { + keys.push_back(batch[i++].array()); + } + + offsets_batch_.clear(); + offsets_batch_.resize(batch.length + 1); + offsets_batch_[0] = 0; + memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); + for (size_t i = 0; i < keys.size(); ++i) { + add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); + } + int32_t total_length = 0; + for (int64_t i = 0; i < batch.length; ++i) { + auto total_length_before = total_length; + total_length += offsets_batch_[i]; + offsets_batch_[i] = total_length_before; + } + offsets_batch_[batch.length] = total_length; + + key_bytes_batch_.clear(); + key_bytes_batch_.resize(total_length); + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; + } + for (size_t i = 0; i < keys.size(); ++i) { + encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); + } + + group_ids_batch_.clear(); + group_ids_batch_.resize(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; + std::string key( + reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), + key_length); + auto iter = map_.find(key); + if (iter == map_.end()) { + group_ids_batch_[i] = n_groups++; + auto next_key_offset = static_cast(key_bytes_.size()); + key_bytes_.resize(next_key_offset + key_length); + offsets_.push_back(next_key_offset + key_length); + memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); + map_.insert(std::make_pair(key, group_ids_batch_[i])); + } else { + group_ids_batch_[i] = iter->second; + } + } + + for (size_t i = 0; i < aggregators.size(); ++i) { + aggregators[i]->Consume(ctx, aggregands[i], group_ids_batch_.data()); + if (ctx->HasError()) return; + } + } + + void MergeFrom(KernelContext* ctx, KernelState&& src) { + // TODO(ARROW-11840) merge two hash tables + ctx->SetStatus(Status::NotImplemented("merging grouped aggregations")); + } + + void Finalize(KernelContext* ctx, Datum* out) { + size_t n_keys = decode_next_impl.size(); + ArrayDataVector out_columns(aggregators.size() + n_keys); + for (size_t i = 0; i < aggregators.size(); ++i) { + Datum aggregand; + aggregators[i]->Finalize(ctx, &aggregand); + if (ctx->HasError()) return; + out_columns[i] = aggregand.array(); + } + + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(n_groups); + for (int64_t i = 0; i < n_groups; ++i) { + key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; + } + + int64_t length = n_groups; + for (size_t i = 0; i < n_keys; ++i) { + std::shared_ptr key_array; + decode_next_impl[i].decode_next_impl(ctx, static_cast(length), + key_buf_ptrs_.data(), &key_array); + out_columns[aggregators.size() + i] = std::move(key_array); + } + + *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, + std::move(out_columns)); + } + + std::vector offsets_batch_; + std::vector key_bytes_batch_; + std::vector key_buf_ptrs_; + std::vector group_ids_batch_; + + std::unordered_map map_; + std::vector offsets_; + std::vector key_bytes_; + uint32_t n_groups; + + std::shared_ptr out_type; + GroupByOptions options; + std::vector> aggregators; + + std::vector add_length_impl; + std::vector encode_next_impl; + std::vector decode_next_impl; +}; + +Result GroupByInit(ExecContext* ctx, const std::vector& aggregands, + const std::vector& keys, + const GroupByOptions& options) { + GroupByImpl impl; + impl.options = options; + const auto& aggregates = impl.options.aggregates; + + impl.n_groups = 0; + impl.offsets_.push_back(0); + + if (aggregates.size() != aggregands.size()) { + return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", + aggregands.size(), " aggregands were provided."); + } + + FieldVector out_fields; + + impl.aggregators.resize(aggregates.size()); + for (size_t i = 0; i < aggregates.size(); ++i) { + const std::string& function = aggregates[i].function; + + if (function == "count") { + impl.aggregators[i] = ::arrow::internal::make_unique(); + } else if (function == "sum") { + impl.aggregators[i] = ::arrow::internal::make_unique(); + } else if (function == "min_max") { + impl.aggregators[i] = ::arrow::internal::make_unique(); + } else { + return Status::NotImplemented("Grouped aggregate ", function); + } + + const FunctionOptions* options = aggregates[i].options; + if (options == nullptr) { + // use known default options for the named function if possible + auto maybe_function = ctx->func_registry()->GetFunction(function); + if (maybe_function.ok()) { + options = maybe_function.ValueOrDie()->default_options(); + } + } + + KernelContext kernel_ctx{ctx}; + impl.aggregators[i]->Init(&kernel_ctx, options, aggregands[i].type()); + if (kernel_ctx.HasError()) return kernel_ctx.status(); + + out_fields.push_back(field("", impl.aggregators[i]->out_type())); + } + + for (const auto& key : keys) { + const auto& key_type = key.type(); + switch (key_type->id()) { + // Supported types of keys + case Type::BOOL: + case Type::UINT8: + case Type::INT8: + case Type::UINT16: + case Type::INT16: + case Type::UINT32: + case Type::INT32: + case Type::UINT64: + case Type::INT64: + case Type::STRING: + case Type::BINARY: + case Type::FIXED_SIZE_BINARY: + break; + default: + return Status::NotImplemented("Key of type", key_type->ToString()); + } + out_fields.push_back(field("", key_type)); + } + + impl.add_length_impl.resize(keys.size()); + impl.encode_next_impl.resize(keys.size()); + impl.decode_next_impl.resize(keys.size()); + for (size_t i = 0; i < keys.size(); ++i) { + const auto& key_type = keys[i].type(); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.add_length_impl[i])); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.encode_next_impl[i])); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.decode_next_impl[i])); + } + + impl.out_type = struct_(std::move(out_fields)); + return impl; +} + +} // namespace aggregate + +Result GroupBy(const std::vector& aggregands, + const std::vector& keys, const GroupByOptions& options, + ExecContext* ctx) { + if (ctx == nullptr) { + ExecContext default_ctx; + return GroupBy(aggregands, keys, options, &default_ctx); + } + + std::vector args = aggregands; + for (const Datum& key : keys) { + args.push_back(key); + } + + ARROW_ASSIGN_OR_RAISE(auto impl, + aggregate::GroupByInit(ctx, aggregands, keys, options)); + + ARROW_ASSIGN_OR_RAISE(auto batch_iterator, + detail::ExecBatchIterator::Make(args, ctx->exec_chunksize())); + + KernelContext kernel_ctx{ctx}; + + ExecBatch batch; + while (batch_iterator->Next(&batch)) { + if (batch.length > 0) { + impl.Consume(&kernel_ctx, batch); + if (kernel_ctx.HasError()) return kernel_ctx.status(); + } + } + + Datum out; + impl.Finalize(&kernel_ctx, &out); + if (kernel_ctx.HasError()) return kernel_ctx.status(); + return out; +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h new file mode 100644 index 00000000000..84591e2de94 --- /dev/null +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include + +#include "arrow/compute/api_aggregate.h" +#include "arrow/compute/kernels/aggregate_basic_internal.h" +#include "arrow/compute/kernels/aggregate_internal.h" +#include "arrow/compute/kernels/common.h" +#include "arrow/util/bit_run_reader.h" +#include "arrow/util/bitmap_ops.h" +#include "arrow/util/cpu_info.h" +#include "arrow/util/make_unique.h" + +namespace arrow { +namespace compute { + +/// \brief Configure a grouped aggregation +struct ARROW_EXPORT GroupByOptions : public FunctionOptions { + struct Aggregate { + /// the name of the aggregation function + std::string function; + + /// options for the aggregation function + const FunctionOptions* options; + }; + + GroupByOptions() = default; + + GroupByOptions(std::initializer_list aggregates) : aggregates(aggregates) {} + + explicit GroupByOptions(std::vector aggregates) + : aggregates(std::move(aggregates)) {} + + std::vector aggregates; +}; + +ARROW_EXPORT +Result GroupBy(const std::vector& aggregands, + const std::vector& keys, const GroupByOptions& options, + ExecContext* ctx = nullptr); + +struct GroupedAggregator { + virtual ~GroupedAggregator() = default; + + virtual void Init(KernelContext*, const FunctionOptions*, + const std::shared_ptr&) = 0; + + virtual void Consume(KernelContext*, const Datum& aggregand, + const uint32_t* group_ids) = 0; + + virtual void Finalize(KernelContext* ctx, Datum* out) = 0; + + virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; + + virtual int64_t num_groups() const = 0; + + void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { + if (length == 0) return; + + // maybe a batch of group_ids should include the min/max group id + int64_t max_group = *std::max_element(group_ids, group_ids + length); + auto old_size = num_groups(); + + if (max_group >= old_size) { + auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); + Resize(ctx, new_size); + } + } + + virtual std::shared_ptr out_type() const = 0; +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc new file mode 100644 index 00000000000..73e0d2a86c1 --- /dev/null +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -0,0 +1,558 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include +#include +#include +#include +#include + +#include + +#include "arrow/array.h" +#include "arrow/chunked_array.h" +#include "arrow/compute/api_aggregate.h" +#include "arrow/compute/api_scalar.h" +#include "arrow/compute/api_vector.h" +#include "arrow/compute/cast.h" +#include "arrow/compute/kernels/aggregate_internal.h" +#include "arrow/compute/kernels/hash_aggregate_internal.h" +#include "arrow/compute/kernels/test_util.h" +#include "arrow/compute/registry.h" +#include "arrow/type.h" +#include "arrow/type_traits.h" +#include "arrow/util/bitmap_reader.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/int_util_internal.h" + +#include "arrow/testing/gtest_common.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/random.h" +#include "arrow/util/logging.h" + +namespace arrow { + +using internal::BitmapReader; +using internal::checked_cast; +using internal::checked_pointer_cast; + +namespace compute { + +// Copy-pasta from partition.cc +// +// In the finished product this will only be a test helper for group_by +// and partition.cc will rely on a no-aggregate call to group_by. +namespace group_helpers { +namespace { + +// Transform an array of counts to offsets which will divide a ListArray +// into an equal number of slices with corresponding lengths. +Result> CountsToOffsets(std::shared_ptr counts) { + TypedBufferBuilder offset_builder; + RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); + + int32_t current_offset = 0; + offset_builder.UnsafeAppend(current_offset); + + for (int64_t i = 0; i < counts->length(); ++i) { + DCHECK_NE(counts->Value(i), 0); + current_offset += static_cast(counts->Value(i)); + offset_builder.UnsafeAppend(current_offset); + } + + std::shared_ptr offsets; + RETURN_NOT_OK(offset_builder.Finish(&offsets)); + return offsets; +} + +class StructDictionary { + public: + struct Encoded { + std::shared_ptr indices; + std::shared_ptr dictionary; + }; + + static Result Encode(const ArrayVector& columns) { + Encoded out{nullptr, std::make_shared()}; + + for (const auto& column : columns) { + if (column->null_count() != 0) { + return Status::NotImplemented("Grouping on a field with nulls"); + } + + RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices)); + } + + return out; + } + + Result> Decode(std::shared_ptr fused_indices, + FieldVector fields) { + std::vector builders(dictionaries_.size()); + for (Int32Builder& b : builders) { + RETURN_NOT_OK(b.Resize(fused_indices->length())); + } + + std::vector codes(dictionaries_.size()); + for (int64_t i = 0; i < fused_indices->length(); ++i) { + Expand(fused_indices->Value(i), codes.data()); + + auto builder_it = builders.begin(); + for (int32_t index : codes) { + builder_it++->UnsafeAppend(index); + } + } + + ArrayVector columns(dictionaries_.size()); + for (size_t i = 0; i < dictionaries_.size(); ++i) { + std::shared_ptr indices; + RETURN_NOT_OK(builders[i].FinishInternal(&indices)); + + ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices)); + + if (fields[i]->type()->id() == Type::DICTIONARY) { + RETURN_NOT_OK(RestoreDictionaryEncoding( + checked_pointer_cast(fields[i]->type()), &column)); + } + + columns[i] = column.make_array(); + } + + return StructArray::Make(std::move(columns), std::move(fields)); + } + + private: + Status AddOne(Datum column, std::shared_ptr* fused_indices) { + if (column.type()->id() != Type::DICTIONARY) { + ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(std::move(column))); + } + + auto dict_column = column.array_as(); + dictionaries_.push_back(dict_column->dictionary()); + ARROW_ASSIGN_OR_RAISE(auto indices, compute::Cast(*dict_column->indices(), int32())); + + if (*fused_indices == nullptr) { + *fused_indices = checked_pointer_cast(std::move(indices)); + return IncreaseSize(); + } + + // It's useful to think about the case where each of dictionaries_ has size 10. + // In this case the decimal digit in the ones place is the code in dictionaries_[0], + // the tens place corresponds to the code in dictionaries_[1], etc. + // The incumbent indices must be shifted to the hundreds place so as not to collide. + ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices, + compute::Multiply(indices, MakeScalar(size_))); + + ARROW_ASSIGN_OR_RAISE(new_fused_indices, + compute::Add(new_fused_indices, *fused_indices)); + + *fused_indices = checked_pointer_cast(new_fused_indices.make_array()); + return IncreaseSize(); + } + + // expand a fused code into component dict codes, order is in order of addition + void Expand(int32_t fused_code, int32_t* codes) { + for (size_t i = 0; i < dictionaries_.size(); ++i) { + auto dictionary_size = static_cast(dictionaries_[i]->length()); + codes[i] = fused_code % dictionary_size; + fused_code /= dictionary_size; + } + } + + Status RestoreDictionaryEncoding(std::shared_ptr expected_type, + Datum* column) { + DCHECK_NE(column->type()->id(), Type::DICTIONARY); + ARROW_ASSIGN_OR_RAISE(*column, compute::DictionaryEncode(std::move(*column))); + + if (expected_type->index_type()->id() == Type::INT32) { + // dictionary_encode has already yielded the expected index_type + return Status::OK(); + } + + // cast the indices to the expected index type + auto dictionary = std::move(column->mutable_array()->dictionary); + column->mutable_array()->type = int32(); + + ARROW_ASSIGN_OR_RAISE(*column, + compute::Cast(std::move(*column), expected_type->index_type())); + + column->mutable_array()->dictionary = std::move(dictionary); + column->mutable_array()->type = expected_type; + return Status::OK(); + } + + Status IncreaseSize() { + auto factor = static_cast(dictionaries_.back()->length()); + + if (arrow::internal::MultiplyWithOverflow(size_, factor, &size_)) { + return Status::CapacityError("Max groups exceeded"); + } + return Status::OK(); + } + + int32_t size_ = 1; + ArrayVector dictionaries_; +}; + +Result> MakeGroupings(const StructArray& keys) { + if (keys.num_fields() == 0) { + return Status::Invalid("Grouping with no keys"); + } + + if (keys.null_count() != 0) { + return Status::Invalid("Grouping with null keys"); + } + + ARROW_ASSIGN_OR_RAISE(auto fused, StructDictionary::Encode(keys.fields())); + + ARROW_ASSIGN_OR_RAISE(auto sort_indices, compute::SortIndices(*fused.indices)); + ARROW_ASSIGN_OR_RAISE(Datum sorted, compute::Take(fused.indices, *sort_indices)); + fused.indices = checked_pointer_cast(sorted.make_array()); + + ARROW_ASSIGN_OR_RAISE(auto fused_counts_and_values, + compute::ValueCounts(fused.indices)); + fused.indices.reset(); + + auto unique_fused_indices = + checked_pointer_cast(fused_counts_and_values->GetFieldByName("values")); + ARROW_ASSIGN_OR_RAISE( + auto unique_rows, + fused.dictionary->Decode(std::move(unique_fused_indices), keys.type()->fields())); + + auto counts = + checked_pointer_cast(fused_counts_and_values->GetFieldByName("counts")); + ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); + + auto grouped_sort_indices = + std::make_shared(list(sort_indices->type()), unique_rows->length(), + std::move(offsets), std::move(sort_indices)); + + return StructArray::Make( + ArrayVector{std::move(unique_rows), std::move(grouped_sort_indices)}, + std::vector{"values", "groupings"}); +} + +Result> ApplyGroupings(const ListArray& groupings, + const Array& array) { + ARROW_ASSIGN_OR_RAISE(Datum sorted, + compute::Take(array, groupings.data()->child_data[0])); + + return std::make_shared(list(array.type()), groupings.length(), + groupings.value_offsets(), sorted.make_array()); +} + +struct ScalarVectorToArray { + template ::BuilderType, + typename ScalarType = typename TypeTraits::ScalarType> + Status UseBuilder(const AppendScalar& append) { + BuilderType builder(type(), default_memory_pool()); + for (const auto& s : scalars_) { + if (s->is_valid) { + RETURN_NOT_OK(append(checked_cast(*s), &builder)); + } else { + RETURN_NOT_OK(builder.AppendNull()); + } + } + return builder.FinishInternal(&data_); + } + + struct AppendValue { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + return builder->Append(s.value); + } + }; + + struct AppendBuffer { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + const Buffer& buffer = *s.value; + return builder->Append(util::string_view{buffer}); + } + }; + + template + enable_if_primitive_ctype Visit(const T&) { + return UseBuilder(AppendValue{}); + } + + template + enable_if_has_string_view Visit(const T&) { + return UseBuilder(AppendBuffer{}); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("ScalarVectorToArray for type ", type); + } + + Result Convert(ScalarVector scalars) && { + if (scalars.size() == 0) { + return Status::NotImplemented("ScalarVectorToArray with no scalars"); + } + scalars_ = std::move(scalars); + RETURN_NOT_OK(VisitTypeInline(*type(), this)); + return Datum(std::move(data_)); + } + + const std::shared_ptr& type() { return scalars_[0]->type; } + + ScalarVector scalars_; + std::shared_ptr data_; +}; + +Result NaiveGroupBy(std::vector aggregands, std::vector keys, + GroupByOptions options) { + ArrayVector keys_arrays; + for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); + std::vector key_names(keys_arrays.size(), ""); + ARROW_ASSIGN_OR_RAISE(auto keys_struct, + StructArray::Make(std::move(keys_arrays), std::move(key_names))); + + ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); + + auto groupings = + checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); + + int64_t n_groups = groupings->length(); + + ArrayVector out_columns; + + for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { + const Datum& aggregand = aggregands[i_agg]; + const std::string& function = options.aggregates[i_agg].function; + + ScalarVector aggregated_scalars; + + ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, + ApplyGroupings(*groupings, *aggregand.make_array())); + + for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ARROW_ASSIGN_OR_RAISE( + Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); + aggregated_scalars.push_back(d.scalar()); + } + + ARROW_ASSIGN_OR_RAISE(Datum aggregated_column, + ScalarVectorToArray{}.Convert(std::move(aggregated_scalars))); + out_columns.push_back(aggregated_column.make_array()); + } + + keys_struct = + checked_pointer_cast(groupings_and_values->GetFieldByName("values")); + for (size_t i_key = 0; i_key < aggregands.size(); ++i_key) { + out_columns.push_back(keys_struct->field(i_key)); + } + + std::vector out_names(out_columns.size(), ""); + return StructArray::Make(std::move(out_columns), std::move(out_names)); +} + +void ValidateGroupBy(GroupByOptions options, std::vector aggregands, + std::vector keys) { + ASSERT_OK_AND_ASSIGN(Datum expected, + group_helpers::NaiveGroupBy(aggregands, keys, options)); + + ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, options)); + + // Ordering of groups is not important, so sort by key columns to ensure the comparison + // doesn't fail spuriously + + for (Datum* out : {&expected, &actual}) { + auto out_columns = out->array_as()->fields(); + + SortOptions sort_options; + FieldVector key_fields; + ArrayVector key_columns; + for (size_t i = 0; i < keys.size(); ++i) { + auto name = std::to_string(i); + sort_options.sort_keys.emplace_back(name); + key_fields.push_back(field(name, out_columns[0]->type())); + key_columns.push_back(out_columns[0]); + } + auto key_batch = RecordBatch::Make(schema(std::move(key_fields)), out->length(), + std::move(key_columns)); + + ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(key_batch, sort_options)); + ASSERT_OK_AND_ASSIGN(*out, Take(*out, sort_indices, TakeOptions::NoBoundsCheck())); + } + + AssertDatumsEqual(expected, actual, /*verbose=*/true); +} + +} // namespace +} // namespace group_helpers + +TEST(GroupBy, SumOnly) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", int64()), + }), + R"([ + [4.25, 1], + [-0.125, 2], + [null, 3], + [0.75, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, MinMaxOnly) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"min_max", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("", int64()), + }), + R"([ + [{"min": 1.0, "max": 3.25}, 1], + [{"min": -0.25, "max": 0.125}, 2], + [{"min": null, "max": null}, 3], + [{"min": 0.75, "max": 0.75}, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, CountAndSum) { + auto aggregand = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); + + CountOptions count_options; + + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + // NB: passing an aggregand twice or also using it as a key is legal + GroupBy({aggregand, aggregand, key}, {key}, + GroupByOptions{ + {"count", &count_options}, + {"sum", nullptr}, + {"sum", nullptr}, + })); + + AssertDatumsEqual( + ArrayFromJSON(struct_({ + field("", int64()), + // NB: summing a float32 array results in float64 sums + field("", float64()), + field("", int64()), + field("", int64()), + }), + R"([ + [1, 1.0, 2, 1], + [2, 0.125, 4, 2], + [2, 3.0, 6, 3], + [1, 0.75, null, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, StringKey) { + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); + auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + GroupBy({aggregand}, {key}, GroupByOptions{{"sum", nullptr}})); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", int64()), + field("", utf8()), + }), + R"([ + [10, "alfa"], + [14, "beta"], + [6, "gamma"], + [12, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, MultipleKeys) { + auto aggregand = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); + auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); + auto str_key = + ArrayFromJSON(utf8(), R"(["beta", "beta", "gamma", "gamma", null, "beta"])"); + + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + GroupBy({aggregand}, {int_key, str_key}, GroupByOptions{{"sum", nullptr}})); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", int32()), + field("", utf8()), + }), + R"([ + [0.125, 0, "beta"], + [2.5, 1, "beta"], + [-0.75, 0, "gamma"], + [8, 1, "gamma"], + [1.0, 0, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); + auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); + + group_helpers::ValidateGroupBy(GroupByOptions{{"sum", nullptr}}, {aggregand}, {key}); +} + +TEST(GroupBy, RandomArraySum) { + auto rand = random::RandomArrayGenerator(0xdeadbeef); + + for (size_t i = 3; i < 14; i += 2) { + for (auto null_probability : {0.0, 0.001, 0.1, 0.5, 0.999, 1.0}) { + int64_t length = 1UL << i; + auto summand = rand.Float32(length, -100, 100, null_probability); + auto key = rand.Int64(length, 0, 12); + + group_helpers::ValidateGroupBy( + GroupByOptions{ + {"sum", nullptr}, + }, + {summand}, {key}); + } + } +} + +} // namespace compute +} // namespace arrow + diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst index 3878aa928f3..8b4f2219989 100644 --- a/docs/source/cpp/compute.rst +++ b/docs/source/cpp/compute.rst @@ -90,16 +90,16 @@ exact semantics of the function:: min_max_options.null_handling = MinMaxOptions::EMIT_NULL; std::shared_ptr array = ...; - arrow::Datum min_max_datum; + arrow::Datum min_max; ARROW_ASSIGN_OR_RAISE(min_max, arrow::compute::CallFunction("min_max", {array}, &min_max_options)); // Unpack struct scalar result (a two-field {"min", "max"} scalar) - const auto& min_max_scalar = min_max.scalar_as(); - std::shared_ptr min_value = min_max_scalar.value[0]; - std::shared_ptr max_value = min_max_scalar.value[1]; + std::shared_ptr min_value, max_value; + min_value = min_max.scalar_as().value[0]; + max_value = min_max.scalar_as().value[1]; .. seealso:: :doc:`Compute API reference ` @@ -212,8 +212,6 @@ Aggregations +--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ | variance | Unary | Numeric | Scalar Float64 | :struct:`VarianceOptions` | +--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ -| group_by | VarArgs | Any | Array Struct | :struct:`GroupByOptions` | -+--------------------------+------------+--------------------+-----------------------+--------------------------------------------+ Notes: @@ -230,49 +228,6 @@ Notes: * \(4) Output is Int64, UInt64 or Float64, depending on the input type. -Grouped Aggregations -~~~~~~~~~~~~~~~~~~~~ - -Aggregations can be grouped by one or more keys using the ``group_by`` -function. :member:`GroupByOptions::aggregates` is a vector specifying which -aggregations will be performed: each element is a -:struct:`GroupByOptions::Aggregate` containing the name of an aggregate -function and a pointer to a :class:`FunctionOptions`. The first arguments to -``group_by`` are interpreted as the corresponding aggregands and the remainder -will be used as grouping keys. The output will be an array with the same -number of fields where each slot contains the aggregation result and keys -for a group:: - - GroupByOptions options{ - {"sum", nullptr}, // first argument will be summed - {"min_max", - &min_max_options}, // second argument's extrema will be found - }; - - std::shared_ptr needs_sum = ...; - std::shared_ptr needs_min_max = ...; - std::shared_ptr key_0 = ...; - std::shared_ptr key_1 = ...; - - ARROW_ASSIGN_OR_RAISE(arrow::Datum out, - arrow::compute::CallFunction("group_by", - { - needs_sum, - needs_min_max, - key_0, - key_1, - }, - &options)); - - // Unpack struct array result (a four-field array) - auto out_array = out.array_as(); - std::shared_ptr sums = out_array->field(0); - std::shared_ptr mins_and_maxes = out_array->field(1); - std::shared_ptr group_key_0 = out_array->field(2); - std::shared_ptr group_key_1 = out_array->field(3); - -The output fields' names are empty. - Element-wise ("scalar") functions --------------------------------- From d4e3f11e11a4cb119063f3d50e4c04900484745b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 16 Mar 2021 08:43:45 -0400 Subject: [PATCH 20/49] add HashAggregateKernel --- cpp/src/arrow/compute/function.h | 4 + cpp/src/arrow/compute/kernel.h | 61 +- .../arrow/compute/kernels/hash_aggregate.cc | 1832 +++++++++-------- .../compute/kernels/hash_aggregate_internal.h | 35 +- 4 files changed, 1021 insertions(+), 911 deletions(-) diff --git a/cpp/src/arrow/compute/function.h b/cpp/src/arrow/compute/function.h index af5d81a30ec..52faa65199f 100644 --- a/cpp/src/arrow/compute/function.h +++ b/cpp/src/arrow/compute/function.h @@ -133,6 +133,10 @@ class ARROW_EXPORT Function { /// A function that computes scalar summary statistics from array input. SCALAR_AGGREGATE, + /// A function that computes grouped summary statistics from array input + /// and an array of group identifiers. + HASH_AGGREGATE, + /// A function that dispatches to other functions and does not contain its /// own kernels. META diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index c8f9cacfb34..0733bbaa597 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -537,7 +537,8 @@ struct Kernel { : signature(std::move(sig)), init(std::move(init)) {} Kernel(std::vector in_types, OutputType out_type, KernelInit init) - : Kernel(KernelSignature::Make(std::move(in_types), out_type), std::move(init)) {} + : Kernel(KernelSignature::Make(std::move(in_types), std::move(out_type)), + std::move(init)) {} /// \brief The "signature" of the kernel containing the InputType input /// argument validators and OutputType output type and shape resolver. @@ -574,7 +575,8 @@ struct ArrayKernel : public Kernel { ArrayKernel(std::vector in_types, OutputType out_type, ArrayKernelExec exec, KernelInit init = NULLPTR) - : Kernel(std::move(in_types), std::move(out_type), init), exec(std::move(exec)) {} + : Kernel(std::move(in_types), std::move(out_type), std::move(init)), + exec(std::move(exec)) {} /// \brief Perform a single invocation of this kernel. Depending on the /// implementation, it may only write into preallocated memory, while in some @@ -617,7 +619,7 @@ struct VectorKernel : public ArrayKernel { VectorKernel() = default; VectorKernel(std::shared_ptr sig, ArrayKernelExec exec) - : ArrayKernel(std::move(sig), exec) {} + : ArrayKernel(std::move(sig), std::move(exec)) {} VectorKernel(std::vector in_types, OutputType out_type, ArrayKernelExec exec, KernelInit init = NULLPTR, VectorFinalize finalize = NULLPTR) @@ -680,12 +682,12 @@ using ScalarAggregateFinalize = std::function; /// * finalize: produces the end result of the aggregation using the /// KernelState in the KernelContext. struct ScalarAggregateKernel : public Kernel { - ScalarAggregateKernel() {} + ScalarAggregateKernel() = default; ScalarAggregateKernel(std::shared_ptr sig, KernelInit init, ScalarAggregateConsume consume, ScalarAggregateMerge merge, ScalarAggregateFinalize finalize) - : Kernel(std::move(sig), init), + : Kernel(std::move(sig), std::move(init)), consume(std::move(consume)), merge(std::move(merge)), finalize(std::move(finalize)) {} @@ -693,13 +695,58 @@ struct ScalarAggregateKernel : public Kernel { ScalarAggregateKernel(std::vector in_types, OutputType out_type, KernelInit init, ScalarAggregateConsume consume, ScalarAggregateMerge merge, ScalarAggregateFinalize finalize) - : ScalarAggregateKernel(KernelSignature::Make(std::move(in_types), out_type), init, - consume, merge, finalize) {} + : ScalarAggregateKernel( + KernelSignature::Make(std::move(in_types), std::move(out_type)), + std::move(init), std::move(consume), std::move(merge), std::move(finalize)) {} ScalarAggregateConsume consume; ScalarAggregateMerge merge; ScalarAggregateFinalize finalize; }; +// ---------------------------------------------------------------------- +// HashAggregateKernel (for HashAggregateFunction) + +using HashAggregateConsume = std::function; + +using HashAggregateMerge = + std::function; + +// Finalize returns Datum to permit multiple return values +using HashAggregateFinalize = std::function; + +/// \brief Kernel data structure for implementations of +/// HashAggregateFunction. The four necessary components of an aggregation +/// kernel are the init, consume, merge, and finalize functions. +/// +/// * init: creates a new KernelState for a kernel. +/// * consume: processes an ExecBatch (which includes the aggregand as well +/// as an array of group identifiers) and updates the KernelState found in the +/// KernelContext. +/// * finalize: produces the end result of the aggregation using the +/// KernelState in the KernelContext. +struct HashAggregateKernel : public Kernel { + HashAggregateKernel() = default; + + HashAggregateKernel(std::shared_ptr sig, KernelInit init, + HashAggregateConsume consume, HashAggregateMerge merge, + HashAggregateFinalize finalize) + : Kernel(std::move(sig), std::move(init)), + consume(std::move(consume)), + merge(std::move(merge)), + finalize(std::move(finalize)) {} + + HashAggregateKernel(std::vector in_types, OutputType out_type, + KernelInit init, HashAggregateMerge merge, + HashAggregateConsume consume, HashAggregateFinalize finalize) + : HashAggregateKernel( + KernelSignature::Make(std::move(in_types), std::move(out_type)), + std::move(init), std::move(consume), std::move(merge), std::move(finalize)) {} + + HashAggregateConsume consume; + HashAggregateMerge merge; + HashAggregateFinalize finalize; +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 199afeed215..a4558781c98 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -33,1045 +33,1133 @@ namespace arrow { namespace compute { namespace aggregate { -// ---------------------------------------------------------------------- -// Count implementation - -struct GroupedCountImpl : public GroupedAggregator { - void Init(KernelContext* ctx, const FunctionOptions* options, - const std::shared_ptr&) override { - options_ = checked_cast(*options); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); - } +struct GroupByImpl { + using AddLengthImpl = std::function&, int32_t*)>; - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups(); - KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); - auto new_size = num_groups(); + struct GetAddLengthImpl { + static constexpr int32_t null_extra_byte = 1; - auto raw_counts = reinterpret_cast(counts_->mutable_data()); - for (auto i = old_size; i < new_size; ++i) { - raw_counts[i] = 0; + static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, + int32_t* lengths) { + for (int64_t i = 0; i < num_repeats; ++i) { + lengths[i] += fixed_length + null_extra_byte; + } } - } - - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - - auto raw_counts = reinterpret_cast(counts_->mutable_data()); - const auto& input = aggregand.array(); + static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { + using offset_type = typename StringType::offset_type; + constexpr int32_t length_extra_bytes = sizeof(offset_type); + auto offset = data->offset; + const auto offsets = data->GetValues(1); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); - if (options_.count_mode == CountOptions::COUNT_NULL) { - for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { - auto g = group_ids[i]; - raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); + for (int64_t i = 0; i < data->length; ++i) { + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + lengths[i] += null_extra_byte + length_extra_bytes; + } else { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - + offsets[offset + i]; + } } - return; } - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_counts[g] += 1; - } - }); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - auto length = num_groups(); - *out = std::make_shared(length, std::move(counts_)); - } - - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - - std::shared_ptr out_type() const override { return int64(); } - - CountOptions options_; - std::shared_ptr counts_; -}; - -// ---------------------------------------------------------------------- -// Sum implementation - -struct GroupedSumImpl : public GroupedAggregator { - // NB: whether we are accumulating into double, int64_t, or uint64_t - // we always have 64 bits per group in the sums buffer. - static constexpr size_t kSumSize = sizeof(int64_t); - - using ConsumeImpl = std::function&, - const uint32_t*, Buffer*, Buffer*)>; - - struct GetConsumeImpl { - template ::Type> - Status Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = reinterpret_cast::CType*>( - input->buffers[1]->data()); - auto raw_sums = reinterpret_cast::CType*>( - sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_sums[g] += raw_input[input_i]; - raw_counts[g] += 1; - } - }); + template + Status Visit(const T& input_type) { + int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); }; - out_type = TypeTraits::type_singleton(); return Status::OK(); } - Status Visit(const BooleanType&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = input->buffers[1]->data(); - auto raw_sums = reinterpret_cast(sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i) { - auto g = group_ids[i]; - raw_sums[g] += BitUtil::GetBit(raw_input, input_i); - raw_counts[g] += 1; - } - }); + Status Visit(const StringType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); }; - out_type = uint64(); return Status::OK(); } - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Summing data of type ", type); + Status Visit(const BinaryType&) { + add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { + AddVarLength(data, lengths); + }; + return Status::OK(); } - Status Visit(const DataType& type) { - return Status::NotImplemented("Summing data of type ", type); + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + add_length_impl = [num_bytes](const std::shared_ptr& data, + int32_t* lengths) { + AddFixedLength(num_bytes, data->length, lengths); + }; + return Status::OK(); } - ConsumeImpl consume_impl; - std::shared_ptr out_type; + AddLengthImpl add_length_impl; }; - void Init(KernelContext* ctx, const FunctionOptions*, - const std::shared_ptr& input_type) override { - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&sums_)); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); - - GetConsumeImpl get_consume_impl; - KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_consume_impl)); + using EncodeNextImpl = + std::function&, uint8_t**)>; - consume_impl_ = std::move(get_consume_impl.consume_impl); - out_type_ = std::move(get_consume_impl.out_type); - } + struct GetEncodeNextImpl { + template + static void EncodeSmallFixed(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + uint64_t null_multiplier = is_null ? 0 : 1; + if (NumBits == 1) { + encoded_ptr[0] = static_cast( + null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = + static_cast(null_multiplier * reinterpret_cast( + raw_input)[offset + i]); + encoded_ptr += 8; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + if (NumBits == 1) { + encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(encoded_ptr)[0] = + reinterpret_cast(raw_input)[offset + i]; + encoded_ptr += 8; + } + } + } + } - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups() * kSumSize; - KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); - KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); - auto new_size = num_groups() * kSumSize; - std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); - std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); - } - - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); - } - - void Finalize(KernelContext* ctx, Datum* out) override { - std::shared_ptr null_bitmap; - int64_t null_count = 0; - - for (int64_t i = 0; i < num_groups(); ++i) { - if (reinterpret_cast(counts_->data())[i] > 0) continue; - - if (null_bitmap == nullptr) { - KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); - BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); + static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, + uint8_t** encoded_bytes) { + auto raw_input = data->buffers[1]->data(); + auto offset = data->offset; + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + encoded_ptr[0] = is_null ? 1 : 0; + encoded_ptr += 1; + if (is_null) { + memset(encoded_ptr, 0, num_bytes); + } else { + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + } + encoded_ptr += num_bytes; + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr += 1; + memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); + encoded_ptr += num_bytes; + } } - - null_count += 1; - BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); } - *out = ArrayData::Make(std::move(out_type_), num_groups(), - {std::move(null_bitmap), std::move(sums_)}, null_count); - } - - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - - std::shared_ptr out_type() const override { return out_type_; } - - // NB: counts are used here instead of a simple "has_values_" bitmap since - // we expect to reuse this kernel to handle Mean - std::shared_ptr sums_, counts_; - std::shared_ptr out_type_; - ConsumeImpl consume_impl_; -}; - -// ---------------------------------------------------------------------- -// MinMax implementation - -struct GroupedMinMaxImpl : public GroupedAggregator { - using ConsumeImpl = std::function&, - const uint32_t*, BufferVector*)>; - - using ResizeImpl = std::function; - - struct GetImpl { - template ::CType> - enable_if_number Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, BufferVector* buffers) { - auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); - auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); - - auto raw_has_nulls = buffers->at(2)->mutable_data(); - auto raw_has_values = buffers->at(3)->mutable_data(); - - auto g = group_ids; - VisitArrayDataInline( - *input, - [&](CType val) { - raw_maxes[*g] = std::max(raw_maxes[*g], val); - raw_mins[*g] = std::min(raw_mins[*g], val); - BitUtil::SetBit(raw_has_values, *g++); - }, - [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); - }; + static void EncodeVarLength(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + using offset_type = typename StringType::offset_type; + auto offset = data->offset; + const auto offsets = data->GetValues(1); + auto raw_input = data->buffers[2]->data(); + if (data->MayHaveNulls()) { + const uint8_t* nulls = data->buffers[0]->data(); + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + bool is_null = !BitUtil::GetBit(nulls, offset + i); + if (is_null) { + encoded_ptr[0] = 1; + encoded_ptr++; + reinterpret_cast(encoded_ptr)[0] = 0; + encoded_ptr += sizeof(offset_type); + } else { + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } else { + for (int64_t i = 0; i < data->length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + encoded_ptr[0] = 0; + encoded_ptr++; + size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; + reinterpret_cast(encoded_ptr)[0] = num_bytes; + encoded_ptr += sizeof(offset_type); + memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); + encoded_ptr += num_bytes; + } + } + } - GetResizeImpls(); + template + Status Visit(const T& input_type) { + int32_t num_bits = bit_width(input_type.id()); + switch (num_bits) { + case 1: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<1>(data, encoded_bytes); + }; + break; + case 8: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<8>(data, encoded_bytes); + }; + break; + case 16: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<16>(data, encoded_bytes); + }; + break; + case 32: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<32>(data, encoded_bytes); + }; + break; + case 64: + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeSmallFixed<64>(data, encoded_bytes); + }; + break; + } return Status::OK(); } - Status Visit(const BooleanType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); + Status Visit(const StringType&) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); } - Status Visit(const HalfFloatType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); + Status Visit(const BinaryType&) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeVarLength(data, encoded_bytes); + }; + return Status::OK(); } - Status Visit(const DataType& type) { - return Status::NotImplemented("Grouped MinMax data of type ", type); + Status Visit(const FixedSizeBinaryType& type) { + int32_t num_bytes = type.byte_width(); + encode_next_impl = [num_bytes](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeBigFixed(num_bytes, data, encoded_bytes); + }; + return Status::OK(); } - template - ResizeImpl MakeResizeImpl(CType anti_extreme) { - // resize a min or max buffer, storing the correct anti extreme - return [anti_extreme](Buffer* vals, int64_t new_num_groups) { - int64_t old_num_groups = vals->size() / sizeof(CType); + EncodeNextImpl encode_next_impl; + }; - int64_t new_size = new_num_groups * sizeof(CType); - RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); + using DecodeNextImpl = std::function*)>; - auto raw_vals = reinterpret_cast(vals->mutable_data()); - for (int64_t i = old_num_groups; i != new_num_groups; ++i) { - raw_vals[i] = anti_extreme; + struct GetDecodeNextImpl { + static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, + int32_t* null_count) { + // Do we have nulls? + *null_count = 0; + for (int32_t i = 0; i < length; ++i) { + *null_count += encoded_bytes[i][0]; + } + if (*null_count > 0) { + ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); + uint8_t* nulls = (*null_buf)->mutable_data(); + memset(nulls, 0, (*null_buf)->size()); + for (int32_t i = 0; i < length; ++i) { + if (!encoded_bytes[i][0]) { + BitUtil::SetBit(nulls, i); + } + encoded_bytes[i] += 1; } - return Status::OK(); - }; - } - - template ::CType> - enable_if_floating_point GetResizeImpls() { - auto inf = std::numeric_limits::infinity(); - resize_min_impl = MakeResizeImpl(inf); - resize_max_impl = MakeResizeImpl(-inf); + } else { + for (int32_t i = 0; i < length; ++i) { + encoded_bytes[i] += 1; + } + } + return Status ::OK(); } - template ::CType> - enable_if_integer GetResizeImpls() { - resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); - resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); - } + template + static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, + int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - ConsumeImpl consume_impl; - ResizeImpl resize_min_impl, resize_max_impl; - }; - - void Init(KernelContext* ctx, const FunctionOptions* options, - const std::shared_ptr& input_type) override { - options_ = *checked_cast(options); - type_ = input_type; - - buffers_.resize(4); - for (auto& buf : buffers_) { - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&buf)); - } - - GetImpl get_impl; - KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_impl)); - - consume_impl_ = std::move(get_impl.consume_impl); - resize_min_impl_ = std::move(get_impl.resize_min_impl); - resize_max_impl_ = std::move(get_impl.resize_max_impl); - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_num_groups = num_groups_; - num_groups_ = new_num_groups; + KERNEL_ASSIGN_OR_RAISE( + auto key_buf, ctx, + ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); - KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); - KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); + uint8_t* raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + if (NumBits == 1) { + BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); + encoded_ptr += 1; + } + if (NumBits == 8) { + raw_output[i] = encoded_ptr[0]; + encoded_ptr += 1; + } + if (NumBits == 16) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 2; + } + if (NumBits == 32) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 4; + } + if (NumBits == 64) { + reinterpret_cast(raw_output)[i] = + reinterpret_cast(encoded_bytes[i])[0]; + encoded_ptr += 8; + } + } - for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { - KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( - BitUtil::BytesForBits(new_num_groups))); - BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); + DCHECK(is_integer(output_type) || output_type == Type::BOOL); + *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); } - } - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, &buffers_); - } + static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, + uint8_t** encoded_bytes, std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - void Finalize(KernelContext* ctx, Datum* out) override { - // aggregation for group is valid if there was at least one value in that group - std::shared_ptr null_bitmap = std::move(buffers_[3]); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); + auto raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); + encoded_bytes[i] += num_bytes; + } - if (options_.null_handling == MinMaxOptions::EMIT_NULL) { - // ... and there were no nulls in that group - arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, - num_groups(), 0, null_bitmap->mutable_data()); + *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, + null_count); } - auto mins = - ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); - - auto maxes = ArrayData::Make(type_, num_groups(), - {std::move(null_bitmap), std::move(buffers_[1])}); - - *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, - {std::move(mins), std::move(maxes)}); - } - - int64_t num_groups() const override { return num_groups_; } - - std::shared_ptr out_type() const override { - return struct_({field("min", type_), field("max", type_)}); - } + static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; + int32_t null_count; + KERNEL_RETURN_IF_ERROR( + ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); - int64_t num_groups_; - BufferVector buffers_; - std::shared_ptr type_; - ConsumeImpl consume_impl_; - ResizeImpl resize_min_impl_, resize_max_impl_; - MinMaxOptions options_; -}; + using offset_type = typename StringType::offset_type; -struct GroupByImpl { - using AddLengthImpl = std::function&, int32_t*)>; + int32_t length_sum = 0; + for (int32_t i = 0; i < length; ++i) { + length_sum += reinterpret_cast(encoded_bytes)[0]; + } - struct GetAddLengthImpl { - static constexpr int32_t null_extra_byte = 1; + KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, + ctx->Allocate(sizeof(offset_type) * (1 + length))); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); - static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, - int32_t* lengths) { - for (int64_t i = 0; i < num_repeats; ++i) { - lengths[i] += fixed_length + null_extra_byte; + auto raw_offsets = offset_buf->mutable_data(); + auto raw_keys = key_buf->mutable_data(); + int32_t current_offset = 0; + for (int32_t i = 0; i < length; ++i) { + offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; + reinterpret_cast(raw_offsets)[i] = current_offset; + encoded_bytes[i] += sizeof(offset_type); + memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); + encoded_bytes[i] += key_length; + current_offset += key_length; } - } - - static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { - using offset_type = typename StringType::offset_type; - constexpr int32_t length_extra_bytes = sizeof(offset_type); - auto offset = data->offset; - const auto offsets = data->GetValues(1); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); + reinterpret_cast(raw_offsets)[length] = current_offset; - for (int64_t i = 0; i < data->length; ++i) { - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - lengths[i] += null_extra_byte + length_extra_bytes; - } else { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } - } + if (is_string) { + *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); } else { - for (int64_t i = 0; i < data->length; ++i) { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } + *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, + null_count, 0); } } template Status Visit(const T& input_type) { - int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); - }; + int32_t num_bits = bit_width(input_type.id()); + auto type_id = input_type.id(); + switch (num_bits) { + case 1: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 8: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 16: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 32: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); + }; + break; + case 64: + decode_next_impl = [type_id](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); + }; + break; + } return Status::OK(); } Status Visit(const StringType&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeVarLength(ctx, true, length, encoded_bytes, out); }; return Status::OK(); } Status Visit(const BinaryType&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); + decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeVarLength(ctx, false, length, encoded_bytes, out); }; return Status::OK(); } Status Visit(const FixedSizeBinaryType& type) { int32_t num_bytes = type.byte_width(); - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); + decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, + uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); }; return Status::OK(); } - AddLengthImpl add_length_impl; + DecodeNextImpl decode_next_impl; }; - using EncodeNextImpl = - std::function&, uint8_t**)>; + void Consume(KernelContext* ctx, const ExecBatch& batch) { + ArrayDataVector aggregands, keys; - struct GetEncodeNextImpl { - template - static void EncodeSmallFixed(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - uint64_t null_multiplier = is_null ? 0 : 1; - if (NumBits == 1) { - encoded_ptr[0] = static_cast( - null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 8; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - if (NumBits == 1) { - encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 8; - } - } - } + size_t i; + for (i = 0; i < aggregators.size(); ++i) { + aggregands.push_back(batch[i].array()); + } + while (i < static_cast(batch.num_values())) { + keys.push_back(batch[i++].array()); } - static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - if (is_null) { - memset(encoded_ptr, 0, num_bytes); - } else { - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - } - encoded_ptr += num_bytes; - } + offsets_batch_.clear(); + offsets_batch_.resize(batch.length + 1); + offsets_batch_[0] = 0; + memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); + for (size_t i = 0; i < keys.size(); ++i) { + add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); + } + int32_t total_length = 0; + for (int64_t i = 0; i < batch.length; ++i) { + auto total_length_before = total_length; + total_length += offsets_batch_[i]; + offsets_batch_[i] = total_length_before; + } + offsets_batch_[batch.length] = total_length; + + key_bytes_batch_.clear(); + key_bytes_batch_.resize(total_length); + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; + } + for (size_t i = 0; i < keys.size(); ++i) { + encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); + } + + group_ids_batch_.clear(); + group_ids_batch_.resize(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; + std::string key( + reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), + key_length); + auto iter = map_.find(key); + if (iter == map_.end()) { + group_ids_batch_[i] = n_groups++; + auto next_key_offset = static_cast(key_bytes_.size()); + key_bytes_.resize(next_key_offset + key_length); + offsets_.push_back(next_key_offset + key_length); + memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); + map_.insert(std::make_pair(key, group_ids_batch_[i])); } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - encoded_ptr += num_bytes; - } + group_ids_batch_[i] = iter->second; } } - static void EncodeVarLength(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - using offset_type = typename StringType::offset_type; - auto offset = data->offset; - const auto offsets = data->GetValues(1); - auto raw_input = data->buffers[2]->data(); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - encoded_ptr[0] = 1; - encoded_ptr++; - reinterpret_cast(encoded_ptr)[0] = 0; - encoded_ptr += sizeof(offset_type); - } else { - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } + UInt32Array group_ids(group_ids_batch_.size(), Buffer::Wrap(group_ids_batch_)); + for (size_t i = 0; i < aggregators.size(); ++i) { + KernelContext batch_ctx{ctx->exec_context()}; + batch_ctx.SetState(aggregator_states[i].get()); + ExecBatch batch({aggregands[i], group_ids}, group_ids.length()); + aggregators[i].consume(&batch_ctx, batch); + ctx->SetStatus(batch_ctx.status()); + if (ctx->HasError()) return; + } + } + + void Finalize(KernelContext* ctx, Datum* out) { + size_t n_keys = decode_next_impl.size(); + ArrayDataVector out_columns(aggregators.size() + n_keys); + for (size_t i = 0; i < aggregators.size(); ++i) { + KernelContext batch_ctx{ctx->exec_context()}; + batch_ctx.SetState(aggregator_states[i].get()); + Datum aggregated; + aggregators[i].finalize(&batch_ctx, &aggregated); + ctx->SetStatus(batch_ctx.status()); + if (ctx->HasError()) return; + out_columns[i] = aggregated.array(); + } + + key_buf_ptrs_.clear(); + key_buf_ptrs_.resize(n_groups); + for (int64_t i = 0; i < n_groups; ++i) { + key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; + } + + int64_t length = n_groups; + for (size_t i = 0; i < n_keys; ++i) { + std::shared_ptr key_array; + decode_next_impl[i].decode_next_impl(ctx, static_cast(length), + key_buf_ptrs_.data(), &key_array); + out_columns[aggregators.size() + i] = std::move(key_array); + } + + *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, + std::move(out_columns)); + } + + std::vector offsets_batch_; + std::vector key_bytes_batch_; + std::vector key_buf_ptrs_; + std::vector group_ids_batch_; + + std::unordered_map map_; + std::vector offsets_; + std::vector key_bytes_; + uint32_t n_groups; + + std::shared_ptr out_type; + GroupByOptions options; + std::vector aggregators; + std::vector> aggregator_states; + + std::vector add_length_impl; + std::vector encode_next_impl; + std::vector decode_next_impl; +}; + +Result MakeKernel(GroupByOptions::Aggregate); + +Result GroupByInit(ExecContext* ctx, const std::vector& aggregands, + const std::vector& keys, + const GroupByOptions& options) { + GroupByImpl impl; + impl.options = options; + const auto& aggregates = impl.options.aggregates; + + impl.n_groups = 0; + impl.offsets_.push_back(0); + + if (aggregates.size() != aggregands.size()) { + return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", + aggregands.size(), " aggregands were provided."); + } + + FieldVector out_fields; + + impl.aggregators.resize(aggregates.size()); + impl.aggregator_states.resize(aggregates.size()); + + for (size_t i = 0; i < aggregates.size(); ++i) { + auto a = aggregates[i]; + + if (a.options == nullptr) { + // use known default options for the named function if possible + auto maybe_function = ctx->func_registry()->GetFunction(a.function); + if (maybe_function.ok()) { + a.options = maybe_function.ValueOrDie()->default_options(); } } - template - Status Visit(const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - switch (num_bits) { - case 1: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<1>(data, encoded_bytes); - }; - break; - case 8: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<8>(data, encoded_bytes); - }; - break; - case 16: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<16>(data, encoded_bytes); - }; - break; - case 32: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<32>(data, encoded_bytes); - }; - break; - case 64: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<64>(data, encoded_bytes); - }; - break; + ARROW_ASSIGN_OR_RAISE(impl.aggregators[i], MakeKernel(a)); + + KernelContext kernel_ctx{ctx}; + impl.aggregator_states[i] = impl.aggregators[i].init( + &kernel_ctx, + KernelInitArgs{&impl.aggregators[i], {aggregands[i].type()}, a.options}); + if (kernel_ctx.HasError()) return kernel_ctx.status(); + + kernel_ctx.SetState(impl.aggregator_states[i].get()); + ARROW_ASSIGN_OR_RAISE(auto descr, impl.aggregators[i].signature->out_type().Resolve( + &kernel_ctx, { + aggregands[i].type(), + uint32(), + })); + out_fields.push_back(field("", descr.type)); + } + + for (const auto& key : keys) { + const auto& key_type = key.type(); + switch (key_type->id()) { + // Supported types of keys + case Type::BOOL: + case Type::UINT8: + case Type::INT8: + case Type::UINT16: + case Type::INT16: + case Type::UINT32: + case Type::INT32: + case Type::UINT64: + case Type::INT64: + case Type::STRING: + case Type::BINARY: + case Type::FIXED_SIZE_BINARY: + break; + default: + return Status::NotImplemented("Key of type", key_type->ToString()); + } + out_fields.push_back(field("", key_type)); + } + + impl.add_length_impl.resize(keys.size()); + impl.encode_next_impl.resize(keys.size()); + impl.decode_next_impl.resize(keys.size()); + for (size_t i = 0; i < keys.size(); ++i) { + const auto& key_type = keys[i].type(); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.add_length_impl[i])); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.encode_next_impl[i])); + RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.decode_next_impl[i])); + } + + impl.out_type = struct_(std::move(out_fields)); + return impl; +} + +/// C++ abstract base class for the HashAggregateKernel interface. +/// Implementations should be default constructible and perform initialization in +/// Init(). +struct GroupedAggregator : KernelState { + virtual void Init(KernelContext*, const FunctionOptions*, + const std::shared_ptr&) = 0; + + virtual void Consume(KernelContext*, const Datum& aggregand, + const uint32_t* group_ids) = 0; + + virtual void Finalize(KernelContext* ctx, Datum* out) = 0; + + virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; + + virtual int64_t num_groups() const = 0; + + void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { + if (length == 0) return; + + // maybe a batch of group_ids should include the min/max group id + int64_t max_group = *std::max_element(group_ids, group_ids + length); + auto old_size = num_groups(); + + if (max_group >= old_size) { + auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); + Resize(ctx, new_size); + } + } + + virtual std::shared_ptr out_type() const = 0; +}; + +// ---------------------------------------------------------------------- +// Count implementation + +struct GroupedCountImpl : public GroupedAggregator { + void Init(KernelContext* ctx, const FunctionOptions* options, + const std::shared_ptr&) override { + options_ = checked_cast(*options); + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); + } + + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups(); + KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); + auto new_size = num_groups(); + + auto raw_counts = reinterpret_cast(counts_->mutable_data()); + for (auto i = old_size; i < new_size; ++i) { + raw_counts[i] = 0; + } + } + + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + + auto raw_counts = reinterpret_cast(counts_->mutable_data()); + + const auto& input = aggregand.array(); + + if (options_.count_mode == CountOptions::COUNT_NULL) { + for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { + auto g = group_ids[i]; + raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); } + return; + } + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_counts[g] += 1; + } + }); + } + + void Finalize(KernelContext* ctx, Datum* out) override { + auto length = num_groups(); + *out = std::make_shared(length, std::move(counts_)); + } + + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } + + std::shared_ptr out_type() const override { return int64(); } + + CountOptions options_; + std::shared_ptr counts_; +}; + +// ---------------------------------------------------------------------- +// Sum implementation + +struct GroupedSumImpl : public GroupedAggregator { + // NB: whether we are accumulating into double, int64_t, or uint64_t + // we always have 64 bits per group in the sums buffer. + static constexpr size_t kSumSize = sizeof(int64_t); + + using ConsumeImpl = std::function&, + const uint32_t*, Buffer*, Buffer*)>; + + struct GetConsumeImpl { + template ::Type> + Status Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { + auto raw_input = reinterpret_cast::CType*>( + input->buffers[1]->data()); + auto raw_sums = reinterpret_cast::CType*>( + sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i, ++i) { + auto g = group_ids[i]; + raw_sums[g] += raw_input[input_i]; + raw_counts[g] += 1; + } + }); + }; + out_type = TypeTraits::type_singleton(); return Status::OK(); } - Status Visit(const StringType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); + Status Visit(const BooleanType&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, Buffer* sums, Buffer* counts) { + auto raw_input = input->buffers[1]->data(); + auto raw_sums = reinterpret_cast(sums->mutable_data()); + auto raw_counts = reinterpret_cast(counts->mutable_data()); + + arrow::internal::VisitSetBitRunsVoid( + input->buffers[0], input->offset, input->length, + [&](int64_t begin, int64_t length) { + for (int64_t input_i = begin, i = begin - input->offset; + input_i < begin + length; ++input_i) { + auto g = group_ids[i]; + raw_sums[g] += BitUtil::GetBit(raw_input, input_i); + raw_counts[g] += 1; + } + }); }; + out_type = uint64(); return Status::OK(); } - Status Visit(const BinaryType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); - }; - return Status::OK(); + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Summing data of type ", type); } - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - encode_next_impl = [num_bytes](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeBigFixed(num_bytes, data, encoded_bytes); - }; - return Status::OK(); + Status Visit(const DataType& type) { + return Status::NotImplemented("Summing data of type ", type); } - EncodeNextImpl encode_next_impl; + ConsumeImpl consume_impl; + std::shared_ptr out_type; }; - using DecodeNextImpl = std::function*)>; + void Init(KernelContext* ctx, const FunctionOptions*, + const std::shared_ptr& input_type) override { + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&sums_)); + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); - struct GetDecodeNextImpl { - static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, - int32_t* null_count) { - // Do we have nulls? - *null_count = 0; - for (int32_t i = 0; i < length; ++i) { - *null_count += encoded_bytes[i][0]; - } - if (*null_count > 0) { - ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); - uint8_t* nulls = (*null_buf)->mutable_data(); - memset(nulls, 0, (*null_buf)->size()); - for (int32_t i = 0; i < length; ++i) { - if (!encoded_bytes[i][0]) { - BitUtil::SetBit(nulls, i); - } - encoded_bytes[i] += 1; - } - } else { - for (int32_t i = 0; i < length; ++i) { - encoded_bytes[i] += 1; - } - } - return Status ::OK(); - } + GetConsumeImpl get_consume_impl; + KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_consume_impl)); - template - static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, - int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + consume_impl_ = std::move(get_consume_impl.consume_impl); + out_type_ = std::move(get_consume_impl.out_type); + } - KERNEL_ASSIGN_OR_RAISE( - auto key_buf, ctx, - ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_size = num_groups() * kSumSize; + KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); + KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); + auto new_size = num_groups() * kSumSize; + std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); + std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); + } - uint8_t* raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - if (NumBits == 1) { - BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - raw_output[i] = encoded_ptr[0]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 8; - } - } + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); + } - DCHECK(is_integer(output_type) || output_type == Type::BOOL); - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); - } + void Finalize(KernelContext* ctx, Datum* out) override { + std::shared_ptr null_bitmap; + int64_t null_count = 0; - static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, - uint8_t** encoded_bytes, std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + for (int64_t i = 0; i < num_groups(); ++i) { + if (reinterpret_cast(counts_->data())[i] > 0) continue; - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); - auto raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); - encoded_bytes[i] += num_bytes; + if (null_bitmap == nullptr) { + KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); + BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); } - *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, - null_count); + null_count += 1; + BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); } - static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + *out = ArrayData::Make(std::move(out_type_), num_groups(), + {std::move(null_bitmap), std::move(sums_)}, null_count); + } - using offset_type = typename StringType::offset_type; + int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - int32_t length_sum = 0; - for (int32_t i = 0; i < length; ++i) { - length_sum += reinterpret_cast(encoded_bytes)[0]; - } + std::shared_ptr out_type() const override { return out_type_; } - KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, - ctx->Allocate(sizeof(offset_type) * (1 + length))); - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); + // NB: counts are used here instead of a simple "has_values_" bitmap since + // we expect to reuse this kernel to handle Mean + std::shared_ptr sums_, counts_; + std::shared_ptr out_type_; + ConsumeImpl consume_impl_; +}; - auto raw_offsets = offset_buf->mutable_data(); - auto raw_keys = key_buf->mutable_data(); - int32_t current_offset = 0; - for (int32_t i = 0; i < length; ++i) { - offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; - reinterpret_cast(raw_offsets)[i] = current_offset; - encoded_bytes[i] += sizeof(offset_type); - memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); - encoded_bytes[i] += key_length; - current_offset += key_length; - } - reinterpret_cast(raw_offsets)[length] = current_offset; +// ---------------------------------------------------------------------- +// MinMax implementation - if (is_string) { - *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } else { - *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } - } +struct GroupedMinMaxImpl : public GroupedAggregator { + using ConsumeImpl = std::function&, + const uint32_t*, BufferVector*)>; - template - Status Visit(const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - auto type_id = input_type.id(); - switch (num_bits) { - case 1: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 8: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 16: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 32: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 64: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); - }; - break; - } + using ResizeImpl = std::function; + + struct GetImpl { + template ::CType> + enable_if_number Visit(const T&) { + consume_impl = [](const std::shared_ptr& input, + const uint32_t* group_ids, BufferVector* buffers) { + auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); + auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); + + auto raw_has_nulls = buffers->at(2)->mutable_data(); + auto raw_has_values = buffers->at(3)->mutable_data(); + + auto g = group_ids; + VisitArrayDataInline( + *input, + [&](CType val) { + raw_maxes[*g] = std::max(raw_maxes[*g], val); + raw_mins[*g] = std::min(raw_mins[*g], val); + BitUtil::SetBit(raw_has_values, *g++); + }, + [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); + }; + + GetResizeImpls(); return Status::OK(); } - Status Visit(const StringType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, true, length, encoded_bytes, out); - }; - return Status::OK(); + Status Visit(const BooleanType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); } - Status Visit(const BinaryType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, false, length, encoded_bytes, out); - }; - return Status::OK(); + Status Visit(const HalfFloatType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); } - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); - }; - return Status::OK(); + Status Visit(const DataType& type) { + return Status::NotImplemented("Grouped MinMax data of type ", type); } - DecodeNextImpl decode_next_impl; - }; + template + ResizeImpl MakeResizeImpl(CType anti_extreme) { + // resize a min or max buffer, storing the correct anti extreme + return [anti_extreme](Buffer* vals, int64_t new_num_groups) { + int64_t old_num_groups = vals->size() / sizeof(CType); - void Consume(KernelContext* ctx, const ExecBatch& batch) { - ArrayDataVector aggregands, keys; + int64_t new_size = new_num_groups * sizeof(CType); + RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); - size_t i; - for (i = 0; i < aggregators.size(); ++i) { - aggregands.push_back(batch[i].array()); - } - while (i < static_cast(batch.num_values())) { - keys.push_back(batch[i++].array()); + auto raw_vals = reinterpret_cast(vals->mutable_data()); + for (int64_t i = old_num_groups; i != new_num_groups; ++i) { + raw_vals[i] = anti_extreme; + } + return Status::OK(); + }; } - offsets_batch_.clear(); - offsets_batch_.resize(batch.length + 1); - offsets_batch_[0] = 0; - memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); - for (size_t i = 0; i < keys.size(); ++i) { - add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); - } - int32_t total_length = 0; - for (int64_t i = 0; i < batch.length; ++i) { - auto total_length_before = total_length; - total_length += offsets_batch_[i]; - offsets_batch_[i] = total_length_before; + template ::CType> + enable_if_floating_point GetResizeImpls() { + auto inf = std::numeric_limits::infinity(); + resize_min_impl = MakeResizeImpl(inf); + resize_max_impl = MakeResizeImpl(-inf); } - offsets_batch_[batch.length] = total_length; - key_bytes_batch_.clear(); - key_bytes_batch_.resize(total_length); - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(batch.length); - for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; - } - for (size_t i = 0; i < keys.size(); ++i) { - encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); + template ::CType> + enable_if_integer GetResizeImpls() { + resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); + resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); } - group_ids_batch_.clear(); - group_ids_batch_.resize(batch.length); - for (int64_t i = 0; i < batch.length; ++i) { - int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; - std::string key( - reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), - key_length); - auto iter = map_.find(key); - if (iter == map_.end()) { - group_ids_batch_[i] = n_groups++; - auto next_key_offset = static_cast(key_bytes_.size()); - key_bytes_.resize(next_key_offset + key_length); - offsets_.push_back(next_key_offset + key_length); - memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); - map_.insert(std::make_pair(key, group_ids_batch_[i])); - } else { - group_ids_batch_[i] = iter->second; - } - } + ConsumeImpl consume_impl; + ResizeImpl resize_min_impl, resize_max_impl; + }; - for (size_t i = 0; i < aggregators.size(); ++i) { - aggregators[i]->Consume(ctx, aggregands[i], group_ids_batch_.data()); - if (ctx->HasError()) return; + void Init(KernelContext* ctx, const FunctionOptions* options, + const std::shared_ptr& input_type) override { + options_ = *checked_cast(options); + type_ = input_type; + + buffers_.resize(4); + for (auto& buf : buffers_) { + KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&buf)); } - } - void MergeFrom(KernelContext* ctx, KernelState&& src) { - // TODO(ARROW-11840) merge two hash tables - ctx->SetStatus(Status::NotImplemented("merging grouped aggregations")); + GetImpl get_impl; + KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_impl)); + + consume_impl_ = std::move(get_impl.consume_impl); + resize_min_impl_ = std::move(get_impl.resize_min_impl); + resize_max_impl_ = std::move(get_impl.resize_max_impl); } - void Finalize(KernelContext* ctx, Datum* out) { - size_t n_keys = decode_next_impl.size(); - ArrayDataVector out_columns(aggregators.size() + n_keys); - for (size_t i = 0; i < aggregators.size(); ++i) { - Datum aggregand; - aggregators[i]->Finalize(ctx, &aggregand); - if (ctx->HasError()) return; - out_columns[i] = aggregand.array(); - } + void Resize(KernelContext* ctx, int64_t new_num_groups) override { + auto old_num_groups = num_groups_; + num_groups_ = new_num_groups; - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(n_groups); - for (int64_t i = 0; i < n_groups; ++i) { - key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; - } + KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); + KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); - int64_t length = n_groups; - for (size_t i = 0; i < n_keys; ++i) { - std::shared_ptr key_array; - decode_next_impl[i].decode_next_impl(ctx, static_cast(length), - key_buf_ptrs_.data(), &key_array); - out_columns[aggregators.size() + i] = std::move(key_array); + for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { + KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( + BitUtil::BytesForBits(new_num_groups))); + BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); } - - *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, - std::move(out_columns)); } - std::vector offsets_batch_; - std::vector key_bytes_batch_; - std::vector key_buf_ptrs_; - std::vector group_ids_batch_; - - std::unordered_map map_; - std::vector offsets_; - std::vector key_bytes_; - uint32_t n_groups; + void Consume(KernelContext* ctx, const Datum& aggregand, + const uint32_t* group_ids) override { + MaybeResize(ctx, aggregand.length(), group_ids); + if (ctx->HasError()) return; + consume_impl_(aggregand.array(), group_ids, &buffers_); + } - std::shared_ptr out_type; - GroupByOptions options; - std::vector> aggregators; + void Finalize(KernelContext* ctx, Datum* out) override { + // aggregation for group is valid if there was at least one value in that group + std::shared_ptr null_bitmap = std::move(buffers_[3]); - std::vector add_length_impl; - std::vector encode_next_impl; - std::vector decode_next_impl; -}; + if (options_.null_handling == MinMaxOptions::EMIT_NULL) { + // ... and there were no nulls in that group + arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, + num_groups(), 0, null_bitmap->mutable_data()); + } -Result GroupByInit(ExecContext* ctx, const std::vector& aggregands, - const std::vector& keys, - const GroupByOptions& options) { - GroupByImpl impl; - impl.options = options; - const auto& aggregates = impl.options.aggregates; + auto mins = + ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); - impl.n_groups = 0; - impl.offsets_.push_back(0); + auto maxes = ArrayData::Make(type_, num_groups(), + {std::move(null_bitmap), std::move(buffers_[1])}); - if (aggregates.size() != aggregands.size()) { - return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", - aggregands.size(), " aggregands were provided."); + *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, + {std::move(mins), std::move(maxes)}); } - FieldVector out_fields; + int64_t num_groups() const override { return num_groups_; } - impl.aggregators.resize(aggregates.size()); - for (size_t i = 0; i < aggregates.size(); ++i) { - const std::string& function = aggregates[i].function; - - if (function == "count") { - impl.aggregators[i] = ::arrow::internal::make_unique(); - } else if (function == "sum") { - impl.aggregators[i] = ::arrow::internal::make_unique(); - } else if (function == "min_max") { - impl.aggregators[i] = ::arrow::internal::make_unique(); - } else { - return Status::NotImplemented("Grouped aggregate ", function); - } + std::shared_ptr out_type() const override { + return struct_({field("min", type_), field("max", type_)}); + } - const FunctionOptions* options = aggregates[i].options; - if (options == nullptr) { - // use known default options for the named function if possible - auto maybe_function = ctx->func_registry()->GetFunction(function); - if (maybe_function.ok()) { - options = maybe_function.ValueOrDie()->default_options(); - } - } + int64_t num_groups_; + BufferVector buffers_; + std::shared_ptr type_; + ConsumeImpl consume_impl_; + ResizeImpl resize_min_impl_, resize_max_impl_; + MinMaxOptions options_; +}; - KernelContext kernel_ctx{ctx}; - impl.aggregators[i]->Init(&kernel_ctx, options, aggregands[i].type()); - if (kernel_ctx.HasError()) return kernel_ctx.status(); +template +KernelInit MakeInit(GroupByOptions::Aggregate a) { + return [a](KernelContext* ctx, + const KernelInitArgs& args) -> std::unique_ptr { + auto impl = ::arrow::internal::make_unique(); + impl->Init(ctx, a.options, args.inputs[0].type); + if (ctx->HasError()) return nullptr; + return impl; + }; +} - out_fields.push_back(field("", impl.aggregators[i]->out_type())); +Result MakeKernel(GroupByOptions::Aggregate a) { + HashAggregateKernel kernel; + + if (a.function == "count") { + kernel.init = MakeInit(a); + } else if (a.function == "sum") { + kernel.init = MakeInit(a); + } else if (a.function == "min_max") { + kernel.init = MakeInit(a); + } else { + return Status::NotImplemented("Grouped aggregate ", a.function); } - for (const auto& key : keys) { - const auto& key_type = key.type(); - switch (key_type->id()) { - // Supported types of keys - case Type::BOOL: - case Type::UINT8: - case Type::INT8: - case Type::UINT16: - case Type::INT16: - case Type::UINT32: - case Type::INT32: - case Type::UINT64: - case Type::INT64: - case Type::STRING: - case Type::BINARY: - case Type::FIXED_SIZE_BINARY: - break; - default: - return Status::NotImplemented("Key of type", key_type->ToString()); - } - out_fields.push_back(field("", key_type)); - } + // this isn't really in the spirit of things, but I'll get around to defining + // HashAggregateFunctions later + kernel.signature = KernelSignature::Make( + {{}, {}}, OutputType([](KernelContext* ctx, + const std::vector&) -> Result { + return checked_cast(ctx->state())->out_type(); + })); + + kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { + auto group_ids = batch[1].array()->GetValues(1); + checked_cast(ctx->state())->Consume(ctx, batch[0], group_ids); + }; - impl.add_length_impl.resize(keys.size()); - impl.encode_next_impl.resize(keys.size()); - impl.decode_next_impl.resize(keys.size()); - for (size_t i = 0; i < keys.size(); ++i) { - const auto& key_type = keys[i].type(); - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.add_length_impl[i])); - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.encode_next_impl[i])); - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.decode_next_impl[i])); - } + kernel.merge = [](KernelContext* ctx, KernelState&&, KernelState*) { + // TODO(ARROW-11840) merge two hash tables + ctx->SetStatus(Status::NotImplemented("Merge hashed aggregations")); + }; - impl.out_type = struct_(std::move(out_fields)); - return impl; + kernel.finalize = [](KernelContext* ctx, Datum* out) { + checked_cast(ctx->state())->Finalize(ctx, out); + }; + + return kernel; } } // namespace aggregate diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h index 84591e2de94..3d3853d3d2d 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h @@ -30,7 +30,7 @@ namespace arrow { namespace compute { /// \brief Configure a grouped aggregation -struct ARROW_EXPORT GroupByOptions : public FunctionOptions { +struct ARROW_EXPORT GroupByOptions { struct Aggregate { /// the name of the aggregation function std::string function; @@ -49,41 +49,12 @@ struct ARROW_EXPORT GroupByOptions : public FunctionOptions { std::vector aggregates; }; +/// Internal use only: helper function for testing HashAggregateKernels. +/// This will be replaced by streaming execution operators. ARROW_EXPORT Result GroupBy(const std::vector& aggregands, const std::vector& keys, const GroupByOptions& options, ExecContext* ctx = nullptr); -struct GroupedAggregator { - virtual ~GroupedAggregator() = default; - - virtual void Init(KernelContext*, const FunctionOptions*, - const std::shared_ptr&) = 0; - - virtual void Consume(KernelContext*, const Datum& aggregand, - const uint32_t* group_ids) = 0; - - virtual void Finalize(KernelContext* ctx, Datum* out) = 0; - - virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; - - virtual int64_t num_groups() const = 0; - - void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { - if (length == 0) return; - - // maybe a batch of group_ids should include the min/max group id - int64_t max_group = *std::max_element(group_ids, group_ids + length); - auto old_size = num_groups(); - - if (max_group >= old_size) { - auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); - Resize(ctx, new_size); - } - } - - virtual std::shared_ptr out_type() const = 0; -}; - } // namespace compute } // namespace arrow From eb90bf627d2da9a35d035c44191af7e78c947b2b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 16 Mar 2021 12:54:30 -0400 Subject: [PATCH 21/49] refactor for simplicity --- cpp/src/arrow/buffer_builder.h | 10 + .../arrow/compute/kernels/hash_aggregate.cc | 609 ++++++++---------- .../compute/kernels/hash_aggregate_test.cc | 46 ++ 3 files changed, 328 insertions(+), 337 deletions(-) diff --git a/cpp/src/arrow/buffer_builder.h b/cpp/src/arrow/buffer_builder.h index 7a33ff85cd0..53761f80c96 100644 --- a/cpp/src/arrow/buffer_builder.h +++ b/cpp/src/arrow/buffer_builder.h @@ -208,6 +208,11 @@ class TypedBufferBuilder< MemoryPool* pool = default_memory_pool()) : bytes_builder_(std::move(buffer), pool) {} + explicit TypedBufferBuilder(BufferBuilder builder) + : bytes_builder_(std::move(builder)) {} + + BufferBuilder& bytes_builder() { return bytes_builder_; } + Status Append(T value) { return bytes_builder_.Append(reinterpret_cast(&value), sizeof(T)); } @@ -286,6 +291,11 @@ class TypedBufferBuilder { explicit TypedBufferBuilder(MemoryPool* pool = default_memory_pool()) : bytes_builder_(pool) {} + explicit TypedBufferBuilder(BufferBuilder builder) + : bytes_builder_(std::move(builder)) {} + + BufferBuilder& bytes_builder() { return bytes_builder_; } + Status Append(bool value) { ARROW_RETURN_NOT_OK(Reserve(1)); UnsafeAppend(value); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index a4558781c98..7fbf98a69f7 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -114,149 +114,130 @@ struct GroupByImpl { template static void EncodeSmallFixed(const std::shared_ptr& data, uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - uint64_t null_multiplier = is_null ? 0 : 1; - if (NumBits == 1) { - encoded_ptr[0] = static_cast( - null_multiplier * (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0)); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - static_cast(null_multiplier * reinterpret_cast( - raw_input)[offset + i]); - encoded_ptr += 8; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - if (NumBits == 1) { - encoded_ptr[0] = (BitUtil::GetBit(raw_input, offset + i) ? 1 : 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - encoded_ptr[0] = reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(encoded_ptr)[0] = - reinterpret_cast(raw_input)[offset + i]; - encoded_ptr += 8; - } - } + switch (NumBits) { + case 1: + return VisitArrayDataInline( + *data, + [&](bool bit) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + *encoded_ptr++ = bit; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + *encoded_ptr++ = 0; + }); + + case 8: + return VisitArrayDataInline( + *data, + [&](uint8_t byte) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + *encoded_ptr++ = byte; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + *encoded_ptr++ = 0; + }); + + case 16: + return VisitArrayDataInline( + *data, + [&](uint16_t word) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + util::SafeStore(encoded_ptr, word); + encoded_ptr += sizeof(uint16_t); + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + util::SafeStore(encoded_ptr, uint16_t(0)); + encoded_ptr += sizeof(uint16_t); + }); + + case 32: + return VisitArrayDataInline( + *data, + [&](uint32_t quad) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + util::SafeStore(encoded_ptr, quad); + encoded_ptr += sizeof(uint32_t); + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + util::SafeStore(encoded_ptr, uint32_t(0)); + encoded_ptr += sizeof(uint32_t); + }); + + case 64: + return VisitArrayDataInline( + *data, + [&](uint64_t oct) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + util::SafeStore(encoded_ptr, oct); + encoded_ptr += sizeof(uint64_t); + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + util::SafeStore(encoded_ptr, uint64_t(0)); + encoded_ptr += sizeof(uint64_t); + }); } } - static void EncodeBigFixed(int num_bytes, const std::shared_ptr& data, + static void EncodeBigFixed(const std::shared_ptr& data, uint8_t** encoded_bytes) { - auto raw_input = data->buffers[1]->data(); - auto offset = data->offset; - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - encoded_ptr[0] = is_null ? 1 : 0; - encoded_ptr += 1; - if (is_null) { + auto num_bytes = checked_cast(*data->type).byte_width(); + return VisitArrayDataInline( + *data, + [&](util::string_view bytes) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + memcpy(encoded_ptr, bytes.data(), num_bytes); + encoded_ptr += num_bytes; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; memset(encoded_ptr, 0, num_bytes); - } else { - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - } - encoded_ptr += num_bytes; - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr += 1; - memcpy(encoded_ptr, raw_input + num_bytes * (offset + i), num_bytes); - encoded_ptr += num_bytes; - } - } + encoded_ptr += num_bytes; + }); } + template static void EncodeVarLength(const std::shared_ptr& data, uint8_t** encoded_bytes) { - using offset_type = typename StringType::offset_type; - auto offset = data->offset; - const auto offsets = data->GetValues(1); - auto raw_input = data->buffers[2]->data(); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - encoded_ptr[0] = 1; - encoded_ptr++; - reinterpret_cast(encoded_ptr)[0] = 0; + using offset_type = typename T::offset_type; + + return VisitArrayDataInline( + *data, + [&](util::string_view bytes) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + util::SafeStore(encoded_ptr, static_cast(bytes.size())); encoded_ptr += sizeof(offset_type); - } else { - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; + memcpy(encoded_ptr, bytes.data(), bytes.size()); + encoded_ptr += bytes.size(); + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + util::SafeStore(encoded_ptr, static_cast(0)); encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - encoded_ptr[0] = 0; - encoded_ptr++; - size_t num_bytes = offsets[offset + i + 1] - offsets[offset + i]; - reinterpret_cast(encoded_ptr)[0] = num_bytes; - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, raw_input + offsets[offset + i], num_bytes); - encoded_ptr += num_bytes; - } - } + }); } - template - Status Visit(const T& input_type) { + template + typename std::enable_if::type Visit( + const T& input_type) { int32_t num_bits = bit_width(input_type.id()); switch (num_bits) { case 1: @@ -293,29 +274,25 @@ struct GroupByImpl { return Status::OK(); } - Status Visit(const StringType&) { + template + enable_if_base_binary Visit(const T&) { encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); + EncodeVarLength(data, encoded_bytes); }; return Status::OK(); } - Status Visit(const BinaryType&) { + Status Visit(const FixedSizeBinaryType&) { encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); + EncodeBigFixed(data, encoded_bytes); }; return Status::OK(); } - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - encode_next_impl = [num_bytes](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeBigFixed(num_bytes, data, encoded_bytes); - }; - return Status::OK(); + Status Visit(const DataType& type) { + return Status::NotImplemented("encoding hashes for type ", type); } EncodeNextImpl encode_next_impl; @@ -585,11 +562,11 @@ struct GroupByImpl { } } - UInt32Array group_ids(group_ids_batch_.size(), Buffer::Wrap(group_ids_batch_)); + UInt32Array group_ids(batch.length, Buffer::Wrap(group_ids_batch_)); for (size_t i = 0; i < aggregators.size(); ++i) { KernelContext batch_ctx{ctx->exec_context()}; batch_ctx.SetState(aggregator_states[i].get()); - ExecBatch batch({aggregands[i], group_ids}, group_ids.length()); + ExecBatch batch({aggregands[i], group_ids, Datum(n_groups)}, group_ids.length()); aggregators[i].consume(&batch_ctx, batch); ctx->SetStatus(batch_ctx.status()); if (ctx->HasError()) return; @@ -693,6 +670,7 @@ Result GroupByInit(ExecContext* ctx, const std::vector& aggr &kernel_ctx, { aggregands[i].type(), uint32(), + uint32(), })); out_fields.push_back(field("", descr.type)); } @@ -738,29 +716,31 @@ Result GroupByInit(ExecContext* ctx, const std::vector& aggr /// Implementations should be default constructible and perform initialization in /// Init(). struct GroupedAggregator : KernelState { - virtual void Init(KernelContext*, const FunctionOptions*, - const std::shared_ptr&) = 0; - - virtual void Consume(KernelContext*, const Datum& aggregand, - const uint32_t* group_ids) = 0; - - virtual void Finalize(KernelContext* ctx, Datum* out) = 0; - - virtual void Resize(KernelContext* ctx, int64_t new_num_groups) = 0; + virtual Status Init(ExecContext*, const FunctionOptions*, + const std::shared_ptr&) = 0; - virtual int64_t num_groups() const = 0; + virtual Status Consume(const ExecBatch& batch) = 0; - void MaybeResize(KernelContext* ctx, int64_t length, const uint32_t* group_ids) { - if (length == 0) return; + virtual Result Finalize() = 0; - // maybe a batch of group_ids should include the min/max group id - int64_t max_group = *std::max_element(group_ids, group_ids + length); - auto old_size = num_groups(); + template + Status MaybeReserve(int64_t old_num_groups, const ExecBatch& batch, + const Reserve& reserve) { + int64_t new_num_groups = batch[2].scalar_as().value; + if (new_num_groups <= old_num_groups) { + return Status::OK(); + } + return reserve(new_num_groups - old_num_groups); + } - if (max_group >= old_size) { - auto new_size = BufferBuilder::GrowByFactor(old_size, max_group + 1); - Resize(ctx, new_size); + template + Status MaybeResize(int64_t old_num_groups, const ExecBatch& batch, + const Resize& resize) { + int64_t new_num_groups = batch[2].scalar_as().value; + if (new_num_groups <= old_num_groups) { + return Status::OK(); } + return resize(old_num_groups, new_num_groups); } virtual std::shared_ptr out_type() const = 0; @@ -770,38 +750,30 @@ struct GroupedAggregator : KernelState { // Count implementation struct GroupedCountImpl : public GroupedAggregator { - void Init(KernelContext* ctx, const FunctionOptions* options, - const std::shared_ptr&) override { + Status Init(ExecContext* ctx, const FunctionOptions* options, + const std::shared_ptr&) override { options_ = checked_cast(*options); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups(); - KERNEL_RETURN_IF_ERROR(ctx, counts_->TypedResize(new_num_groups)); - auto new_size = num_groups(); - - auto raw_counts = reinterpret_cast(counts_->mutable_data()); - for (auto i = old_size; i < new_size; ++i) { - raw_counts[i] = 0; - } + counts_ = BufferBuilder(ctx->memory_pool()); + return Status::OK(); } - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; + Status Consume(const ExecBatch& batch) override { + RETURN_NOT_OK(MaybeReserve(counts_.length(), batch, [&](int64_t added_groups) { + num_groups_ += added_groups; + return counts_.Append(added_groups * sizeof(int64_t), 0); + })); - auto raw_counts = reinterpret_cast(counts_->mutable_data()); + auto group_ids = batch[1].array()->GetValues(1); + auto raw_counts = reinterpret_cast(counts_.mutable_data()); - const auto& input = aggregand.array(); + const auto& input = batch[0].array(); if (options_.count_mode == CountOptions::COUNT_NULL) { for (int64_t i = 0, input_i = input->offset; i < input->length; ++i, ++input_i) { auto g = group_ids[i]; raw_counts[g] += !BitUtil::GetBit(input->buffers[0]->data(), input_i); } - return; + return Status::OK(); } arrow::internal::VisitSetBitRunsVoid( @@ -813,19 +785,19 @@ struct GroupedCountImpl : public GroupedAggregator { raw_counts[g] += 1; } }); + return Status::OK(); } - void Finalize(KernelContext* ctx, Datum* out) override { - auto length = num_groups(); - *out = std::make_shared(length, std::move(counts_)); + Result Finalize() override { + ARROW_ASSIGN_OR_RAISE(auto counts, counts_.Finish()); + return std::make_shared(num_groups_, std::move(counts)); } - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } - std::shared_ptr out_type() const override { return int64(); } + int64_t num_groups_ = 0; CountOptions options_; - std::shared_ptr counts_; + BufferBuilder counts_; }; // ---------------------------------------------------------------------- @@ -837,54 +809,25 @@ struct GroupedSumImpl : public GroupedAggregator { static constexpr size_t kSumSize = sizeof(int64_t); using ConsumeImpl = std::function&, - const uint32_t*, Buffer*, Buffer*)>; + const uint32_t*, void*, int64_t*)>; struct GetConsumeImpl { - template ::Type> + template ::Type> Status Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = reinterpret_cast::CType*>( - input->buffers[1]->data()); - auto raw_sums = reinterpret_cast::CType*>( - sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i, ++i) { - auto g = group_ids[i]; - raw_sums[g] += raw_input[input_i]; - raw_counts[g] += 1; - } - }); - }; - out_type = TypeTraits::type_singleton(); - return Status::OK(); - } + consume_impl = [](const std::shared_ptr& input, const uint32_t* group, + void* boxed_sums, int64_t* counts) { + auto sums = reinterpret_cast::CType*>(boxed_sums); - Status Visit(const BooleanType&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, Buffer* sums, Buffer* counts) { - auto raw_input = input->buffers[1]->data(); - auto raw_sums = reinterpret_cast(sums->mutable_data()); - auto raw_counts = reinterpret_cast(counts->mutable_data()); - - arrow::internal::VisitSetBitRunsVoid( - input->buffers[0], input->offset, input->length, - [&](int64_t begin, int64_t length) { - for (int64_t input_i = begin, i = begin - input->offset; - input_i < begin + length; ++input_i) { - auto g = group_ids[i]; - raw_sums[g] += BitUtil::GetBit(raw_input, input_i); - raw_counts[g] += 1; - } - }); + VisitArrayDataInline( + *input, + [&](typename TypeTraits::CType value) { + sums[*group] += value; + counts[*group] += 1; + ++group; + }, + [&] { ++group; }); }; - out_type = uint64(); + out_type = TypeTraits::type_singleton(); return Status::OK(); } @@ -900,94 +843,106 @@ struct GroupedSumImpl : public GroupedAggregator { std::shared_ptr out_type; }; - void Init(KernelContext* ctx, const FunctionOptions*, - const std::shared_ptr& input_type) override { - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&sums_)); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&counts_)); + Status Init(ExecContext* ctx, const FunctionOptions*, + const std::shared_ptr& input_type) override { + pool_ = ctx->memory_pool(); + sums_ = BufferBuilder(pool_); + counts_ = BufferBuilder(pool_); GetConsumeImpl get_consume_impl; - KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_consume_impl)); + RETURN_NOT_OK(VisitTypeInline(*input_type, &get_consume_impl)); consume_impl_ = std::move(get_consume_impl.consume_impl); out_type_ = std::move(get_consume_impl.out_type); - } - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_size = num_groups() * kSumSize; - KERNEL_RETURN_IF_ERROR(ctx, sums_->Resize(new_num_groups * kSumSize)); - KERNEL_RETURN_IF_ERROR(ctx, counts_->Resize(new_num_groups * sizeof(int64_t))); - auto new_size = num_groups() * kSumSize; - std::memset(sums_->mutable_data() + old_size, 0, new_size - old_size); - std::memset(counts_->mutable_data() + old_size, 0, new_size - old_size); + return Status::OK(); } - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, sums_.get(), counts_.get()); + Status Consume(const ExecBatch& batch) override { + RETURN_NOT_OK(MaybeReserve(num_groups_, batch, [&](int64_t added_groups) { + num_groups_ += added_groups; + RETURN_NOT_OK(sums_.Append(added_groups * kSumSize, 0)); + RETURN_NOT_OK(counts_.Append(added_groups * sizeof(int64_t), 0)); + return Status::OK(); + })); + + auto group_ids = batch[1].array()->GetValues(1); + consume_impl_(batch[0].array(), group_ids, sums_.mutable_data(), + reinterpret_cast(counts_.mutable_data())); + return Status::OK(); } - void Finalize(KernelContext* ctx, Datum* out) override { + Result Finalize() override { std::shared_ptr null_bitmap; int64_t null_count = 0; - for (int64_t i = 0; i < num_groups(); ++i) { - if (reinterpret_cast(counts_->data())[i] > 0) continue; + for (int64_t i = 0; i < num_groups_; ++i) { + if (reinterpret_cast(counts_.data())[i] > 0) continue; if (null_bitmap == nullptr) { - KERNEL_ASSIGN_OR_RAISE(null_bitmap, ctx, ctx->AllocateBitmap(num_groups())); - BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups(), true); + ARROW_ASSIGN_OR_RAISE(null_bitmap, AllocateBitmap(num_groups_, pool_)); + BitUtil::SetBitsTo(null_bitmap->mutable_data(), 0, num_groups_, true); } null_count += 1; BitUtil::SetBitTo(null_bitmap->mutable_data(), i, false); } - *out = ArrayData::Make(std::move(out_type_), num_groups(), - {std::move(null_bitmap), std::move(sums_)}, null_count); - } + ARROW_ASSIGN_OR_RAISE(auto sums, sums_.Finish()); - int64_t num_groups() const override { return counts_->size() / sizeof(int64_t); } + return ArrayData::Make(std::move(out_type_), num_groups_, + {std::move(null_bitmap), std::move(sums)}, null_count); + } std::shared_ptr out_type() const override { return out_type_; } // NB: counts are used here instead of a simple "has_values_" bitmap since // we expect to reuse this kernel to handle Mean - std::shared_ptr sums_, counts_; + int64_t num_groups_ = 0; + BufferBuilder sums_, counts_; std::shared_ptr out_type_; ConsumeImpl consume_impl_; + MemoryPool* pool_; }; // ---------------------------------------------------------------------- // MinMax implementation struct GroupedMinMaxImpl : public GroupedAggregator { - using ConsumeImpl = std::function&, - const uint32_t*, BufferVector*)>; - - using ResizeImpl = std::function; + using ConsumeImpl = + std::function&, const uint32_t*, void*, void*, + uint8_t*, uint8_t*)>; + + using ResizeImpl = std::function; + + template + static ResizeImpl MakeResizeImpl(CType anti_extreme) { + // resize a min or max buffer, storing the correct anti extreme + return [anti_extreme](BufferBuilder* builder, int64_t added_groups) { + TypedBufferBuilder typed_builder(std::move(*builder)); + RETURN_NOT_OK(typed_builder.Append(added_groups, anti_extreme)); + *builder = std::move(typed_builder.bytes_builder()); + return Status::OK(); + }; + } struct GetImpl { template ::CType> enable_if_number Visit(const T&) { - consume_impl = [](const std::shared_ptr& input, - const uint32_t* group_ids, BufferVector* buffers) { - auto raw_mins = reinterpret_cast(buffers->at(0)->mutable_data()); - auto raw_maxes = reinterpret_cast(buffers->at(1)->mutable_data()); + consume_impl = [](const std::shared_ptr& input, const uint32_t* group, + void* mins, void* maxes, uint8_t* has_values, + uint8_t* has_nulls) { + auto raw_mins = reinterpret_cast(mins); + auto raw_maxes = reinterpret_cast(maxes); - auto raw_has_nulls = buffers->at(2)->mutable_data(); - auto raw_has_values = buffers->at(3)->mutable_data(); - - auto g = group_ids; VisitArrayDataInline( *input, [&](CType val) { - raw_maxes[*g] = std::max(raw_maxes[*g], val); - raw_mins[*g] = std::min(raw_mins[*g], val); - BitUtil::SetBit(raw_has_values, *g++); + raw_maxes[*group] = std::max(raw_maxes[*group], val); + raw_mins[*group] = std::min(raw_mins[*group], val); + BitUtil::SetBit(has_values, *group++); }, - [&] { BitUtil::SetBit(raw_has_nulls, *g++); }); + [&] { BitUtil::SetBit(has_nulls, *group++); }); }; GetResizeImpls(); @@ -1006,23 +961,6 @@ struct GroupedMinMaxImpl : public GroupedAggregator { return Status::NotImplemented("Grouped MinMax data of type ", type); } - template - ResizeImpl MakeResizeImpl(CType anti_extreme) { - // resize a min or max buffer, storing the correct anti extreme - return [anti_extreme](Buffer* vals, int64_t new_num_groups) { - int64_t old_num_groups = vals->size() / sizeof(CType); - - int64_t new_size = new_num_groups * sizeof(CType); - RETURN_NOT_OK(checked_cast(vals)->Resize(new_size)); - - auto raw_vals = reinterpret_cast(vals->mutable_data()); - for (int64_t i = old_num_groups; i != new_num_groups; ++i) { - raw_vals[i] = anti_extreme; - } - return Status::OK(); - }; - } - template ::CType> enable_if_floating_point GetResizeImpls() { auto inf = std::numeric_limits::infinity(); @@ -1040,76 +978,73 @@ struct GroupedMinMaxImpl : public GroupedAggregator { ResizeImpl resize_min_impl, resize_max_impl; }; - void Init(KernelContext* ctx, const FunctionOptions* options, - const std::shared_ptr& input_type) override { + Status Init(ExecContext* ctx, const FunctionOptions* options, + const std::shared_ptr& input_type) override { options_ = *checked_cast(options); type_ = input_type; - buffers_.resize(4); - for (auto& buf : buffers_) { - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(0).Value(&buf)); - } + mins_ = BufferBuilder(ctx->memory_pool()); + maxes_ = BufferBuilder(ctx->memory_pool()); + has_values_ = BufferBuilder(ctx->memory_pool()); + has_nulls_ = BufferBuilder(ctx->memory_pool()); GetImpl get_impl; - KERNEL_RETURN_IF_ERROR(ctx, VisitTypeInline(*input_type, &get_impl)); + RETURN_NOT_OK(VisitTypeInline(*input_type, &get_impl)); consume_impl_ = std::move(get_impl.consume_impl); resize_min_impl_ = std::move(get_impl.resize_min_impl); resize_max_impl_ = std::move(get_impl.resize_max_impl); - } - - void Resize(KernelContext* ctx, int64_t new_num_groups) override { - auto old_num_groups = num_groups_; - num_groups_ = new_num_groups; - - KERNEL_RETURN_IF_ERROR(ctx, resize_min_impl_(buffers_[0].get(), new_num_groups)); - KERNEL_RETURN_IF_ERROR(ctx, resize_max_impl_(buffers_[1].get(), new_num_groups)); + resize_bitmap_impl_ = MakeResizeImpl(false); - for (auto buffer : {buffers_[2].get(), buffers_[3].get()}) { - KERNEL_RETURN_IF_ERROR(ctx, checked_cast(buffer)->Resize( - BitUtil::BytesForBits(new_num_groups))); - BitUtil::SetBitsTo(buffer->mutable_data(), old_num_groups, new_num_groups, false); - } + return Status::OK(); } - void Consume(KernelContext* ctx, const Datum& aggregand, - const uint32_t* group_ids) override { - MaybeResize(ctx, aggregand.length(), group_ids); - if (ctx->HasError()) return; - consume_impl_(aggregand.array(), group_ids, &buffers_); + Status Consume(const ExecBatch& batch) override { + RETURN_NOT_OK(MaybeReserve(num_groups_, batch, [&](int64_t added_groups) { + num_groups_ += added_groups; + RETURN_NOT_OK(resize_min_impl_(&mins_, added_groups)); + RETURN_NOT_OK(resize_max_impl_(&maxes_, added_groups)); + RETURN_NOT_OK(resize_bitmap_impl_(&has_values_, added_groups)); + RETURN_NOT_OK(resize_bitmap_impl_(&has_nulls_, added_groups)); + return Status::OK(); + })); + + auto group_ids = batch[1].array()->GetValues(1); + consume_impl_(batch[0].array(), group_ids, mins_.mutable_data(), + maxes_.mutable_data(), has_values_.mutable_data(), + has_nulls_.mutable_data()); + return Status::OK(); } - void Finalize(KernelContext* ctx, Datum* out) override { + Result Finalize() override { // aggregation for group is valid if there was at least one value in that group - std::shared_ptr null_bitmap = std::move(buffers_[3]); + ARROW_ASSIGN_OR_RAISE(auto null_bitmap, has_values_.Finish()); if (options_.null_handling == MinMaxOptions::EMIT_NULL) { // ... and there were no nulls in that group - arrow::internal::BitmapAndNot(null_bitmap->data(), 0, buffers_[2]->data(), 0, - num_groups(), 0, null_bitmap->mutable_data()); + ARROW_ASSIGN_OR_RAISE(auto has_nulls, has_nulls_.Finish()); + arrow::internal::BitmapAndNot(null_bitmap->data(), 0, has_nulls->data(), 0, + num_groups_, 0, null_bitmap->mutable_data()); } - auto mins = - ArrayData::Make(type_, num_groups(), {null_bitmap, std::move(buffers_[0])}); + auto mins = ArrayData::Make(type_, num_groups_, {null_bitmap, nullptr}); + auto maxes = ArrayData::Make(type_, num_groups_, {std::move(null_bitmap), nullptr}); + ARROW_ASSIGN_OR_RAISE(mins->buffers[1], mins_.Finish()); + ARROW_ASSIGN_OR_RAISE(maxes->buffers[1], maxes_.Finish()); - auto maxes = ArrayData::Make(type_, num_groups(), - {std::move(null_bitmap), std::move(buffers_[1])}); - - *out = ArrayData::Make(out_type(), num_groups(), {nullptr}, + return ArrayData::Make(out_type(), num_groups_, {nullptr}, {std::move(mins), std::move(maxes)}); } - int64_t num_groups() const override { return num_groups_; } - std::shared_ptr out_type() const override { return struct_({field("min", type_), field("max", type_)}); } int64_t num_groups_; - BufferVector buffers_; + BufferBuilder mins_, maxes_, has_values_, has_nulls_; std::shared_ptr type_; ConsumeImpl consume_impl_; - ResizeImpl resize_min_impl_, resize_max_impl_; + ResizeImpl resize_min_impl_, resize_max_impl_, resize_bitmap_impl_; MinMaxOptions options_; }; @@ -1118,7 +1053,7 @@ KernelInit MakeInit(GroupByOptions::Aggregate a) { return [a](KernelContext* ctx, const KernelInitArgs& args) -> std::unique_ptr { auto impl = ::arrow::internal::make_unique(); - impl->Init(ctx, a.options, args.inputs[0].type); + ctx->SetStatus(impl->Init(ctx->exec_context(), a.options, args.inputs[0].type)); if (ctx->HasError()) return nullptr; return impl; }; @@ -1140,14 +1075,13 @@ Result MakeKernel(GroupByOptions::Aggregate a) { // this isn't really in the spirit of things, but I'll get around to defining // HashAggregateFunctions later kernel.signature = KernelSignature::Make( - {{}, {}}, OutputType([](KernelContext* ctx, - const std::vector&) -> Result { + {{}, {}, {}}, OutputType([](KernelContext* ctx, + const std::vector&) -> Result { return checked_cast(ctx->state())->out_type(); })); kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { - auto group_ids = batch[1].array()->GetValues(1); - checked_cast(ctx->state())->Consume(ctx, batch[0], group_ids); + ctx->SetStatus(checked_cast(ctx->state())->Consume(batch)); }; kernel.merge = [](KernelContext* ctx, KernelState&&, KernelState*) { @@ -1156,7 +1090,8 @@ Result MakeKernel(GroupByOptions::Aggregate a) { }; kernel.finalize = [](KernelContext* ctx, Datum* out) { - checked_cast(ctx->state())->Finalize(ctx, out); + KERNEL_ASSIGN_OR_RAISE(*out, ctx, + checked_cast(ctx->state())->Finalize()); }; return kernel; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 73e0d2a86c1..700d7811a5a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -398,6 +398,52 @@ void ValidateGroupBy(GroupByOptions options, std::vector aggregands, } // namespace } // namespace group_helpers +TEST(GroupBy, SumOnly8bitKey) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int8(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", int8()), + }), + R"([ + [4.25, 1], + [-0.125, 2], + [null, 3], + [0.75, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + +TEST(GroupBy, SumOnly32bitKey) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int32(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", int32()), + }), + R"([ + [4.25, 1], + [-0.125, 2], + [null, 3], + [0.75, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + TEST(GroupBy, SumOnly) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); From 2ab608ddedb1e99495b6c2e0d4881ab4e317fb64 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 16 Mar 2021 14:52:59 -0400 Subject: [PATCH 22/49] only consider bit width of key types --- .../arrow/compute/kernels/hash_aggregate.cc | 420 +++++------------- .../compute/kernels/hash_aggregate_test.cc | 24 + 2 files changed, 141 insertions(+), 303 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 7fbf98a69f7..9e6468c33e8 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -46,57 +46,30 @@ struct GroupByImpl { } } + template static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { - using offset_type = typename StringType::offset_type; + using offset_type = typename T::offset_type; constexpr int32_t length_extra_bytes = sizeof(offset_type); - auto offset = data->offset; - const auto offsets = data->GetValues(1); - if (data->MayHaveNulls()) { - const uint8_t* nulls = data->buffers[0]->data(); - - for (int64_t i = 0; i < data->length; ++i) { - bool is_null = !BitUtil::GetBit(nulls, offset + i); - if (is_null) { - lengths[i] += null_extra_byte + length_extra_bytes; - } else { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } - } - } else { - for (int64_t i = 0; i < data->length; ++i) { - lengths[i] += null_extra_byte + length_extra_bytes + offsets[offset + i + 1] - - offsets[offset + i]; - } - } - } - - template - Status Visit(const T& input_type) { - int32_t num_bytes = (bit_width(input_type.id()) + 7) / 8; - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); - }; - return Status::OK(); - } - Status Visit(const StringType&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); - }; - return Status::OK(); + int64_t i = 0; + return VisitArrayDataInline( + *data, + [&](util::string_view bytes) { + lengths[i++] += null_extra_byte + length_extra_bytes + bytes.size(); + }, + [&] { lengths[i++] += null_extra_byte + length_extra_bytes; }); } - Status Visit(const BinaryType&) { + template + enable_if_base_binary Visit(const T&) { add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); + AddVarLength(data, lengths); }; return Status::OK(); } - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); + Status Visit(const FixedWidthType& type) { + int32_t num_bytes = BitUtil::BytesForBits(type.bit_width()); add_length_impl = [num_bytes](const std::shared_ptr& data, int32_t* lengths) { AddFixedLength(num_bytes, data->length, lengths); @@ -104,6 +77,10 @@ struct GroupByImpl { return Status::OK(); } + Status Visit(const DataType& type) { + return Status::NotImplemented("Computing encoded key lengths for type ", type); + } + AddLengthImpl add_length_impl; }; @@ -111,93 +88,31 @@ struct GroupByImpl { std::function&, uint8_t**)>; struct GetEncodeNextImpl { - template - static void EncodeSmallFixed(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - switch (NumBits) { - case 1: - return VisitArrayDataInline( - *data, - [&](bool bit) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - *encoded_ptr++ = bit; - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - *encoded_ptr++ = 0; - }); - - case 8: - return VisitArrayDataInline( - *data, - [&](uint8_t byte) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - *encoded_ptr++ = byte; - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - *encoded_ptr++ = 0; - }); - - case 16: - return VisitArrayDataInline( - *data, - [&](uint16_t word) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - util::SafeStore(encoded_ptr, word); - encoded_ptr += sizeof(uint16_t); - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - util::SafeStore(encoded_ptr, uint16_t(0)); - encoded_ptr += sizeof(uint16_t); - }); - - case 32: - return VisitArrayDataInline( - *data, - [&](uint32_t quad) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - util::SafeStore(encoded_ptr, quad); - encoded_ptr += sizeof(uint32_t); - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - util::SafeStore(encoded_ptr, uint32_t(0)); - encoded_ptr += sizeof(uint32_t); - }); - - case 64: - return VisitArrayDataInline( - *data, - [&](uint64_t oct) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - util::SafeStore(encoded_ptr, oct); - encoded_ptr += sizeof(uint64_t); - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - util::SafeStore(encoded_ptr, uint64_t(0)); - encoded_ptr += sizeof(uint64_t); - }); - } + static void EncodeBoolean(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + VisitArrayDataInline( + *data, + [&](bool value) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + *encoded_ptr++ = value; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + *encoded_ptr++ = 0; + }); } - static void EncodeBigFixed(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto num_bytes = checked_cast(*data->type).byte_width(); + static void EncodeFixed(const std::shared_ptr& data, + uint8_t** encoded_bytes) { + auto num_bytes = checked_cast(*data->type).bit_width() / 8; + + ArrayData viewed(fixed_size_binary(num_bytes), data->length, data->buffers, + data->null_count, data->offset); + return VisitArrayDataInline( - *data, + viewed, [&](util::string_view bytes) { auto& encoded_ptr = *encoded_bytes++; *encoded_ptr++ = 0; @@ -235,58 +150,27 @@ struct GroupByImpl { }); } - template - typename std::enable_if::type Visit( - const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - switch (num_bits) { - case 1: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<1>(data, encoded_bytes); - }; - break; - case 8: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<8>(data, encoded_bytes); - }; - break; - case 16: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<16>(data, encoded_bytes); - }; - break; - case 32: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<32>(data, encoded_bytes); - }; - break; - case 64: - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeSmallFixed<64>(data, encoded_bytes); - }; - break; - } + Status Visit(const BooleanType&) { + encode_next_impl = [](const std::shared_ptr& data, + uint8_t** encoded_bytes) { + EncodeBoolean(data, encoded_bytes); + }; return Status::OK(); } - template - enable_if_base_binary Visit(const T&) { + Status Visit(const FixedWidthType&) { encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); + EncodeFixed(data, encoded_bytes); }; return Status::OK(); } - Status Visit(const FixedSizeBinaryType&) { + template + enable_if_base_binary Visit(const T&) { encode_next_impl = [](const std::shared_ptr& data, uint8_t** encoded_bytes) { - EncodeBigFixed(data, encoded_bytes); + EncodeVarLength(data, encoded_bytes); }; return Status::OK(); } @@ -302,22 +186,19 @@ struct GroupByImpl { std::shared_ptr*)>; struct GetDecodeNextImpl { - static Status DecodeNulls(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, - int32_t* null_count) { + static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, int32_t* null_count) { // Do we have nulls? *null_count = 0; for (int32_t i = 0; i < length; ++i) { *null_count += encoded_bytes[i][0]; } if (*null_count > 0) { - ARROW_ASSIGN_OR_RAISE(*null_buf, ctx->AllocateBitmap(length)); + ARROW_ASSIGN_OR_RAISE(*null_buf, AllocateBitmap(length, pool)); + uint8_t* nulls = (*null_buf)->mutable_data(); - memset(nulls, 0, (*null_buf)->size()); for (int32_t i = 0; i < length; ++i) { - if (!encoded_bytes[i][0]) { - BitUtil::SetBit(nulls, i); - } + BitUtil::SetBitTo(nulls, i, !encoded_bytes[i][0]); encoded_bytes[i] += 1; } } else { @@ -328,80 +209,61 @@ struct GroupByImpl { return Status ::OK(); } - template - static void DecodeSmallFixed(KernelContext* ctx, const Type::type& output_type, - int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; + static void DecodeBoolean(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, + &null_buf, &null_count)); - KERNEL_ASSIGN_OR_RAISE( - auto key_buf, ctx, - ctx->Allocate(NumBits == 1 ? (length + 7) / 8 : (NumBits / 8) * length)); + KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->AllocateBitmap(length)); uint8_t* raw_output = key_buf->mutable_data(); for (int32_t i = 0; i < length; ++i) { auto& encoded_ptr = encoded_bytes[i]; - if (NumBits == 1) { - BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); - encoded_ptr += 1; - } - if (NumBits == 8) { - raw_output[i] = encoded_ptr[0]; - encoded_ptr += 1; - } - if (NumBits == 16) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 2; - } - if (NumBits == 32) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 4; - } - if (NumBits == 64) { - reinterpret_cast(raw_output)[i] = - reinterpret_cast(encoded_bytes[i])[0]; - encoded_ptr += 8; - } + BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); + encoded_ptr += 1; } - DCHECK(is_integer(output_type) || output_type == Type::BOOL); - *out = ArrayData::Make(int64(), length, {null_buf, key_buf}, null_count); + *out = ArrayData::Make(boolean(), length, {std::move(null_buf), std::move(key_buf)}, + null_count); } - static void DecodeBigFixed(KernelContext* ctx, int num_bytes, int32_t length, - uint8_t** encoded_bytes, std::shared_ptr* out) { - std::shared_ptr null_buf; + static void DecodeFixed(KernelContext* ctx, std::shared_ptr output_type, + int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + std::shared_ptr null_buf; int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, + &null_buf, &null_count)); + auto num_bytes = checked_cast(*output_type).bit_width() / 8; KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); - auto raw_output = key_buf->mutable_data(); + + uint8_t* raw_output = key_buf->mutable_data(); for (int32_t i = 0; i < length; ++i) { - memcpy(raw_output + i * num_bytes, encoded_bytes[i], num_bytes); - encoded_bytes[i] += num_bytes; + auto& encoded_ptr = encoded_bytes[i]; + std::memcpy(raw_output, encoded_ptr, num_bytes); + encoded_ptr += num_bytes; + raw_output += num_bytes; } - *out = ArrayData::Make(fixed_size_binary(num_bytes), length, {null_buf, key_buf}, - null_count); + *out = ArrayData::Make(std::move(output_type), length, + {std::move(null_buf), std::move(key_buf)}, null_count); } - static void DecodeVarLength(KernelContext* ctx, bool is_string, int32_t length, - uint8_t** encoded_bytes, + template + static void DecodeVarLength(KernelContext* ctx, std::shared_ptr output_type, + int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { - std::shared_ptr null_buf; + std::shared_ptr null_buf; int32_t null_count; - KERNEL_RETURN_IF_ERROR( - ctx, DecodeNulls(ctx, length, encoded_bytes, &null_buf, &null_count)); + KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, + &null_buf, &null_count)); - using offset_type = typename StringType::offset_type; + using offset_type = typename T::offset_type; - int32_t length_sum = 0; + offset_type length_sum = 0; for (int32_t i = 0; i < length; ++i) { length_sum += reinterpret_cast(encoded_bytes)[0]; } @@ -410,101 +272,62 @@ struct GroupByImpl { ctx->Allocate(sizeof(offset_type) * (1 + length))); KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); - auto raw_offsets = offset_buf->mutable_data(); + auto raw_offsets = reinterpret_cast(offset_buf->mutable_data()); auto raw_keys = key_buf->mutable_data(); + int32_t current_offset = 0; for (int32_t i = 0; i < length; ++i) { offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; - reinterpret_cast(raw_offsets)[i] = current_offset; + raw_offsets[i] = current_offset; encoded_bytes[i] += sizeof(offset_type); memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); encoded_bytes[i] += key_length; current_offset += key_length; } - reinterpret_cast(raw_offsets)[length] = current_offset; - - if (is_string) { - *out = ArrayData::Make(utf8(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } else { - *out = ArrayData::Make(binary(), length, {null_buf, offset_buf, key_buf}, - null_count, 0); - } - } + raw_offsets[length] = current_offset; - template - Status Visit(const T& input_type) { - int32_t num_bits = bit_width(input_type.id()); - auto type_id = input_type.id(); - switch (num_bits) { - case 1: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<1>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 8: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<8>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 16: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<16>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 32: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<32>(ctx, type_id, length, encoded_bytes, out); - }; - break; - case 64: - decode_next_impl = [type_id](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeSmallFixed<64>(ctx, type_id, length, encoded_bytes, out); - }; - break; - } - return Status::OK(); + *out = ArrayData::Make( + std::move(output_type), length, + {std::move(null_buf), std::move(offset_buf), std::move(key_buf)}, null_count); } - Status Visit(const StringType&) { + Status Visit(const BooleanType&) { decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, std::shared_ptr* out) { - DecodeVarLength(ctx, true, length, encoded_bytes, out); + DecodeBoolean(ctx, length, encoded_bytes, out); }; return Status::OK(); } - Status Visit(const BinaryType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, false, length, encoded_bytes, out); + Status Visit(const FixedWidthType&) { + auto output_type = out_type; + decode_next_impl = [=](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeFixed(ctx, output_type, length, encoded_bytes, out); }; return Status::OK(); } - Status Visit(const FixedSizeBinaryType& type) { - int32_t num_bytes = type.byte_width(); - decode_next_impl = [num_bytes](KernelContext* ctx, int32_t length, - uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeBigFixed(ctx, num_bytes, length, encoded_bytes, out); + template + enable_if_base_binary Visit(const T&) { + auto output_type = out_type; + decode_next_impl = [=](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* out) { + DecodeVarLength(ctx, output_type, length, encoded_bytes, out); }; return Status::OK(); } + Status Visit(const DataType& type) { + return Status::NotImplemented("decoding keys for type ", type); + } + + std::shared_ptr out_type; DecodeNextImpl decode_next_impl; }; + // TODO(bkietz) Here I'd like to make it clear that Consume produces a batch of + // group_ids; extract the immediate pass-to-HashAggregateKernel. void Consume(KernelContext* ctx, const ExecBatch& batch) { ArrayDataVector aggregands, keys; @@ -693,7 +516,7 @@ Result GroupByInit(ExecContext* ctx, const std::vector& aggr case Type::FIXED_SIZE_BINARY: break; default: - return Status::NotImplemented("Key of type", key_type->ToString()); + return Status::NotImplemented("Key of type ", key_type->ToString()); } out_fields.push_back(field("", key_type)); } @@ -705,6 +528,7 @@ Result GroupByInit(ExecContext* ctx, const std::vector& aggr const auto& key_type = keys[i].type(); RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.add_length_impl[i])); RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.encode_next_impl[i])); + impl.decode_next_impl[i].out_type = key_type; RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.decode_next_impl[i])); } @@ -733,16 +557,6 @@ struct GroupedAggregator : KernelState { return reserve(new_num_groups - old_num_groups); } - template - Status MaybeResize(int64_t old_num_groups, const ExecBatch& batch, - const Resize& resize) { - int64_t new_num_groups = batch[2].scalar_as().value; - if (new_num_groups <= old_num_groups) { - return Status::OK(); - } - return resize(old_num_groups, new_num_groups); - } - virtual std::shared_ptr out_type() const = 0; }; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 700d7811a5a..024d5a6d883 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -398,6 +398,28 @@ void ValidateGroupBy(GroupByOptions options, std::vector aggregands, } // namespace } // namespace group_helpers +TEST(GroupBy, SumOnlyBooleanKey) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(boolean(), "[1, 0, 1, 0, null, 0, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, + GroupByOptions{ + {"sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", boolean()), + }), + R"([ + [1, true], + [3, false], + [0.875, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + TEST(GroupBy, SumOnly8bitKey) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int8(), "[1, 2, 3, 1, 2, 2, null]"); @@ -407,6 +429,8 @@ TEST(GroupBy, SumOnly8bitKey) { {"sum", nullptr}, })); + ASSERT_OK(aggregated_and_grouped.array_as()->ValidateFull()); + AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), field("", int8()), From 0d4b1d4ab4760bca1530a1d57c69dc5e43db8950 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 16 Mar 2021 17:07:04 -0400 Subject: [PATCH 23/49] extract key encoding to KeyEncoder --- .../arrow/compute/kernels/hash_aggregate.cc | 663 ++++++++---------- 1 file changed, 291 insertions(+), 372 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 9e6468c33e8..2bc1351613b 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -33,308 +33,315 @@ namespace arrow { namespace compute { namespace aggregate { -struct GroupByImpl { - using AddLengthImpl = std::function&, int32_t*)>; +struct KeyEncoder { + // the first byte of an encoded key is used to indicate nullity + static constexpr bool kExtraByteForNull = true; - struct GetAddLengthImpl { - static constexpr int32_t null_extra_byte = 1; + virtual ~KeyEncoder() = default; - static void AddFixedLength(int32_t fixed_length, int64_t num_repeats, - int32_t* lengths) { - for (int64_t i = 0; i < num_repeats; ++i) { - lengths[i] += fixed_length + null_extra_byte; - } - } + virtual void AddLength(const ArrayData&, int32_t* lengths) = 0; - template - static void AddVarLength(const std::shared_ptr& data, int32_t* lengths) { - using offset_type = typename T::offset_type; - constexpr int32_t length_extra_bytes = sizeof(offset_type); + virtual void Encode(const ArrayData&, uint8_t** encoded_bytes) = 0; - int64_t i = 0; - return VisitArrayDataInline( - *data, - [&](util::string_view bytes) { - lengths[i++] += null_extra_byte + length_extra_bytes + bytes.size(); - }, - [&] { lengths[i++] += null_extra_byte + length_extra_bytes; }); - } + virtual Result> Decode(uint8_t** encoded_bytes, + int32_t length, MemoryPool*) = 0; - template - enable_if_base_binary Visit(const T&) { - add_length_impl = [](const std::shared_ptr& data, int32_t* lengths) { - AddVarLength(data, lengths); - }; - return Status::OK(); + // extract the null bitmap from the leading nullity bytes of encoded keys + static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, + std::shared_ptr* null_buf, int32_t* null_count) { + // first count nulls to determine if a null bitmap is necessary + *null_count = 0; + for (int32_t i = 0; i < length; ++i) { + *null_count += encoded_bytes[i][0]; } - Status Visit(const FixedWidthType& type) { - int32_t num_bytes = BitUtil::BytesForBits(type.bit_width()); - add_length_impl = [num_bytes](const std::shared_ptr& data, - int32_t* lengths) { - AddFixedLength(num_bytes, data->length, lengths); - }; - return Status::OK(); - } + if (*null_count > 0) { + ARROW_ASSIGN_OR_RAISE(*null_buf, AllocateBitmap(length, pool)); - Status Visit(const DataType& type) { - return Status::NotImplemented("Computing encoded key lengths for type ", type); + uint8_t* nulls = (*null_buf)->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + BitUtil::SetBitTo(nulls, i, !encoded_bytes[i][0]); + encoded_bytes[i] += 1; + } + } else { + for (int32_t i = 0; i < length; ++i) { + encoded_bytes[i] += 1; + } } + return Status ::OK(); + } +}; - AddLengthImpl add_length_impl; - }; +struct BooleanKeyEncoder : KeyEncoder { + static constexpr int kByteWidth = 1; - using EncodeNextImpl = - std::function&, uint8_t**)>; - - struct GetEncodeNextImpl { - static void EncodeBoolean(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - VisitArrayDataInline( - *data, - [&](bool value) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - *encoded_ptr++ = value; - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - *encoded_ptr++ = 0; - }); - } - - static void EncodeFixed(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - auto num_bytes = checked_cast(*data->type).bit_width() / 8; - - ArrayData viewed(fixed_size_binary(num_bytes), data->length, data->buffers, - data->null_count, data->offset); - - return VisitArrayDataInline( - viewed, - [&](util::string_view bytes) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - memcpy(encoded_ptr, bytes.data(), num_bytes); - encoded_ptr += num_bytes; - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - memset(encoded_ptr, 0, num_bytes); - encoded_ptr += num_bytes; - }); - } - - template - static void EncodeVarLength(const std::shared_ptr& data, - uint8_t** encoded_bytes) { - using offset_type = typename T::offset_type; - - return VisitArrayDataInline( - *data, - [&](util::string_view bytes) { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; - util::SafeStore(encoded_ptr, static_cast(bytes.size())); - encoded_ptr += sizeof(offset_type); - memcpy(encoded_ptr, bytes.data(), bytes.size()); - encoded_ptr += bytes.size(); - }, - [&] { - auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; - util::SafeStore(encoded_ptr, static_cast(0)); - encoded_ptr += sizeof(offset_type); - }); - } - - Status Visit(const BooleanType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeBoolean(data, encoded_bytes); - }; - return Status::OK(); + void AddLength(const ArrayData& data, int32_t* lengths) override { + for (int64_t i = 0; i < data.length; ++i) { + lengths[i] += kByteWidth + kExtraByteForNull; } + } - Status Visit(const FixedWidthType&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeFixed(data, encoded_bytes); - }; - return Status::OK(); - } + void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + VisitArrayDataInline( + data, + [&](bool value) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + *encoded_ptr++ = value; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + *encoded_ptr++ = 0; + }); + } - template - enable_if_base_binary Visit(const T&) { - encode_next_impl = [](const std::shared_ptr& data, - uint8_t** encoded_bytes) { - EncodeVarLength(data, encoded_bytes); - }; - return Status::OK(); + Result> Decode(uint8_t** encoded_bytes, int32_t length, + MemoryPool* pool) override { + std::shared_ptr null_buf; + int32_t null_count; + RETURN_NOT_OK(DecodeNulls(pool, length, encoded_bytes, &null_buf, &null_count)); + + ARROW_ASSIGN_OR_RAISE(auto key_buf, AllocateBitmap(length, pool)); + + uint8_t* raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); + encoded_ptr += 1; } - Status Visit(const DataType& type) { - return Status::NotImplemented("encoding hashes for type ", type); + return ArrayData::Make(boolean(), length, {std::move(null_buf), std::move(key_buf)}, + null_count); + } +}; + +struct FixedWidthKeyEncoder : KeyEncoder { + FixedWidthKeyEncoder(int byte_width, std::shared_ptr type) + : byte_width_(byte_width), type_(std::move(type)) {} + + void AddLength(const ArrayData& data, int32_t* lengths) override { + for (int64_t i = 0; i < data.length; ++i) { + lengths[i] += byte_width_ + kExtraByteForNull; } + } - EncodeNextImpl encode_next_impl; - }; + void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + ArrayData viewed(fixed_size_binary(byte_width_), data.length, data.buffers, + data.null_count, data.offset); + + VisitArrayDataInline( + viewed, + [&](util::string_view bytes) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + memcpy(encoded_ptr, bytes.data(), byte_width_); + encoded_ptr += byte_width_; + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + memset(encoded_ptr, 0, byte_width_); + encoded_ptr += byte_width_; + }); + } - using DecodeNextImpl = std::function*)>; + Result> Decode(uint8_t** encoded_bytes, int32_t length, + MemoryPool* pool) override { + std::shared_ptr null_buf; + int32_t null_count; + RETURN_NOT_OK(DecodeNulls(pool, length, encoded_bytes, &null_buf, &null_count)); - struct GetDecodeNextImpl { - static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, int32_t* null_count) { - // Do we have nulls? - *null_count = 0; - for (int32_t i = 0; i < length; ++i) { - *null_count += encoded_bytes[i][0]; - } - if (*null_count > 0) { - ARROW_ASSIGN_OR_RAISE(*null_buf, AllocateBitmap(length, pool)); + ARROW_ASSIGN_OR_RAISE(auto key_buf, AllocateBuffer(length * byte_width_, pool)); - uint8_t* nulls = (*null_buf)->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - BitUtil::SetBitTo(nulls, i, !encoded_bytes[i][0]); - encoded_bytes[i] += 1; - } - } else { - for (int32_t i = 0; i < length; ++i) { - encoded_bytes[i] += 1; - } - } - return Status ::OK(); + uint8_t* raw_output = key_buf->mutable_data(); + for (int32_t i = 0; i < length; ++i) { + auto& encoded_ptr = encoded_bytes[i]; + std::memcpy(raw_output, encoded_ptr, byte_width_); + encoded_ptr += byte_width_; + raw_output += byte_width_; } - static void DecodeBoolean(KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, - &null_buf, &null_count)); + return ArrayData::Make(type_, length, {std::move(null_buf), std::move(key_buf)}, + null_count); + } - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->AllocateBitmap(length)); + int byte_width_; + std::shared_ptr type_; +}; - uint8_t* raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - BitUtil::SetBitTo(raw_output, i, encoded_ptr[0] != 0); - encoded_ptr += 1; - } +template +struct VarLengthKeyEncoder : KeyEncoder { + using Offset = typename T::offset_type; + + void AddLength(const ArrayData& data, int32_t* lengths) override { + int64_t i = 0; + VisitArrayDataInline( + data, + [&](util::string_view bytes) { + lengths[i++] += kExtraByteForNull + sizeof(Offset) + bytes.size(); + }, + [&] { lengths[i++] += kExtraByteForNull + sizeof(Offset); }); + } - *out = ArrayData::Make(boolean(), length, {std::move(null_buf), std::move(key_buf)}, - null_count); - } + void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + return VisitArrayDataInline( + data, + [&](util::string_view bytes) { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 0; + util::SafeStore(encoded_ptr, static_cast(bytes.size())); + encoded_ptr += sizeof(Offset); + memcpy(encoded_ptr, bytes.data(), bytes.size()); + encoded_ptr += bytes.size(); + }, + [&] { + auto& encoded_ptr = *encoded_bytes++; + *encoded_ptr++ = 1; + util::SafeStore(encoded_ptr, static_cast(0)); + encoded_ptr += sizeof(Offset); + }); + } - static void DecodeFixed(KernelContext* ctx, std::shared_ptr output_type, - int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, - &null_buf, &null_count)); + Result> Decode(uint8_t** encoded_bytes, int32_t length, + MemoryPool* pool) override { + std::shared_ptr null_buf; + int32_t null_count; + RETURN_NOT_OK(DecodeNulls(pool, length, encoded_bytes, &null_buf, &null_count)); - auto num_bytes = checked_cast(*output_type).bit_width() / 8; - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(num_bytes * length)); + Offset length_sum = 0; + for (int32_t i = 0; i < length; ++i) { + length_sum += reinterpret_cast(encoded_bytes)[0]; + } - uint8_t* raw_output = key_buf->mutable_data(); - for (int32_t i = 0; i < length; ++i) { - auto& encoded_ptr = encoded_bytes[i]; - std::memcpy(raw_output, encoded_ptr, num_bytes); - encoded_ptr += num_bytes; - raw_output += num_bytes; - } + ARROW_ASSIGN_OR_RAISE(auto offset_buf, + AllocateBuffer(sizeof(Offset) * (1 + length), pool)); + ARROW_ASSIGN_OR_RAISE(auto key_buf, AllocateBuffer(length_sum)); - *out = ArrayData::Make(std::move(output_type), length, - {std::move(null_buf), std::move(key_buf)}, null_count); + auto raw_offsets = reinterpret_cast(offset_buf->mutable_data()); + auto raw_keys = key_buf->mutable_data(); + + int32_t current_offset = 0; + for (int32_t i = 0; i < length; ++i) { + auto key_length = util::SafeLoadAs(encoded_bytes[i]); + raw_offsets[i] = current_offset; + encoded_bytes[i] += sizeof(Offset); + memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); + encoded_bytes[i] += key_length; + current_offset += key_length; } + raw_offsets[length] = current_offset; - template - static void DecodeVarLength(KernelContext* ctx, std::shared_ptr output_type, - int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - std::shared_ptr null_buf; - int32_t null_count; - KERNEL_RETURN_IF_ERROR(ctx, DecodeNulls(ctx->memory_pool(), length, encoded_bytes, - &null_buf, &null_count)); + return ArrayData::Make( + type_, length, {std::move(null_buf), std::move(offset_buf), std::move(key_buf)}, + null_count); + } - using offset_type = typename T::offset_type; + explicit VarLengthKeyEncoder(std::shared_ptr type) : type_(std::move(type)) {} - offset_type length_sum = 0; - for (int32_t i = 0; i < length; ++i) { - length_sum += reinterpret_cast(encoded_bytes)[0]; - } + std::shared_ptr type_; +}; + +Result MakeKernel(GroupByOptions::Aggregate); - KERNEL_ASSIGN_OR_RAISE(auto offset_buf, ctx, - ctx->Allocate(sizeof(offset_type) * (1 + length))); - KERNEL_ASSIGN_OR_RAISE(auto key_buf, ctx, ctx->Allocate(length_sum)); +struct GroupByImpl { + static Result Make(ExecContext* ctx, const std::vector& aggregands, + const std::vector& keys, + const GroupByOptions& options) { + GroupByImpl impl; + impl.options_ = options; + const auto& aggregates = impl.options_.aggregates; - auto raw_offsets = reinterpret_cast(offset_buf->mutable_data()); - auto raw_keys = key_buf->mutable_data(); + impl.num_groups_ = 0; + impl.offsets_.push_back(0); - int32_t current_offset = 0; - for (int32_t i = 0; i < length; ++i) { - offset_type key_length = reinterpret_cast(encoded_bytes[i])[0]; - raw_offsets[i] = current_offset; - encoded_bytes[i] += sizeof(offset_type); - memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); - encoded_bytes[i] += key_length; - current_offset += key_length; + if (aggregates.size() != aggregands.size()) { + return Status::Invalid(aggregates.size(), + " aggregate functions were specified but ", + aggregands.size(), " aggregands were provided."); + } + + FieldVector out_fields; + + impl.aggregators_.resize(aggregates.size()); + impl.aggregator_states_.resize(aggregates.size()); + + for (size_t i = 0; i < aggregates.size(); ++i) { + auto a = aggregates[i]; + + if (a.options == nullptr) { + // use known default options for the named function if possible + auto maybe_function = ctx->func_registry()->GetFunction(a.function); + if (maybe_function.ok()) { + a.options = maybe_function.ValueOrDie()->default_options(); + } } - raw_offsets[length] = current_offset; - *out = ArrayData::Make( - std::move(output_type), length, - {std::move(null_buf), std::move(offset_buf), std::move(key_buf)}, null_count); - } + ARROW_ASSIGN_OR_RAISE(impl.aggregators_[i], MakeKernel(a)); - Status Visit(const BooleanType&) { - decode_next_impl = [](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeBoolean(ctx, length, encoded_bytes, out); - }; - return Status::OK(); - } + KernelContext kernel_ctx{ctx}; + impl.aggregator_states_[i] = impl.aggregators_[i].init( + &kernel_ctx, + KernelInitArgs{&impl.aggregators_[i], {aggregands[i].type()}, a.options}); + if (kernel_ctx.HasError()) return kernel_ctx.status(); - Status Visit(const FixedWidthType&) { - auto output_type = out_type; - decode_next_impl = [=](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeFixed(ctx, output_type, length, encoded_bytes, out); - }; - return Status::OK(); + kernel_ctx.SetState(impl.aggregator_states_[i].get()); + ARROW_ASSIGN_OR_RAISE(auto descr, + impl.aggregators_[i].signature->out_type().Resolve( + &kernel_ctx, { + aggregands[i].type(), + uint32(), + uint32(), + })); + out_fields.push_back(field("", descr.type)); } - template - enable_if_base_binary Visit(const T&) { - auto output_type = out_type; - decode_next_impl = [=](KernelContext* ctx, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* out) { - DecodeVarLength(ctx, output_type, length, encoded_bytes, out); - }; - return Status::OK(); - } + impl.encoders_.resize(keys.size()); + for (size_t i = 0; i < keys.size(); ++i) { + const auto& key = keys[i].type(); - Status Visit(const DataType& type) { - return Status::NotImplemented("decoding keys for type ", type); + out_fields.push_back(field("", key)); + + if (key->id() == Type::BOOL) { + impl.encoders_[i] = ::arrow::internal::make_unique(); + continue; + } + + if (auto byte_width = bit_width(key->id()) / 8) { + impl.encoders_[i] = + ::arrow::internal::make_unique(byte_width, key); + continue; + } + + if (is_binary_like(key->id())) { + impl.encoders_[i] = + ::arrow::internal::make_unique>(key); + continue; + } + + if (is_large_binary_like(key->id())) { + impl.encoders_[i] = + ::arrow::internal::make_unique>(key); + continue; + } + + return Status::NotImplemented("Keys of type ", *key); } - std::shared_ptr out_type; - DecodeNextImpl decode_next_impl; - }; + impl.out_type_ = struct_(std::move(out_fields)); + return impl; + } - // TODO(bkietz) Here I'd like to make it clear that Consume produces a batch of - // group_ids; extract the immediate pass-to-HashAggregateKernel. + // TODO(bkietz) Here I'd like to make this clearly a node which classifies keys; extract + // the immediate pass-to-HashAggregateKernel. void Consume(KernelContext* ctx, const ExecBatch& batch) { ArrayDataVector aggregands, keys; size_t i; - for (i = 0; i < aggregators.size(); ++i) { + for (i = 0; i < aggregators_.size(); ++i) { aggregands.push_back(batch[i].array()); } + while (i < static_cast(batch.num_values())) { keys.push_back(batch[i++].array()); } @@ -344,8 +351,9 @@ struct GroupByImpl { offsets_batch_[0] = 0; memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); for (size_t i = 0; i < keys.size(); ++i) { - add_length_impl[i].add_length_impl(keys[i], offsets_batch_.data()); + encoders_[i]->AddLength(*keys[i], offsets_batch_.data()); } + int32_t total_length = 0; for (int64_t i = 0; i < batch.length; ++i) { auto total_length_before = total_length; @@ -361,8 +369,9 @@ struct GroupByImpl { for (int64_t i = 0; i < batch.length; ++i) { key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; } + for (size_t i = 0; i < keys.size(); ++i) { - encode_next_impl[i].encode_next_impl(keys[i], key_buf_ptrs_.data()); + encoders_[i]->Encode(*keys[i], key_buf_ptrs_.data()); } group_ids_batch_.clear(); @@ -372,58 +381,59 @@ struct GroupByImpl { std::string key( reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), key_length); - auto iter = map_.find(key); - if (iter == map_.end()) { - group_ids_batch_[i] = n_groups++; + + auto it_success = map_.emplace(key, num_groups_); + if (it_success.second) { + // new key; update offsets and key_bytes + ++num_groups_; auto next_key_offset = static_cast(key_bytes_.size()); key_bytes_.resize(next_key_offset + key_length); offsets_.push_back(next_key_offset + key_length); memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); - map_.insert(std::make_pair(key, group_ids_batch_[i])); - } else { - group_ids_batch_[i] = iter->second; } + group_ids_batch_[i] = it_success.first->second; } UInt32Array group_ids(batch.length, Buffer::Wrap(group_ids_batch_)); - for (size_t i = 0; i < aggregators.size(); ++i) { + for (size_t i = 0; i < aggregators_.size(); ++i) { KernelContext batch_ctx{ctx->exec_context()}; - batch_ctx.SetState(aggregator_states[i].get()); - ExecBatch batch({aggregands[i], group_ids, Datum(n_groups)}, group_ids.length()); - aggregators[i].consume(&batch_ctx, batch); + batch_ctx.SetState(aggregator_states_[i].get()); + ExecBatch batch({aggregands[i], group_ids, Datum(num_groups_)}, group_ids.length()); + aggregators_[i].consume(&batch_ctx, batch); ctx->SetStatus(batch_ctx.status()); if (ctx->HasError()) return; } } void Finalize(KernelContext* ctx, Datum* out) { - size_t n_keys = decode_next_impl.size(); - ArrayDataVector out_columns(aggregators.size() + n_keys); - for (size_t i = 0; i < aggregators.size(); ++i) { + size_t n_keys = encoders_.size(); + ArrayDataVector out_columns(aggregators_.size() + n_keys); + for (size_t i = 0; i < aggregators_.size(); ++i) { KernelContext batch_ctx{ctx->exec_context()}; - batch_ctx.SetState(aggregator_states[i].get()); + batch_ctx.SetState(aggregator_states_[i].get()); Datum aggregated; - aggregators[i].finalize(&batch_ctx, &aggregated); + aggregators_[i].finalize(&batch_ctx, &aggregated); ctx->SetStatus(batch_ctx.status()); if (ctx->HasError()) return; out_columns[i] = aggregated.array(); } key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(n_groups); - for (int64_t i = 0; i < n_groups; ++i) { + key_buf_ptrs_.resize(num_groups_); + for (int64_t i = 0; i < num_groups_; ++i) { key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; } - int64_t length = n_groups; + int64_t length = num_groups_; for (size_t i = 0; i < n_keys; ++i) { - std::shared_ptr key_array; - decode_next_impl[i].decode_next_impl(ctx, static_cast(length), - key_buf_ptrs_.data(), &key_array); - out_columns[aggregators.size() + i] = std::move(key_array); + KERNEL_ASSIGN_OR_RAISE( + auto key_array, ctx, + encoders_[i]->Decode(key_buf_ptrs_.data(), static_cast(length), + ctx->memory_pool())); + out_columns[aggregators_.size() + i] = std::move(key_array); } - *out = ArrayData::Make(std::move(out_type), length, {/*null_bitmap=*/nullptr}, + *out = ArrayData::Make(std::move(out_type_), length, {/*null_bitmap=*/nullptr}, std::move(out_columns)); } @@ -435,107 +445,16 @@ struct GroupByImpl { std::unordered_map map_; std::vector offsets_; std::vector key_bytes_; - uint32_t n_groups; + uint32_t num_groups_; - std::shared_ptr out_type; - GroupByOptions options; - std::vector aggregators; - std::vector> aggregator_states; + std::shared_ptr out_type_; + GroupByOptions options_; + std::vector aggregators_; + std::vector> aggregator_states_; - std::vector add_length_impl; - std::vector encode_next_impl; - std::vector decode_next_impl; + std::vector> encoders_; }; -Result MakeKernel(GroupByOptions::Aggregate); - -Result GroupByInit(ExecContext* ctx, const std::vector& aggregands, - const std::vector& keys, - const GroupByOptions& options) { - GroupByImpl impl; - impl.options = options; - const auto& aggregates = impl.options.aggregates; - - impl.n_groups = 0; - impl.offsets_.push_back(0); - - if (aggregates.size() != aggregands.size()) { - return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", - aggregands.size(), " aggregands were provided."); - } - - FieldVector out_fields; - - impl.aggregators.resize(aggregates.size()); - impl.aggregator_states.resize(aggregates.size()); - - for (size_t i = 0; i < aggregates.size(); ++i) { - auto a = aggregates[i]; - - if (a.options == nullptr) { - // use known default options for the named function if possible - auto maybe_function = ctx->func_registry()->GetFunction(a.function); - if (maybe_function.ok()) { - a.options = maybe_function.ValueOrDie()->default_options(); - } - } - - ARROW_ASSIGN_OR_RAISE(impl.aggregators[i], MakeKernel(a)); - - KernelContext kernel_ctx{ctx}; - impl.aggregator_states[i] = impl.aggregators[i].init( - &kernel_ctx, - KernelInitArgs{&impl.aggregators[i], {aggregands[i].type()}, a.options}); - if (kernel_ctx.HasError()) return kernel_ctx.status(); - - kernel_ctx.SetState(impl.aggregator_states[i].get()); - ARROW_ASSIGN_OR_RAISE(auto descr, impl.aggregators[i].signature->out_type().Resolve( - &kernel_ctx, { - aggregands[i].type(), - uint32(), - uint32(), - })); - out_fields.push_back(field("", descr.type)); - } - - for (const auto& key : keys) { - const auto& key_type = key.type(); - switch (key_type->id()) { - // Supported types of keys - case Type::BOOL: - case Type::UINT8: - case Type::INT8: - case Type::UINT16: - case Type::INT16: - case Type::UINT32: - case Type::INT32: - case Type::UINT64: - case Type::INT64: - case Type::STRING: - case Type::BINARY: - case Type::FIXED_SIZE_BINARY: - break; - default: - return Status::NotImplemented("Key of type ", key_type->ToString()); - } - out_fields.push_back(field("", key_type)); - } - - impl.add_length_impl.resize(keys.size()); - impl.encode_next_impl.resize(keys.size()); - impl.decode_next_impl.resize(keys.size()); - for (size_t i = 0; i < keys.size(); ++i) { - const auto& key_type = keys[i].type(); - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.add_length_impl[i])); - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.encode_next_impl[i])); - impl.decode_next_impl[i].out_type = key_type; - RETURN_NOT_OK(VisitTypeInline(*key_type, &impl.decode_next_impl[i])); - } - - impl.out_type = struct_(std::move(out_fields)); - return impl; -} - /// C++ abstract base class for the HashAggregateKernel interface. /// Implementations should be default constructible and perform initialization in /// Init(). @@ -927,7 +846,7 @@ Result GroupBy(const std::vector& aggregands, } ARROW_ASSIGN_OR_RAISE(auto impl, - aggregate::GroupByInit(ctx, aggregands, keys, options)); + aggregate::GroupByImpl::Make(ctx, aggregands, keys, options)); ARROW_ASSIGN_OR_RAISE(auto batch_iterator, detail::ExecBatchIterator::Make(args, ctx->exec_chunksize())); From 2f939f57acc6baea87e2aac9cb4fa1719ee9516b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 12:31:23 -0400 Subject: [PATCH 24/49] separate group identification from aggregation --- cpp/src/arrow/compute/exec.h | 28 ++ .../arrow/compute/kernels/hash_aggregate.cc | 340 +++++++++--------- 2 files changed, 208 insertions(+), 160 deletions(-) diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index f491489ed8a..391dab3fa6a 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -169,6 +169,34 @@ struct ExecBatch { ExecBatch(std::vector values, int64_t length) : values(std::move(values)), length(length) {} + static Result Make(std::vector values) { + if (values.empty()) { + return Status::Invalid("Cannot infer ExecBatch length without at least one value"); + } + + int64_t length = -1; + for (const auto& value : values) { + if (value.is_scalar()) { + if (length == -1) { + length = 1; + } + continue; + } + + if (length == -1) { + length = value.length(); + continue; + } + + if (length != value.length()) { + return Status::Invalid( + "Arrays used to construct an ExecBatch must have equal length"); + } + } + + return ExecBatch(std::move(values), length); + } + /// The values representing positional arguments to be passed to a kernel's /// exec function for processing. std::vector values; diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 2bc1351613b..b2832e91989 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -242,65 +242,18 @@ struct VarLengthKeyEncoder : KeyEncoder { std::shared_ptr type_; }; -Result MakeKernel(GroupByOptions::Aggregate); - struct GroupByImpl { - static Result Make(ExecContext* ctx, const std::vector& aggregands, - const std::vector& keys, + static Result Make(ExecContext* ctx, const std::vector& keys, const GroupByOptions& options) { GroupByImpl impl; - impl.options_ = options; - const auto& aggregates = impl.options_.aggregates; - - impl.num_groups_ = 0; - impl.offsets_.push_back(0); - - if (aggregates.size() != aggregands.size()) { - return Status::Invalid(aggregates.size(), - " aggregate functions were specified but ", - aggregands.size(), " aggregands were provided."); - } - - FieldVector out_fields; - - impl.aggregators_.resize(aggregates.size()); - impl.aggregator_states_.resize(aggregates.size()); - - for (size_t i = 0; i < aggregates.size(); ++i) { - auto a = aggregates[i]; - - if (a.options == nullptr) { - // use known default options for the named function if possible - auto maybe_function = ctx->func_registry()->GetFunction(a.function); - if (maybe_function.ok()) { - a.options = maybe_function.ValueOrDie()->default_options(); - } - } + impl.encoders_.resize(keys.size()); + impl.ctx_ = ctx; - ARROW_ASSIGN_OR_RAISE(impl.aggregators_[i], MakeKernel(a)); - - KernelContext kernel_ctx{ctx}; - impl.aggregator_states_[i] = impl.aggregators_[i].init( - &kernel_ctx, - KernelInitArgs{&impl.aggregators_[i], {aggregands[i].type()}, a.options}); - if (kernel_ctx.HasError()) return kernel_ctx.status(); - - kernel_ctx.SetState(impl.aggregator_states_[i].get()); - ARROW_ASSIGN_OR_RAISE(auto descr, - impl.aggregators_[i].signature->out_type().Resolve( - &kernel_ctx, { - aggregands[i].type(), - uint32(), - uint32(), - })); - out_fields.push_back(field("", descr.type)); - } + FieldVector out_fields(keys.size()); - impl.encoders_.resize(keys.size()); for (size_t i = 0; i < keys.size(); ++i) { - const auto& key = keys[i].type(); - - out_fields.push_back(field("", key)); + const auto& key = keys[i].type; + out_fields[i] = field("", key); if (key->id() == Type::BOOL) { impl.encoders_[i] = ::arrow::internal::make_unique(); @@ -332,54 +285,37 @@ struct GroupByImpl { return impl; } - // TODO(bkietz) Here I'd like to make this clearly a node which classifies keys; extract - // the immediate pass-to-HashAggregateKernel. - void Consume(KernelContext* ctx, const ExecBatch& batch) { - ArrayDataVector aggregands, keys; - - size_t i; - for (i = 0; i < aggregators_.size(); ++i) { - aggregands.push_back(batch[i].array()); - } - - while (i < static_cast(batch.num_values())) { - keys.push_back(batch[i++].array()); - } - - offsets_batch_.clear(); - offsets_batch_.resize(batch.length + 1); - offsets_batch_[0] = 0; - memset(offsets_batch_.data(), 0, sizeof(offsets_batch_[0]) * offsets_batch_.size()); - for (size_t i = 0; i < keys.size(); ++i) { - encoders_[i]->AddLength(*keys[i], offsets_batch_.data()); + Result Consume(const ExecBatch& batch) { + std::vector offsets_batch(batch.length + 1); + for (int i = 0; i < batch.num_values(); ++i) { + encoders_[i]->AddLength(*batch[i].array(), offsets_batch.data()); } int32_t total_length = 0; for (int64_t i = 0; i < batch.length; ++i) { auto total_length_before = total_length; - total_length += offsets_batch_[i]; - offsets_batch_[i] = total_length_before; + total_length += offsets_batch[i]; + offsets_batch[i] = total_length_before; } - offsets_batch_[batch.length] = total_length; + offsets_batch[batch.length] = total_length; - key_bytes_batch_.clear(); - key_bytes_batch_.resize(total_length); - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(batch.length); + std::vector key_bytes_batch(total_length); + std::vector key_buf_ptrs(batch.length); for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs_[i] = key_bytes_batch_.data() + offsets_batch_[i]; + key_buf_ptrs[i] = key_bytes_batch.data() + offsets_batch[i]; } - for (size_t i = 0; i < keys.size(); ++i) { - encoders_[i]->Encode(*keys[i], key_buf_ptrs_.data()); + for (int i = 0; i < batch.num_values(); ++i) { + encoders_[i]->Encode(*batch[i].array(), key_buf_ptrs.data()); } - group_ids_batch_.clear(); - group_ids_batch_.resize(batch.length); + TypedBufferBuilder group_ids_batch(ctx_->memory_pool()); + RETURN_NOT_OK(group_ids_batch.Resize(batch.length)); + for (int64_t i = 0; i < batch.length; ++i) { - int32_t key_length = offsets_batch_[i + 1] - offsets_batch_[i]; + int32_t key_length = offsets_batch[i + 1] - offsets_batch[i]; std::string key( - reinterpret_cast(key_bytes_batch_.data() + offsets_batch_[i]), + reinterpret_cast(key_bytes_batch.data() + offsets_batch[i]), key_length); auto it_success = map_.emplace(key, num_groups_); @@ -391,66 +327,41 @@ struct GroupByImpl { offsets_.push_back(next_key_offset + key_length); memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); } - group_ids_batch_[i] = it_success.first->second; + group_ids_batch.UnsafeAppend(it_success.first->second); } - UInt32Array group_ids(batch.length, Buffer::Wrap(group_ids_batch_)); - for (size_t i = 0; i < aggregators_.size(); ++i) { - KernelContext batch_ctx{ctx->exec_context()}; - batch_ctx.SetState(aggregator_states_[i].get()); - ExecBatch batch({aggregands[i], group_ids, Datum(num_groups_)}, group_ids.length()); - aggregators_[i].consume(&batch_ctx, batch); - ctx->SetStatus(batch_ctx.status()); - if (ctx->HasError()) return; - } + ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); + return ExecBatch( + {UInt32Array(batch.length, std::move(group_ids)), Datum(num_groups_)}, + batch.length); } - void Finalize(KernelContext* ctx, Datum* out) { - size_t n_keys = encoders_.size(); - ArrayDataVector out_columns(aggregators_.size() + n_keys); - for (size_t i = 0; i < aggregators_.size(); ++i) { - KernelContext batch_ctx{ctx->exec_context()}; - batch_ctx.SetState(aggregator_states_[i].get()); - Datum aggregated; - aggregators_[i].finalize(&batch_ctx, &aggregated); - ctx->SetStatus(batch_ctx.status()); - if (ctx->HasError()) return; - out_columns[i] = aggregated.array(); - } + Result Finalize(ExecContext* ctx) { + ArrayDataVector out_columns(encoders_.size()); - key_buf_ptrs_.clear(); - key_buf_ptrs_.resize(num_groups_); + std::vector key_buf_ptrs(num_groups_); for (int64_t i = 0; i < num_groups_; ++i) { - key_buf_ptrs_[i] = key_bytes_.data() + offsets_[i]; + key_buf_ptrs[i] = key_bytes_.data() + offsets_[i]; } - int64_t length = num_groups_; - for (size_t i = 0; i < n_keys; ++i) { - KERNEL_ASSIGN_OR_RAISE( - auto key_array, ctx, - encoders_[i]->Decode(key_buf_ptrs_.data(), static_cast(length), + for (size_t i = 0; i < encoders_.size(); ++i) { + ARROW_ASSIGN_OR_RAISE( + out_columns[i], + encoders_[i]->Decode(key_buf_ptrs.data(), static_cast(num_groups_), ctx->memory_pool())); - out_columns[aggregators_.size() + i] = std::move(key_array); } - *out = ArrayData::Make(std::move(out_type_), length, {/*null_bitmap=*/nullptr}, + return ArrayData::Make(std::move(out_type_), num_groups_, {/*null_bitmap=*/nullptr}, std::move(out_columns)); } - std::vector offsets_batch_; - std::vector key_bytes_batch_; - std::vector key_buf_ptrs_; - std::vector group_ids_batch_; - + ExecContext* ctx_; std::unordered_map map_; - std::vector offsets_; + std::vector offsets_ = {0}; std::vector key_bytes_; - uint32_t num_groups_; + uint32_t num_groups_ = 0; std::shared_ptr out_type_; - GroupByOptions options_; - std::vector aggregators_; - std::vector> aggregator_states_; std::vector> encoders_; }; @@ -782,31 +693,31 @@ struct GroupedMinMaxImpl : public GroupedAggregator { }; template -KernelInit MakeInit(GroupByOptions::Aggregate a) { - return [a](KernelContext* ctx, - const KernelInitArgs& args) -> std::unique_ptr { +KernelInit MakeInit() { + return [](KernelContext* ctx, + const KernelInitArgs& args) -> std::unique_ptr { auto impl = ::arrow::internal::make_unique(); - ctx->SetStatus(impl->Init(ctx->exec_context(), a.options, args.inputs[0].type)); + ctx->SetStatus(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); if (ctx->HasError()) return nullptr; return impl; }; } -Result MakeKernel(GroupByOptions::Aggregate a) { +// this isn't really in the spirit of things, but I'll get around to defining +// HashAggregateFunctions later +Result MakeKernel(const std::string& function_name) { HashAggregateKernel kernel; - if (a.function == "count") { - kernel.init = MakeInit(a); - } else if (a.function == "sum") { - kernel.init = MakeInit(a); - } else if (a.function == "min_max") { - kernel.init = MakeInit(a); + if (function_name == "count") { + kernel.init = MakeInit(); + } else if (function_name == "sum") { + kernel.init = MakeInit(); + } else if (function_name == "min_max") { + kernel.init = MakeInit(); } else { - return Status::NotImplemented("Grouped aggregate ", a.function); + return Status::NotImplemented("Grouped aggregate ", function_name); } - // this isn't really in the spirit of things, but I'll get around to defining - // HashAggregateFunctions later kernel.signature = KernelSignature::Make( {{}, {}, {}}, OutputType([](KernelContext* ctx, const std::vector&) -> Result { @@ -830,6 +741,68 @@ Result MakeKernel(GroupByOptions::Aggregate a) { return kernel; } +Result> MakeKernels( + const std::vector& aggregates, + const std::vector& in_descrs) { + if (aggregates.size() != in_descrs.size()) { + return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", + in_descrs.size(), " aggregands were provided."); + } + + std::vector kernels(in_descrs.size()); + + for (size_t i = 0; i < aggregates.size(); ++i) { + ARROW_ASSIGN_OR_RAISE(kernels[i], MakeKernel(aggregates[i].function)); + } + return kernels; +} + +Result>> InitKernels( + const std::vector& kernels, ExecContext* ctx, + const std::vector& aggregates, + const std::vector& in_descrs) { + std::vector> states(kernels.size()); + + for (size_t i = 0; i < aggregates.size(); ++i) { + auto options = aggregates[i].options; + + if (options == nullptr) { + // use known default options for the named function if possible + auto maybe_function = ctx->func_registry()->GetFunction(aggregates[i].function); + if (maybe_function.ok()) { + options = maybe_function.ValueOrDie()->default_options(); + } + } + + KernelContext kernel_ctx{ctx}; + states[i] = kernels[i].init( + &kernel_ctx, KernelInitArgs{&kernels[i], {in_descrs[i].type}, options}); + if (kernel_ctx.HasError()) return kernel_ctx.status(); + } + return states; +} + +Result ResolveKernels( + const std::vector& kernels, + const std::vector>& states, ExecContext* ctx, + const std::vector& descrs) { + FieldVector fields(descrs.size()); + + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext kernel_ctx{ctx}; + kernel_ctx.SetState(states[i].get()); + + ARROW_ASSIGN_OR_RAISE(auto descr, kernels[i].signature->out_type().Resolve( + &kernel_ctx, { + descrs[i].type, + uint32(), + uint32(), + })); + fields[i] = field("", std::move(descr.type)); + } + return fields; +} + } // namespace aggregate Result GroupBy(const std::vector& aggregands, @@ -840,31 +813,78 @@ Result GroupBy(const std::vector& aggregands, return GroupBy(aggregands, keys, options, &default_ctx); } - std::vector args = aggregands; - for (const Datum& key : keys) { - args.push_back(key); - } + ARROW_ASSIGN_OR_RAISE(auto aggregand_descrs, + ExecBatch::Make(aggregands).Map([](ExecBatch batch) { + return batch.GetDescriptors(); + })); + + ARROW_ASSIGN_OR_RAISE(auto kernels, + aggregate::MakeKernels(options.aggregates, aggregand_descrs)); + + ARROW_ASSIGN_OR_RAISE( + auto states, + aggregate::InitKernels(kernels, ctx, options.aggregates, aggregand_descrs)); + + ARROW_ASSIGN_OR_RAISE( + FieldVector out_fields, + aggregate::ResolveKernels(kernels, states, ctx, aggregand_descrs)); + + ARROW_ASSIGN_OR_RAISE( + auto aggregand_batch_iterator, + detail::ExecBatchIterator::Make(aggregands, ctx->exec_chunksize())); + + ARROW_ASSIGN_OR_RAISE(auto key_descrs, ExecBatch::Make(keys).Map([](ExecBatch batch) { + return batch.GetDescriptors(); + })); ARROW_ASSIGN_OR_RAISE(auto impl, - aggregate::GroupByImpl::Make(ctx, aggregands, keys, options)); + aggregate::GroupByImpl::Make(ctx, key_descrs, options)); - ARROW_ASSIGN_OR_RAISE(auto batch_iterator, - detail::ExecBatchIterator::Make(args, ctx->exec_chunksize())); + for (ValueDescr& key_descr : key_descrs) { + out_fields.push_back(field("", std::move(key_descr.type))); + } + + ARROW_ASSIGN_OR_RAISE(auto key_batch_iterator, + detail::ExecBatchIterator::Make(keys, ctx->exec_chunksize())); + + // start "streaming" execution + ExecBatch key_batch, aggregand_batch; + while (aggregand_batch_iterator->Next(&aggregand_batch) && + key_batch_iterator->Next(&key_batch)) { + if (key_batch.length == 0) continue; - KernelContext kernel_ctx{ctx}; + // compute a batch of group ids + ARROW_ASSIGN_OR_RAISE(ExecBatch id_batch, impl.Consume(key_batch)); - ExecBatch batch; - while (batch_iterator->Next(&batch)) { - if (batch.length > 0) { - impl.Consume(&kernel_ctx, batch); - if (kernel_ctx.HasError()) return kernel_ctx.status(); + // consume group ids with HashAggregateKernels + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[i].get()); + ARROW_ASSIGN_OR_RAISE( + auto batch, ExecBatch::Make({aggregand_batch[i], id_batch[0], id_batch[1]})); + kernels[i].consume(&batch_ctx, batch); + if (batch_ctx.HasError()) return batch_ctx.status(); } } - Datum out; - impl.Finalize(&kernel_ctx, &out); - if (kernel_ctx.HasError()) return kernel_ctx.status(); - return out; + ArrayDataVector out_data(aggregands.size() + keys.size()); + for (size_t i = 0; i < kernels.size(); ++i) { + KernelContext batch_ctx{ctx}; + batch_ctx.SetState(states[i].get()); + Datum out; + kernels[i].finalize(&batch_ctx, &out); + if (batch_ctx.HasError()) return batch_ctx.status(); + out_data[i] = out.array(); + } + + ARROW_ASSIGN_OR_RAISE(Datum out_keys, impl.Finalize(ctx)); + std::move(out_keys.array()->child_data.begin(), out_keys.array()->child_data.end(), + out_data.begin() + aggregands.size()); + + int64_t length = out_data[0]->length; + return ArrayData::Make(struct_(std::move(out_fields)), length, + {/*null_bitmap=*/nullptr}, std::move(out_data), + /*null_count=*/0); } } // namespace compute From 2651dfda6d669be0ec18f46ff0d6e01b52bad2fc Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 13:59:22 -0400 Subject: [PATCH 25/49] extract GroupIdentifier for reuse --- .../arrow/compute/kernels/hash_aggregate.cc | 117 ++++++++++-------- .../compute/kernels/hash_aggregate_internal.h | 52 ++++---- .../compute/kernels/hash_aggregate_test.cc | 85 +++++++------ 3 files changed, 136 insertions(+), 118 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index b2832e91989..c0eac99d6ed 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -17,21 +17,30 @@ #include "arrow/compute/kernels/hash_aggregate_internal.h" +#include +#include #include +#include +#include "arrow/buffer_builder.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/exec_internal.h" -#include "arrow/compute/kernels/aggregate_basic_internal.h" +#include "arrow/compute/kernel.h" #include "arrow/compute/kernels/aggregate_internal.h" #include "arrow/compute/kernels/common.h" #include "arrow/util/bit_run_reader.h" #include "arrow/util/bitmap_ops.h" -#include "arrow/util/cpu_info.h" +#include "arrow/util/checked_cast.h" #include "arrow/util/make_unique.h" +#include "arrow/visitor_inline.h" namespace arrow { + +using internal::checked_cast; + namespace compute { -namespace aggregate { +namespace internal { +namespace { struct KeyEncoder { // the first byte of an encoded key is used to indicate nullity @@ -242,50 +251,46 @@ struct VarLengthKeyEncoder : KeyEncoder { std::shared_ptr type_; }; -struct GroupByImpl { - static Result Make(ExecContext* ctx, const std::vector& keys, - const GroupByOptions& options) { - GroupByImpl impl; - impl.encoders_.resize(keys.size()); - impl.ctx_ = ctx; +struct GroupIdentifierImpl : GroupIdentifier { + static Result> Make( + ExecContext* ctx, const std::vector& keys) { + auto impl = ::arrow::internal::make_unique(); - FieldVector out_fields(keys.size()); + impl->encoders_.resize(keys.size()); + impl->ctx_ = ctx; for (size_t i = 0; i < keys.size(); ++i) { const auto& key = keys[i].type; - out_fields[i] = field("", key); if (key->id() == Type::BOOL) { - impl.encoders_[i] = ::arrow::internal::make_unique(); + impl->encoders_[i] = ::arrow::internal::make_unique(); continue; } if (auto byte_width = bit_width(key->id()) / 8) { - impl.encoders_[i] = + impl->encoders_[i] = ::arrow::internal::make_unique(byte_width, key); continue; } if (is_binary_like(key->id())) { - impl.encoders_[i] = + impl->encoders_[i] = ::arrow::internal::make_unique>(key); continue; } if (is_large_binary_like(key->id())) { - impl.encoders_[i] = + impl->encoders_[i] = ::arrow::internal::make_unique>(key); continue; } return Status::NotImplemented("Keys of type ", *key); } - - impl.out_type_ = struct_(std::move(out_fields)); return impl; } - Result Consume(const ExecBatch& batch) { + Result Consume(const ExecBatch& batch) override { std::vector offsets_batch(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { encoders_[i]->AddLength(*batch[i].array(), offsets_batch.data()); @@ -336,23 +341,23 @@ struct GroupByImpl { batch.length); } - Result Finalize(ExecContext* ctx) { - ArrayDataVector out_columns(encoders_.size()); + Result GetUniqueKeys() override { + ExecBatch out({}, num_groups_); std::vector key_buf_ptrs(num_groups_); for (int64_t i = 0; i < num_groups_; ++i) { key_buf_ptrs[i] = key_bytes_.data() + offsets_[i]; } + out.values.resize(encoders_.size()); for (size_t i = 0; i < encoders_.size(); ++i) { ARROW_ASSIGN_OR_RAISE( - out_columns[i], + out.values[i], encoders_[i]->Decode(key_buf_ptrs.data(), static_cast(num_groups_), - ctx->memory_pool())); + ctx_->memory_pool())); } - return ArrayData::Make(std::move(out_type_), num_groups_, {/*null_bitmap=*/nullptr}, - std::move(out_columns)); + return out; } ExecContext* ctx_; @@ -360,9 +365,6 @@ struct GroupByImpl { std::vector offsets_ = {0}; std::vector key_bytes_; uint32_t num_groups_ = 0; - - std::shared_ptr out_type_; - std::vector> encoders_; }; @@ -742,8 +744,7 @@ Result MakeKernel(const std::string& function_name) { } Result> MakeKernels( - const std::vector& aggregates, - const std::vector& in_descrs) { + const std::vector& aggregates, const std::vector& in_descrs) { if (aggregates.size() != in_descrs.size()) { return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", in_descrs.size(), " aggregands were provided."); @@ -759,8 +760,7 @@ Result> MakeKernels( Result>> InitKernels( const std::vector& kernels, ExecContext* ctx, - const std::vector& aggregates, - const std::vector& in_descrs) { + const std::vector& aggregates, const std::vector& in_descrs) { std::vector> states(kernels.size()); for (size_t i = 0; i < aggregates.size(); ++i) { @@ -803,49 +803,53 @@ Result ResolveKernels( return fields; } -} // namespace aggregate +} // namespace + +Result> GroupIdentifier::Make( + ExecContext* ctx, const std::vector& descrs) { + return GroupIdentifierImpl::Make(ctx, descrs); +} Result GroupBy(const std::vector& aggregands, - const std::vector& keys, const GroupByOptions& options, - ExecContext* ctx) { + const std::vector& keys, + const std::vector& aggregates, ExecContext* ctx) { if (ctx == nullptr) { ExecContext default_ctx; - return GroupBy(aggregands, keys, options, &default_ctx); + return GroupBy(aggregands, keys, aggregates, &default_ctx); } + // Construct and initialize HashAggregateKernels ARROW_ASSIGN_OR_RAISE(auto aggregand_descrs, ExecBatch::Make(aggregands).Map([](ExecBatch batch) { return batch.GetDescriptors(); })); - ARROW_ASSIGN_OR_RAISE(auto kernels, - aggregate::MakeKernels(options.aggregates, aggregand_descrs)); + ARROW_ASSIGN_OR_RAISE(auto kernels, MakeKernels(aggregates, aggregand_descrs)); + + ARROW_ASSIGN_OR_RAISE(auto states, + InitKernels(kernels, ctx, aggregates, aggregand_descrs)); - ARROW_ASSIGN_OR_RAISE( - auto states, - aggregate::InitKernels(kernels, ctx, options.aggregates, aggregand_descrs)); + ARROW_ASSIGN_OR_RAISE(FieldVector out_fields, + ResolveKernels(kernels, states, ctx, aggregand_descrs)); - ARROW_ASSIGN_OR_RAISE( - FieldVector out_fields, - aggregate::ResolveKernels(kernels, states, ctx, aggregand_descrs)); + using arrow::compute::detail::ExecBatchIterator; - ARROW_ASSIGN_OR_RAISE( - auto aggregand_batch_iterator, - detail::ExecBatchIterator::Make(aggregands, ctx->exec_chunksize())); + ARROW_ASSIGN_OR_RAISE(auto aggregand_batch_iterator, + ExecBatchIterator::Make(aggregands, ctx->exec_chunksize())); + // Construct GroupIdentifier ARROW_ASSIGN_OR_RAISE(auto key_descrs, ExecBatch::Make(keys).Map([](ExecBatch batch) { return batch.GetDescriptors(); })); - ARROW_ASSIGN_OR_RAISE(auto impl, - aggregate::GroupByImpl::Make(ctx, key_descrs, options)); + ARROW_ASSIGN_OR_RAISE(auto group_identifier, GroupIdentifier::Make(ctx, key_descrs)); for (ValueDescr& key_descr : key_descrs) { out_fields.push_back(field("", std::move(key_descr.type))); } ARROW_ASSIGN_OR_RAISE(auto key_batch_iterator, - detail::ExecBatchIterator::Make(keys, ctx->exec_chunksize())); + ExecBatchIterator::Make(keys, ctx->exec_chunksize())); // start "streaming" execution ExecBatch key_batch, aggregand_batch; @@ -854,7 +858,7 @@ Result GroupBy(const std::vector& aggregands, if (key_batch.length == 0) continue; // compute a batch of group ids - ARROW_ASSIGN_OR_RAISE(ExecBatch id_batch, impl.Consume(key_batch)); + ARROW_ASSIGN_OR_RAISE(ExecBatch id_batch, group_identifier->Consume(key_batch)); // consume group ids with HashAggregateKernels for (size_t i = 0; i < kernels.size(); ++i) { @@ -867,19 +871,23 @@ Result GroupBy(const std::vector& aggregands, } } + // Finalize output ArrayDataVector out_data(aggregands.size() + keys.size()); + auto it = out_data.begin(); + for (size_t i = 0; i < kernels.size(); ++i) { KernelContext batch_ctx{ctx}; batch_ctx.SetState(states[i].get()); Datum out; kernels[i].finalize(&batch_ctx, &out); if (batch_ctx.HasError()) return batch_ctx.status(); - out_data[i] = out.array(); + *it++ = out.array(); } - ARROW_ASSIGN_OR_RAISE(Datum out_keys, impl.Finalize(ctx)); - std::move(out_keys.array()->child_data.begin(), out_keys.array()->child_data.end(), - out_data.begin() + aggregands.size()); + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, group_identifier->GetUniqueKeys()); + for (const auto& key : out_keys.values) { + *it++ = key.array(); + } int64_t length = out_data[0]->length; return ArrayData::Make(struct_(std::move(out_fields)), length, @@ -887,5 +895,6 @@ Result GroupBy(const std::vector& aggregands, /*null_count=*/0); } +} // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h index 3d3853d3d2d..f89ea4b4383 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h @@ -15,46 +15,52 @@ // specific language governing permissions and limitations // under the License. -#include +#include +#include +#include -#include "arrow/compute/api_aggregate.h" -#include "arrow/compute/kernels/aggregate_basic_internal.h" -#include "arrow/compute/kernels/aggregate_internal.h" -#include "arrow/compute/kernels/common.h" -#include "arrow/util/bit_run_reader.h" -#include "arrow/util/bitmap_ops.h" -#include "arrow/util/cpu_info.h" -#include "arrow/util/make_unique.h" +#include "arrow/compute/function.h" +#include "arrow/datum.h" namespace arrow { namespace compute { +namespace internal { -/// \brief Configure a grouped aggregation -struct ARROW_EXPORT GroupByOptions { - struct Aggregate { - /// the name of the aggregation function - std::string function; +/// Internal use only: streaming group identifier. +/// Consumes batches of keys and yields batches of the group ids. +class ARROW_EXPORT GroupIdentifier { + public: + virtual ~GroupIdentifier() = default; - /// options for the aggregation function - const FunctionOptions* options; - }; + /// Construct a GroupIdentifier which receives the specified key types + static Result> Make( + ExecContext* ctx, const std::vector& descrs); - GroupByOptions() = default; + /// Consume a batch of keys, producing an array of the corresponding + /// group ids as an integer column. + virtual Result Consume(const ExecBatch& batch) = 0; - GroupByOptions(std::initializer_list aggregates) : aggregates(aggregates) {} + /// Get current unique keys. May be called repeatedly. + virtual Result GetUniqueKeys() = 0; +}; - explicit GroupByOptions(std::vector aggregates) - : aggregates(std::move(aggregates)) {} +/// \brief Configure a grouped aggregation +struct ARROW_EXPORT Aggregate { + /// the name of the aggregation function + std::string function; - std::vector aggregates; + /// options for the aggregation function + const FunctionOptions* options; }; /// Internal use only: helper function for testing HashAggregateKernels. /// This will be replaced by streaming execution operators. ARROW_EXPORT Result GroupBy(const std::vector& aggregands, - const std::vector& keys, const GroupByOptions& options, + const std::vector& keys, + const std::vector& aggregates, ExecContext* ctx = nullptr); +} // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 024d5a6d883..92f4540b072 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -317,7 +317,7 @@ struct ScalarVectorToArray { }; Result NaiveGroupBy(std::vector aggregands, std::vector keys, - GroupByOptions options) { + const std::vector& aggregates) { ArrayVector keys_arrays; for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); std::vector key_names(keys_arrays.size(), ""); @@ -334,17 +334,15 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key ArrayVector out_columns; for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { - const Datum& aggregand = aggregands[i_agg]; - const std::string& function = options.aggregates[i_agg].function; - ScalarVector aggregated_scalars; ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, - ApplyGroupings(*groupings, *aggregand.make_array())); + ApplyGroupings(*groupings, *aggregands[i_agg].make_array())); for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE( - Datum d, CallFunction(function, {grouped_aggregand->value_slice(i_group)})); + ARROW_ASSIGN_OR_RAISE(Datum d, + CallFunction(aggregates[i_agg].function, + {grouped_aggregand->value_slice(i_group)})); aggregated_scalars.push_back(d.scalar()); } @@ -363,12 +361,12 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key return StructArray::Make(std::move(out_columns), std::move(out_names)); } -void ValidateGroupBy(GroupByOptions options, std::vector aggregands, - std::vector keys) { +void ValidateGroupBy(const std::vector& aggregates, + std::vector aggregands, std::vector keys) { ASSERT_OK_AND_ASSIGN(Datum expected, - group_helpers::NaiveGroupBy(aggregands, keys, options)); + group_helpers::NaiveGroupBy(aggregands, keys, aggregates)); - ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, options)); + ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, aggregates)); // Ordering of groups is not important, so sort by key columns to ensure the comparison // doesn't fail spuriously @@ -402,10 +400,11 @@ TEST(GroupBy, SumOnlyBooleanKey) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(boolean(), "[1, 0, 1, 0, null, 0, null]"); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"sum", nullptr}, + })); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -424,10 +423,11 @@ TEST(GroupBy, SumOnly8bitKey) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int8(), "[1, 2, 3, 1, 2, 2, null]"); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"sum", nullptr}, + })); ASSERT_OK(aggregated_and_grouped.array_as()->ValidateFull()); @@ -449,10 +449,11 @@ TEST(GroupBy, SumOnly32bitKey) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int32(), "[1, 2, 3, 1, 2, 2, null]"); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"sum", nullptr}, + })); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -472,10 +473,11 @@ TEST(GroupBy, SumOnly) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"sum", nullptr}, + })); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -495,10 +497,11 @@ TEST(GroupBy, MinMaxOnly) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, GroupBy({aggregand}, {key}, - GroupByOptions{ - {"min_max", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"min_max", nullptr}, + })); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", struct_({ @@ -526,12 +529,12 @@ TEST(GroupBy, CountAndSum) { ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, // NB: passing an aggregand twice or also using it as a key is legal - GroupBy({aggregand, aggregand, key}, {key}, - GroupByOptions{ - {"count", &count_options}, - {"sum", nullptr}, - {"sum", nullptr}, - })); + internal::GroupBy({aggregand, aggregand, key}, {key}, + { + {"count", &count_options}, + {"sum", nullptr}, + {"sum", nullptr}, + })); AssertDatumsEqual( ArrayFromJSON(struct_({ @@ -556,7 +559,7 @@ TEST(GroupBy, StringKey) { auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - GroupBy({aggregand}, {key}, GroupByOptions{{"sum", nullptr}})); + internal::GroupBy({aggregand}, {key}, {{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", int64()), @@ -580,7 +583,7 @@ TEST(GroupBy, MultipleKeys) { ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - GroupBy({aggregand}, {int_key, str_key}, GroupByOptions{{"sum", nullptr}})); + internal::GroupBy({aggregand}, {int_key, str_key}, {{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -602,7 +605,7 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - group_helpers::ValidateGroupBy(GroupByOptions{{"sum", nullptr}}, {aggregand}, {key}); + group_helpers::ValidateGroupBy({{"sum", nullptr}}, {aggregand}, {key}); } TEST(GroupBy, RandomArraySum) { @@ -615,7 +618,7 @@ TEST(GroupBy, RandomArraySum) { auto key = rand.Int64(length, 0, 12); group_helpers::ValidateGroupBy( - GroupByOptions{ + { {"sum", nullptr}, }, {summand}, {key}); From f612ef84c1f808c8d8798628344e8080850f66ca Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 16:11:33 -0400 Subject: [PATCH 26/49] reuse GroupIdentifier for partitioned writing --- .../arrow/compute/kernels/hash_aggregate.cc | 57 ++++- .../compute/kernels/hash_aggregate_internal.h | 31 ++- cpp/src/arrow/dataset/partition.cc | 221 ++---------------- cpp/src/arrow/dataset/partition.h | 2 +- 4 files changed, 107 insertions(+), 204 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index c0eac99d6ed..9ab896a2905 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -24,6 +24,7 @@ #include "arrow/buffer_builder.h" #include "arrow/compute/api_aggregate.h" +#include "arrow/compute/api_vector.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/kernel.h" #include "arrow/compute/kernels/aggregate_internal.h" @@ -324,6 +325,8 @@ struct GroupIdentifierImpl : GroupIdentifier { key_length); auto it_success = map_.emplace(key, num_groups_); + auto group_id = it_success.first->second; + if (it_success.second) { // new key; update offsets and key_bytes ++num_groups_; @@ -332,7 +335,8 @@ struct GroupIdentifierImpl : GroupIdentifier { offsets_.push_back(next_key_offset + key_length); memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); } - group_ids_batch.UnsafeAppend(it_success.first->second); + + group_ids_batch.UnsafeAppend(group_id); } ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); @@ -341,7 +345,7 @@ struct GroupIdentifierImpl : GroupIdentifier { batch.length); } - Result GetUniqueKeys() override { + Result GetUniques() override { ExecBatch out({}, num_groups_); std::vector key_buf_ptrs(num_groups_); @@ -884,7 +888,7 @@ Result GroupBy(const std::vector& aggregands, *it++ = out.array(); } - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, group_identifier->GetUniqueKeys()); + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, group_identifier->GetUniques()); for (const auto& key : out_keys.values) { *it++ = key.array(); } @@ -895,6 +899,53 @@ Result GroupBy(const std::vector& aggregands, /*null_count=*/0); } +namespace { +Result> CountsToOffsets(std::shared_ptr counts) { + TypedBufferBuilder offset_builder; + RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); + + int32_t current_offset = 0; + offset_builder.UnsafeAppend(current_offset); + + for (int64_t i = 0; i < counts->length(); ++i) { + DCHECK_NE(counts->Value(i), 0); + current_offset += static_cast(counts->Value(i)); + offset_builder.UnsafeAppend(current_offset); + } + + return offset_builder.Finish(); +} +} // namespace + +Result> MakeGroupings(Datum ids, ExecContext* ctx) { + if (ctx == nullptr) { + ExecContext default_ctx; + return MakeGroupings(ids, &default_ctx); + } + + if (ids.null_count() != 0) { + return Status::Invalid("MakeGroupings with null ids"); + } + + ARROW_ASSIGN_OR_RAISE(auto sort_indices, + compute::SortIndices(ids, compute::SortOptions::Defaults(), ctx)); + + ARROW_ASSIGN_OR_RAISE(auto counts_and_values, compute::ValueCounts(ids)); + + auto unique_ids = counts_and_values->GetFieldByName("values"); + + auto counts = + checked_pointer_cast(counts_and_values->GetFieldByName("counts")); + ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); + + auto groupings = + std::make_shared(list(sort_indices->type()), unique_ids->length(), + std::move(offsets), std::move(sort_indices)); + + return StructArray::Make(ArrayVector{std::move(unique_ids), std::move(groupings)}, + std::vector{"ids", "groupings"}); +} + } // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h index f89ea4b4383..d8414c1361e 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +#include #include #include #include @@ -37,11 +38,12 @@ class ARROW_EXPORT GroupIdentifier { ExecContext* ctx, const std::vector& descrs); /// Consume a batch of keys, producing an array of the corresponding - /// group ids as an integer column. + /// group ids as an integer column. The yielded batch also includes the current group + /// count, which is necessary for efficient resizing of kernel storage. virtual Result Consume(const ExecBatch& batch) = 0; - /// Get current unique keys. May be called repeatedly. - virtual Result GetUniqueKeys() = 0; + /// Get current unique keys. May be called multiple times. + virtual Result GetUniques() = 0; }; /// \brief Configure a grouped aggregation @@ -61,6 +63,29 @@ Result GroupBy(const std::vector& aggregands, const std::vector& aggregates, ExecContext* ctx = nullptr); +/// \brief Assemble lists of indices of identical elements. +/// +/// \param[in] ids An integral array which will be used as grouping criteria. +/// Nulls are invalid. +/// \return A array of type `struct>`, +/// which is a mapping from unique ids to lists of +/// indices into `ids` where that value appears +/// +/// MakeGroupings([ +/// 7, +/// 7, +/// 5, +/// 5, +/// 7, +/// 3 +/// ]) == [ +/// {"ids": 7, "groupings": [0, 1, 4]}, +/// {"ids": 5, "groupings": [2, 3]}, +/// {"ids": 3, "groupings": [5]} +/// ] +ARROW_EXPORT +Result> MakeGroupings(Datum ids, ExecContext* ctx = nullptr); + } // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index ec4a28c8a0e..91c72f0c330 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -29,6 +29,7 @@ #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" +#include "arrow/compute/kernels/hash_aggregate_internal.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/filesystem/path_util.h" #include "arrow/scalar.h" @@ -576,182 +577,6 @@ Result> PartitioningOrFactory::GetOrInferSchema( return factory()->Inspect(paths); } -// Transform an array of counts to offsets which will divide a ListArray -// into an equal number of slices with corresponding lengths. -inline Result> CountsToOffsets( - std::shared_ptr counts) { - TypedBufferBuilder offset_builder; - RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); - - int32_t current_offset = 0; - offset_builder.UnsafeAppend(current_offset); - - for (int64_t i = 0; i < counts->length(); ++i) { - DCHECK_NE(counts->Value(i), 0); - current_offset += static_cast(counts->Value(i)); - offset_builder.UnsafeAppend(current_offset); - } - - std::shared_ptr offsets; - RETURN_NOT_OK(offset_builder.Finish(&offsets)); - return offsets; -} - -// Helper for simultaneous dictionary encoding of multiple arrays. -// -// The fused dictionary is the Cartesian product of the individual dictionaries. -// For example given two arrays A, B where A has unique values ["ex", "why"] -// and B has unique values [0, 1] the fused dictionary is the set of tuples -// [["ex", 0], ["ex", 1], ["why", 0], ["ex", 1]]. -// -// TODO(bkietz) this capability belongs in an Action of the hash kernels, where -// it can be used to group aggregates without materializing a grouped batch. -// For the purposes of writing we need the materialized grouped batch anyway -// since no Writers accept a selection vector. -class StructDictionary { - public: - struct Encoded { - std::shared_ptr indices; - std::shared_ptr dictionary; - }; - - static Result Encode(const ArrayVector& columns) { - Encoded out{nullptr, std::make_shared()}; - - for (const auto& column : columns) { - RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices)); - } - - return out; - } - - Result> Decode(std::shared_ptr fused_indices, - FieldVector fields) { - std::vector builders(dictionaries_.size()); - for (Int32Builder& b : builders) { - RETURN_NOT_OK(b.Resize(fused_indices->length())); - } - - std::vector codes(dictionaries_.size()); - for (int64_t i = 0; i < fused_indices->length(); ++i) { - Expand(fused_indices->Value(i), codes.data()); - - auto builder_it = builders.begin(); - for (int32_t index : codes) { - builder_it++->UnsafeAppend(index); - } - } - - ArrayVector columns(dictionaries_.size()); - for (size_t i = 0; i < dictionaries_.size(); ++i) { - std::shared_ptr indices; - RETURN_NOT_OK(builders[i].FinishInternal(&indices)); - - ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices)); - - if (fields[i]->type()->id() == Type::DICTIONARY) { - RETURN_NOT_OK(RestoreDictionaryEncoding( - checked_pointer_cast(fields[i]->type()), &column)); - } - - columns[i] = column.make_array(); - } - - return StructArray::Make(std::move(columns), std::move(fields)); - } - - private: - Status AddOne(Datum column, std::shared_ptr* fused_indices) { - if (column.type()->id() == Type::DICTIONARY) { - if (column.null_count() != 0) { - // TODO(ARROW-11732) Optimize this by allowign DictionaryEncode to transfer a - // null-masked dictionary to a null-encoded dictionary. At the moment we decode - // and then encode causing one extra copy, and a potentially expansive decoding - // copy at that. - ARROW_ASSIGN_OR_RAISE( - auto decoded_dictionary, - compute::Cast( - column, - std::static_pointer_cast(column.type())->value_type(), - compute::CastOptions())); - column = decoded_dictionary; - } - } - if (column.type()->id() != Type::DICTIONARY) { - compute::DictionaryEncodeOptions options; - options.null_encoding_behavior = - compute::DictionaryEncodeOptions::NullEncodingBehavior::ENCODE; - ARROW_ASSIGN_OR_RAISE(column, - compute::DictionaryEncode(std::move(column), options)); - } - - auto dict_column = column.array_as(); - dictionaries_.push_back(dict_column->dictionary()); - ARROW_ASSIGN_OR_RAISE(auto indices, compute::Cast(*dict_column->indices(), int32())); - - if (*fused_indices == nullptr) { - *fused_indices = checked_pointer_cast(std::move(indices)); - return IncreaseSize(); - } - - // It's useful to think about the case where each of dictionaries_ has size 10. - // In this case the decimal digit in the ones place is the code in dictionaries_[0], - // the tens place corresponds to the code in dictionaries_[1], etc. - // The incumbent indices must be shifted to the hundreds place so as not to collide. - ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices, - compute::Multiply(indices, MakeScalar(size_))); - - ARROW_ASSIGN_OR_RAISE(new_fused_indices, - compute::Add(new_fused_indices, *fused_indices)); - - *fused_indices = checked_pointer_cast(new_fused_indices.make_array()); - return IncreaseSize(); - } - - // expand a fused code into component dict codes, order is in order of addition - void Expand(int32_t fused_code, int32_t* codes) { - for (size_t i = 0; i < dictionaries_.size(); ++i) { - auto dictionary_size = static_cast(dictionaries_[i]->length()); - codes[i] = fused_code % dictionary_size; - fused_code /= dictionary_size; - } - } - - Status RestoreDictionaryEncoding(std::shared_ptr expected_type, - Datum* column) { - DCHECK_NE(column->type()->id(), Type::DICTIONARY); - ARROW_ASSIGN_OR_RAISE(*column, compute::DictionaryEncode(std::move(*column))); - - if (expected_type->index_type()->id() == Type::INT32) { - // dictionary_encode has already yielded the expected index_type - return Status::OK(); - } - - // cast the indices to the expected index type - auto dictionary = std::move(column->mutable_array()->dictionary); - column->mutable_array()->type = int32(); - - ARROW_ASSIGN_OR_RAISE(*column, - compute::Cast(std::move(*column), expected_type->index_type())); - - column->mutable_array()->dictionary = std::move(dictionary); - column->mutable_array()->type = expected_type; - return Status::OK(); - } - - Status IncreaseSize() { - auto factor = static_cast(dictionaries_.back()->length()); - - if (internal::MultiplyWithOverflow(size_, factor, &size_)) { - return Status::CapacityError("Max groups exceeded"); - } - return Status::OK(); - } - - int32_t size_ = 1; - ArrayVector dictionaries_; -}; - Result> MakeGroupings(const StructArray& by) { if (by.num_fields() == 0) { return Status::Invalid("Grouping with no criteria"); @@ -761,33 +586,35 @@ Result> MakeGroupings(const StructArray& by) { return Status::Invalid("Grouping with null criteria"); } - ARROW_ASSIGN_OR_RAISE(auto fused, StructDictionary::Encode(by.fields())); + compute::ExecBatch key_batch({}, by.length()); + for (const auto& key : by.fields()) { + key_batch.values.emplace_back(key); + } + + compute::ExecContext ctx; + ARROW_ASSIGN_OR_RAISE(auto group_identifier, compute::internal::GroupIdentifier::Make( + &ctx, key_batch.GetDescriptors())); - ARROW_ASSIGN_OR_RAISE(auto sort_indices, compute::SortIndices(*fused.indices)); - ARROW_ASSIGN_OR_RAISE(Datum sorted, compute::Take(fused.indices, *sort_indices)); - fused.indices = checked_pointer_cast(sorted.make_array()); + ARROW_ASSIGN_OR_RAISE(auto id_batch, group_identifier->Consume(key_batch)); - ARROW_ASSIGN_OR_RAISE(auto fused_counts_and_values, - compute::ValueCounts(fused.indices)); - fused.indices.reset(); + ARROW_ASSIGN_OR_RAISE(auto unique_ids_and_groupings, + compute::internal::MakeGroupings(id_batch[0])); - auto unique_fused_indices = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("values")); - ARROW_ASSIGN_OR_RAISE( - auto unique_rows, - fused.dictionary->Decode(std::move(unique_fused_indices), by.type()->fields())); + auto unique_ids = MakeArray(std::move(unique_ids_and_groupings->data()->child_data[0])); + // if unique_ids is not sorted then groupings are out of order WRT groupings - auto counts = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("counts")); - ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); + ARROW_ASSIGN_OR_RAISE(auto uniques, group_identifier->GetUniques()); + ArrayVector unique_rows_fields(uniques.num_values()); + for (int i = 0; i < by.num_fields(); ++i) { + unique_rows_fields[i] = uniques[i].make_array(); + } + ARROW_ASSIGN_OR_RAISE(auto unique_rows, StructArray::Make(std::move(unique_rows_fields), + by.type()->fields())); - auto grouped_sort_indices = - std::make_shared(list(sort_indices->type()), unique_rows->length(), - std::move(offsets), std::move(sort_indices)); + auto groupings = MakeArray(std::move(unique_ids_and_groupings->data()->child_data[1])); - return StructArray::Make( - ArrayVector{std::move(unique_rows), std::move(grouped_sort_indices)}, - std::vector{"values", "groupings"}); + return StructArray::Make(ArrayVector{std::move(unique_rows), std::move(groupings)}, + std::vector{"values", "groupings"}); } Result> ApplyGroupings(const ListArray& groupings, diff --git a/cpp/src/arrow/dataset/partition.h b/cpp/src/arrow/dataset/partition.h index c49ac5e923e..bd739f4cf09 100644 --- a/cpp/src/arrow/dataset/partition.h +++ b/cpp/src/arrow/dataset/partition.h @@ -200,7 +200,7 @@ class ARROW_DS_EXPORT HivePartitioning : public KeyValuePartitioning { explicit HivePartitioning(std::shared_ptr schema, ArrayVector dictionaries = {}, std::string null_fallback = kDefaultHiveNullFallback) : KeyValuePartitioning(std::move(schema), std::move(dictionaries)), - null_fallback_(null_fallback) {} + null_fallback_(std::move(null_fallback)) {} std::string type_name() const override { return "hive"; } std::string null_fallback() const { return null_fallback_; } From f1b16647ce852d4a42e28c2c9a570604fe0a5ea1 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 16:48:30 -0400 Subject: [PATCH 27/49] add DictionaryKeyEncoder --- .../arrow/compute/kernels/hash_aggregate.cc | 68 ++++++++++++++++--- .../compute/kernels/hash_aggregate_test.cc | 46 +++++++++++++ 2 files changed, 103 insertions(+), 11 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 9ab896a2905..7a3550ba279 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -51,7 +51,7 @@ struct KeyEncoder { virtual void AddLength(const ArrayData&, int32_t* lengths) = 0; - virtual void Encode(const ArrayData&, uint8_t** encoded_bytes) = 0; + virtual Status Encode(const ArrayData&, uint8_t** encoded_bytes) = 0; virtual Result> Decode(uint8_t** encoded_bytes, int32_t length, MemoryPool*) = 0; @@ -91,7 +91,7 @@ struct BooleanKeyEncoder : KeyEncoder { } } - void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + Status Encode(const ArrayData& data, uint8_t** encoded_bytes) override { VisitArrayDataInline( data, [&](bool value) { @@ -104,6 +104,7 @@ struct BooleanKeyEncoder : KeyEncoder { *encoded_ptr++ = 1; *encoded_ptr++ = 0; }); + return Status::OK(); } Result> Decode(uint8_t** encoded_bytes, int32_t length, @@ -127,8 +128,9 @@ struct BooleanKeyEncoder : KeyEncoder { }; struct FixedWidthKeyEncoder : KeyEncoder { - FixedWidthKeyEncoder(int byte_width, std::shared_ptr type) - : byte_width_(byte_width), type_(std::move(type)) {} + explicit FixedWidthKeyEncoder(std::shared_ptr type) + : type_(std::move(type)), + byte_width_(checked_cast(*type_).bit_width() / 8) {} void AddLength(const ArrayData& data, int32_t* lengths) override { for (int64_t i = 0; i < data.length; ++i) { @@ -136,7 +138,7 @@ struct FixedWidthKeyEncoder : KeyEncoder { } } - void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + Status Encode(const ArrayData& data, uint8_t** encoded_bytes) override { ArrayData viewed(fixed_size_binary(byte_width_), data.length, data.buffers, data.null_count, data.offset); @@ -154,6 +156,7 @@ struct FixedWidthKeyEncoder : KeyEncoder { memset(encoded_ptr, 0, byte_width_); encoded_ptr += byte_width_; }); + return Status::OK(); } Result> Decode(uint8_t** encoded_bytes, int32_t length, @@ -176,8 +179,45 @@ struct FixedWidthKeyEncoder : KeyEncoder { null_count); } - int byte_width_; std::shared_ptr type_; + int byte_width_; +}; + +struct DictionaryKeyEncoder : FixedWidthKeyEncoder { + DictionaryKeyEncoder(std::shared_ptr type, MemoryPool* pool) + : FixedWidthKeyEncoder(std::move(type)), pool_(pool) {} + + Status Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + auto dict = MakeArray(data.dictionary); + if (dictionary_) { + if (!dictionary_->Equals(dict)) { + // TODO(bkietz) unify if necessary + return Status::NotImplemented("Dictionary keys with multiple dictionaries"); + } + } else { + dictionary_ = std::move(dict); + } + return FixedWidthKeyEncoder::Encode(data, encoded_bytes); + } + + Result> Decode(uint8_t** encoded_bytes, int32_t length, + MemoryPool* pool) override { + ARROW_ASSIGN_OR_RAISE(auto data, + FixedWidthKeyEncoder::Decode(encoded_bytes, length, pool)); + + if (dictionary_) { + data->dictionary = dictionary_->data(); + } else { + ARROW_ASSIGN_OR_RAISE(auto dict, MakeArrayOfNull(type_, 0)); + data->dictionary = dict->data(); + } + + data->type = type_; + return data; + } + + MemoryPool* pool_; + std::shared_ptr dictionary_; }; template @@ -194,8 +234,8 @@ struct VarLengthKeyEncoder : KeyEncoder { [&] { lengths[i++] += kExtraByteForNull + sizeof(Offset); }); } - void Encode(const ArrayData& data, uint8_t** encoded_bytes) override { - return VisitArrayDataInline( + Status Encode(const ArrayData& data, uint8_t** encoded_bytes) override { + VisitArrayDataInline( data, [&](util::string_view bytes) { auto& encoded_ptr = *encoded_bytes++; @@ -211,6 +251,7 @@ struct VarLengthKeyEncoder : KeyEncoder { util::SafeStore(encoded_ptr, static_cast(0)); encoded_ptr += sizeof(Offset); }); + return Status::OK(); } Result> Decode(uint8_t** encoded_bytes, int32_t length, @@ -268,9 +309,14 @@ struct GroupIdentifierImpl : GroupIdentifier { continue; } - if (auto byte_width = bit_width(key->id()) / 8) { + if (key->id() == Type::DICTIONARY) { impl->encoders_[i] = - ::arrow::internal::make_unique(byte_width, key); + ::arrow::internal::make_unique(key, ctx->memory_pool()); + continue; + } + + if (is_fixed_width(key->id())) { + impl->encoders_[i] = ::arrow::internal::make_unique(key); continue; } @@ -312,7 +358,7 @@ struct GroupIdentifierImpl : GroupIdentifier { } for (int i = 0; i < batch.num_values(); ++i) { - encoders_[i]->Encode(*batch[i].array(), key_buf_ptrs.data()); + RETURN_NOT_OK(encoders_[i]->Encode(*batch[i].array(), key_buf_ptrs.data())); } TypedBufferBuilder group_ids_batch(ctx_->memory_pool()); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 92f4540b072..51dbba6af13 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -493,6 +493,30 @@ TEST(GroupBy, SumOnly) { /*verbose=*/true); } +TEST(GroupBy, SumOnlyFloatingPointKey) { + auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(float64(), "[1, 2, 3, 1, 2, 2, null]"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, + { + {"sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", float64()), + }), + R"([ + [4.25, 1], + [-0.125, 2], + [null, 3], + [0.75, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + TEST(GroupBy, MinMaxOnly) { auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); @@ -575,6 +599,28 @@ TEST(GroupBy, StringKey) { /*verbose=*/true); } +TEST(GroupBy, DictKey) { + auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); + auto key = ArrayFromJSON(dictionary(int32(), utf8()), + R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({aggregand}, {key}, {{"sum", nullptr}})); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", int64()), + field("", dictionary(int32(), utf8())), + }), + R"([ + [10, "alfa"], + [14, "beta"], + [6, "gamma"], + [12, null] + ])"), + aggregated_and_grouped, + /*verbose=*/true); +} + TEST(GroupBy, MultipleKeys) { auto aggregand = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); From b44f13b64bec12e296edbf5c6ea5496c643e82f7 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 17:33:37 -0400 Subject: [PATCH 28/49] reintroduce R binding --- cpp/src/arrow/compute/api_aggregate.h | 62 +++++++++++++ .../arrow/compute/kernels/aggregate_test.cc | 1 - .../arrow/compute/kernels/hash_aggregate.cc | 3 +- .../compute/kernels/hash_aggregate_internal.h | 91 ------------------- .../compute/kernels/hash_aggregate_test.cc | 1 - cpp/src/arrow/dataset/partition.cc | 2 +- r/R/arrowExports.R | 4 + r/R/dplyr.R | 50 +++++++++- r/src/arrowExports.cpp | 11 +++ r/src/compute.cpp | 23 +++++ 10 files changed, 149 insertions(+), 99 deletions(-) delete mode 100644 cpp/src/arrow/compute/kernels/hash_aggregate_internal.h diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index eef1587bb73..abfa350299c 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -306,5 +306,67 @@ Result TDigest(const Datum& value, const TDigestOptions& options = TDigestOptions::Defaults(), ExecContext* ctx = NULLPTR); +namespace internal { + +/// Internal use only: streaming group identifier. +/// Consumes batches of keys and yields batches of the group ids. +class ARROW_EXPORT GroupIdentifier { + public: + virtual ~GroupIdentifier() = default; + + /// Construct a GroupIdentifier which receives the specified key types + static Result> Make( + ExecContext* ctx, const std::vector& descrs); + + /// Consume a batch of keys, producing an array of the corresponding + /// group ids as an integer column. The yielded batch also includes the current group + /// count, which is necessary for efficient resizing of kernel storage. + virtual Result Consume(const ExecBatch& batch) = 0; + + /// Get current unique keys. May be called multiple times. + virtual Result GetUniques() = 0; +}; + +/// \brief Configure a grouped aggregation +struct ARROW_EXPORT Aggregate { + /// the name of the aggregation function + std::string function; + + /// options for the aggregation function + const FunctionOptions* options; +}; + +/// Internal use only: helper function for testing HashAggregateKernels. +/// This will be replaced by streaming execution operators. +ARROW_EXPORT +Result GroupBy(const std::vector& aggregands, + const std::vector& keys, + const std::vector& aggregates, + ExecContext* ctx = nullptr); + +/// Interna use only: Assemble lists of indices of identical elements. +/// +/// \param[in] ids An integral array which will be used as grouping criteria. +/// Nulls are invalid. +/// \return A array of type `struct>`, +/// which is a mapping from unique ids to lists of +/// indices into `ids` where that value appears +/// +/// MakeGroupings([ +/// 7, +/// 7, +/// 5, +/// 5, +/// 7, +/// 3 +/// ]) == [ +/// {"ids": 7, "groupings": [0, 1, 4]}, +/// {"ids": 5, "groupings": [2, 3]}, +/// {"ids": 3, "groupings": [5]} +/// ] +ARROW_EXPORT +Result> MakeGroupings(Datum ids, ExecContext* ctx = nullptr); + +} // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/aggregate_test.cc b/cpp/src/arrow/compute/kernels/aggregate_test.cc index 1b2896b629f..22e7f512e97 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_test.cc @@ -31,7 +31,6 @@ #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" -#include "arrow/compute/kernels/hash_aggregate_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" #include "arrow/type.h" diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 7a3550ba279..0833d1641f6 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/kernels/hash_aggregate_internal.h" +#include "arrow/compute/api_aggregate.h" #include #include @@ -23,7 +23,6 @@ #include #include "arrow/buffer_builder.h" -#include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/kernel.h" diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h b/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h deleted file mode 100644 index d8414c1361e..00000000000 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_internal.h +++ /dev/null @@ -1,91 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include -#include -#include -#include - -#include "arrow/compute/function.h" -#include "arrow/datum.h" - -namespace arrow { -namespace compute { -namespace internal { - -/// Internal use only: streaming group identifier. -/// Consumes batches of keys and yields batches of the group ids. -class ARROW_EXPORT GroupIdentifier { - public: - virtual ~GroupIdentifier() = default; - - /// Construct a GroupIdentifier which receives the specified key types - static Result> Make( - ExecContext* ctx, const std::vector& descrs); - - /// Consume a batch of keys, producing an array of the corresponding - /// group ids as an integer column. The yielded batch also includes the current group - /// count, which is necessary for efficient resizing of kernel storage. - virtual Result Consume(const ExecBatch& batch) = 0; - - /// Get current unique keys. May be called multiple times. - virtual Result GetUniques() = 0; -}; - -/// \brief Configure a grouped aggregation -struct ARROW_EXPORT Aggregate { - /// the name of the aggregation function - std::string function; - - /// options for the aggregation function - const FunctionOptions* options; -}; - -/// Internal use only: helper function for testing HashAggregateKernels. -/// This will be replaced by streaming execution operators. -ARROW_EXPORT -Result GroupBy(const std::vector& aggregands, - const std::vector& keys, - const std::vector& aggregates, - ExecContext* ctx = nullptr); - -/// \brief Assemble lists of indices of identical elements. -/// -/// \param[in] ids An integral array which will be used as grouping criteria. -/// Nulls are invalid. -/// \return A array of type `struct>`, -/// which is a mapping from unique ids to lists of -/// indices into `ids` where that value appears -/// -/// MakeGroupings([ -/// 7, -/// 7, -/// 5, -/// 5, -/// 7, -/// 3 -/// ]) == [ -/// {"ids": 7, "groupings": [0, 1, 4]}, -/// {"ids": 5, "groupings": [2, 3]}, -/// {"ids": 3, "groupings": [5]} -/// ] -ARROW_EXPORT -Result> MakeGroupings(Datum ids, ExecContext* ctx = nullptr); - -} // namespace internal -} // namespace compute -} // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 51dbba6af13..a978daf0f35 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -31,7 +31,6 @@ #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" -#include "arrow/compute/kernels/hash_aggregate_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" #include "arrow/type.h" diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 91c72f0c330..5a4fc34b48d 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -26,10 +26,10 @@ #include "arrow/array/array_dict.h" #include "arrow/array/array_nested.h" #include "arrow/array/builder_dict.h" +#include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" -#include "arrow/compute/kernels/hash_aggregate_internal.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/filesystem/path_util.h" #include "arrow/scalar.h" diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R index 0d0d3d30f8d..7fb01f33607 100644 --- a/r/R/arrowExports.R +++ b/r/R/arrowExports.R @@ -284,6 +284,10 @@ compute__CallFunction <- function(func_name, args, options){ .Call(`_arrow_compute__CallFunction`, func_name, args, options) } +compute__GroupBy <- function(aggregands, keys, options){ + .Call(`_arrow_compute__GroupBy`, aggregands, keys, options) +} + list_compute_functions <- function(){ .Call(`_arrow_list_compute_functions`) } diff --git a/r/R/dplyr.R b/r/R/dplyr.R index 14e210546f8..eda43206c18 100644 --- a/r/R/dplyr.R +++ b/r/R/dplyr.R @@ -456,21 +456,65 @@ pull.arrow_dplyr_query <- function(.data, var = -1) { pull.Dataset <- pull.ArrowTabular <- pull.arrow_dplyr_query summarise.arrow_dplyr_query <- function(.data, ...) { + call <- match.call() .data <- arrow_dplyr_query(.data) if (query_on_dataset(.data)) { not_implemented_for_dataset("summarize()") } + exprs <- quos(...) # Only retain the columns we need to do our aggregations vars_to_keep <- unique(c( - unlist(lapply(quos(...), all.vars)), # vars referenced in summarise + unlist(lapply(exprs, all.vars)), # vars referenced in summarise dplyr::group_vars(.data) # vars needed for grouping )) .data <- dplyr::select(.data, vars_to_keep) - # TODO: determine whether work can be pushed down to Arrow - dplyr::summarise(dplyr::collect(.data), ...) + if (isTRUE(getOption("arrow.summarize", FALSE))) { + # Try stuff, if successful return() + out <- try(do_arrow_group_by(.data, ...), silent = TRUE) + if (inherits(out, "try-error")) { + return(abandon_ship(call, .data, format(out))) + } else { + return(out) + } + } else { + # If unsuccessful or if option not set, do the work in R + dplyr::summarise(dplyr::collect(.data), ...) + } } summarise.Dataset <- summarise.ArrowTabular <- summarise.arrow_dplyr_query +do_arrow_group_by <- function(.data, ...) { + exprs <- quos(...) + mask <- arrow_mask(.data) + # Add aggregation wrappers to arrow_mask somehow + # (this is not ideal, would overwrite same-named objects) + mask$sum <- function(x, na.rm = FALSE) { + list( + fun = "sum", + data = x, + options = list(na.rm = na.rm) + ) + } + results <- list() + for (i in seq_along(exprs)) { + # Iterate over the indices and not the names because names may be repeated + # (which overwrites the previous name) + new_var <- names(exprs)[i] + results[[new_var]] <- arrow_eval(exprs[[i]], mask) + if (inherits(results[[new_var]], "try-error")) { + msg <- paste('Expression', as_label(exprs[[i]]), 'not supported in Arrow') + stop(msg, call. = FALSE) + } + # Put it in the data mask too? + #mask[[new_var]] <- mask$.data[[new_var]] <- results[[new_var]] + } + # Now, from that, split out the array (expressions) and options + opts <- lapply(results, function(x) x[c("fun", "options")]) + inputs <- lapply(results, function(x) eval_array_expression(x$data, .data$.data)) + grouping_vars <- lapply(.data$group_by_vars, function(x) eval_array_expression(.data$selected_columns[[x]], .data$.data)) + compute__GroupBy(inputs, grouping_vars, opts) +} + group_by.arrow_dplyr_query <- function(.data, ..., .add = FALSE, diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp index 7229e60b649..65bb7de6cc8 100644 --- a/r/src/arrowExports.cpp +++ b/r/src/arrowExports.cpp @@ -616,6 +616,16 @@ BEGIN_CPP11 END_CPP11 } // compute.cpp +SEXP compute__GroupBy(cpp11::list aggregands, cpp11::list keys, cpp11::list options); +extern "C" SEXP _arrow_compute__GroupBy(SEXP aggregands_sexp, SEXP keys_sexp, SEXP options_sexp){ +BEGIN_CPP11 + arrow::r::Input::type aggregands(aggregands_sexp); + arrow::r::Input::type keys(keys_sexp); + arrow::r::Input::type options(options_sexp); + return cpp11::as_sexp(compute__GroupBy(aggregands, keys, options)); +END_CPP11 +} +// compute.cpp std::vector list_compute_functions(); extern "C" SEXP _arrow_list_compute_functions(){ BEGIN_CPP11 @@ -4186,6 +4196,7 @@ static const R_CallMethodDef CallEntries[] = { { "_arrow_RecordBatch__cast", (DL_FUNC) &_arrow_RecordBatch__cast, 3}, { "_arrow_Table__cast", (DL_FUNC) &_arrow_Table__cast, 3}, { "_arrow_compute__CallFunction", (DL_FUNC) &_arrow_compute__CallFunction, 3}, + { "_arrow_compute__GroupBy", (DL_FUNC) &_arrow_compute__GroupBy, 3}, { "_arrow_list_compute_functions", (DL_FUNC) &_arrow_list_compute_functions, 0}, { "_arrow_csv___ReadOptions__initialize", (DL_FUNC) &_arrow_csv___ReadOptions__initialize, 1}, { "_arrow_csv___ParseOptions__initialize", (DL_FUNC) &_arrow_csv___ParseOptions__initialize, 1}, diff --git a/r/src/compute.cpp b/r/src/compute.cpp index 7bcded78f0d..b1b303ff784 100644 --- a/r/src/compute.cpp +++ b/r/src/compute.cpp @@ -199,6 +199,29 @@ SEXP compute__CallFunction(std::string func_name, cpp11::list args, cpp11::list return from_datum(std::move(out)); } +// [[arrow::export]] +SEXP compute__GroupBy(cpp11::list aggregands, cpp11::list keys, cpp11::list options) { + // options is a list of pairs: string function name, list of options + + std::vector> keep_alives; + std::vector aggregates; + + for (cpp11::list name_opts : options) { + auto name = cpp11::as_cpp(name_opts[0]); + auto opts = make_compute_options(name, name_opts[1]); + + aggregates.push_back( + arrow::compute::internal::Aggregate{std::move(name), opts.get()}); + keep_alives.push_back(std::move(opts)); + } + + auto datum_aggregands = arrow::r::from_r_list(aggregands); + auto datum_keys = arrow::r::from_r_list(keys); + auto out = ValueOrStop(arrow::compute::internal::GroupBy(datum_aggregands, datum_keys, + aggregates, gc_context())); + return from_datum(std::move(out)); +} + // [[arrow::export]] std::vector list_compute_functions() { return arrow::compute::GetFunctionRegistry()->GetFunctionNames(); From 23c2cf6242618c20e39df8659b24e3efd92a96ac Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 18:08:29 -0400 Subject: [PATCH 29/49] rename aggregand to argument --- cpp/src/arrow/compute/api_aggregate.h | 2 +- cpp/src/arrow/compute/kernel.h | 2 +- .../compute/kernels/aggregate_benchmark.cc | 6 +- .../arrow/compute/kernels/hash_aggregate.cc | 28 +++--- .../compute/kernels/hash_aggregate_test.cc | 86 +++++++++---------- r/R/arrowExports.R | 4 +- r/src/arrowExports.cpp | 8 +- r/src/compute.cpp | 6 +- 8 files changed, 71 insertions(+), 71 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index abfa350299c..7b9321c24e7 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -339,7 +339,7 @@ struct ARROW_EXPORT Aggregate { /// Internal use only: helper function for testing HashAggregateKernels. /// This will be replaced by streaming execution operators. ARROW_EXPORT -Result GroupBy(const std::vector& aggregands, +Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, ExecContext* ctx = nullptr); diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index 0733bbaa597..d3866853e0d 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -720,7 +720,7 @@ using HashAggregateFinalize = std::function; /// kernel are the init, consume, merge, and finalize functions. /// /// * init: creates a new KernelState for a kernel. -/// * consume: processes an ExecBatch (which includes the aggregand as well +/// * consume: processes an ExecBatch (which includes the argument as well /// as an array of group identifiers) and updates the KernelState found in the /// KernelContext. /// * finalize: produces the end result of the aggregation using the diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index 76db483be96..4f395d9a84a 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -306,10 +306,10 @@ BENCHMARK_TEMPLATE(ReferenceSum, SumBitmapVectorizeUnroll) static void BenchmarkGroupBy(benchmark::State& state, std::vector aggregates, - std::vector aggregands, std::vector keys) { + std::vector arguments, std::vector keys) { std::vector arguments; - for (const Datum& aggregand : aggregands) { - arguments.push_back(aggregand); + for (const Datum& argument : arguments) { + arguments.push_back(argument); } for (const Datum& key : keys) { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 0833d1641f6..badd828207d 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -796,7 +796,7 @@ Result> MakeKernels( const std::vector& aggregates, const std::vector& in_descrs) { if (aggregates.size() != in_descrs.size()) { return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", - in_descrs.size(), " aggregands were provided."); + in_descrs.size(), " arguments were provided."); } std::vector kernels(in_descrs.size()); @@ -859,32 +859,32 @@ Result> GroupIdentifier::Make( return GroupIdentifierImpl::Make(ctx, descrs); } -Result GroupBy(const std::vector& aggregands, +Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, ExecContext* ctx) { if (ctx == nullptr) { ExecContext default_ctx; - return GroupBy(aggregands, keys, aggregates, &default_ctx); + return GroupBy(arguments, keys, aggregates, &default_ctx); } // Construct and initialize HashAggregateKernels - ARROW_ASSIGN_OR_RAISE(auto aggregand_descrs, - ExecBatch::Make(aggregands).Map([](ExecBatch batch) { + ARROW_ASSIGN_OR_RAISE(auto argument_descrs, + ExecBatch::Make(arguments).Map([](ExecBatch batch) { return batch.GetDescriptors(); })); - ARROW_ASSIGN_OR_RAISE(auto kernels, MakeKernels(aggregates, aggregand_descrs)); + ARROW_ASSIGN_OR_RAISE(auto kernels, MakeKernels(aggregates, argument_descrs)); ARROW_ASSIGN_OR_RAISE(auto states, - InitKernels(kernels, ctx, aggregates, aggregand_descrs)); + InitKernels(kernels, ctx, aggregates, argument_descrs)); ARROW_ASSIGN_OR_RAISE(FieldVector out_fields, - ResolveKernels(kernels, states, ctx, aggregand_descrs)); + ResolveKernels(kernels, states, ctx, argument_descrs)); using arrow::compute::detail::ExecBatchIterator; - ARROW_ASSIGN_OR_RAISE(auto aggregand_batch_iterator, - ExecBatchIterator::Make(aggregands, ctx->exec_chunksize())); + ARROW_ASSIGN_OR_RAISE(auto argument_batch_iterator, + ExecBatchIterator::Make(arguments, ctx->exec_chunksize())); // Construct GroupIdentifier ARROW_ASSIGN_OR_RAISE(auto key_descrs, ExecBatch::Make(keys).Map([](ExecBatch batch) { @@ -901,8 +901,8 @@ Result GroupBy(const std::vector& aggregands, ExecBatchIterator::Make(keys, ctx->exec_chunksize())); // start "streaming" execution - ExecBatch key_batch, aggregand_batch; - while (aggregand_batch_iterator->Next(&aggregand_batch) && + ExecBatch key_batch, argument_batch; + while (argument_batch_iterator->Next(&argument_batch) && key_batch_iterator->Next(&key_batch)) { if (key_batch.length == 0) continue; @@ -914,14 +914,14 @@ Result GroupBy(const std::vector& aggregands, KernelContext batch_ctx{ctx}; batch_ctx.SetState(states[i].get()); ARROW_ASSIGN_OR_RAISE( - auto batch, ExecBatch::Make({aggregand_batch[i], id_batch[0], id_batch[1]})); + auto batch, ExecBatch::Make({argument_batch[i], id_batch[0], id_batch[1]})); kernels[i].consume(&batch_ctx, batch); if (batch_ctx.HasError()) return batch_ctx.status(); } } // Finalize output - ArrayDataVector out_data(aggregands.size() + keys.size()); + ArrayDataVector out_data(arguments.size() + keys.size()); auto it = out_data.begin(); for (size_t i = 0; i < kernels.size(); ++i) { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index a978daf0f35..d9876df2f4c 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -315,7 +315,7 @@ struct ScalarVectorToArray { std::shared_ptr data_; }; -Result NaiveGroupBy(std::vector aggregands, std::vector keys, +Result NaiveGroupBy(std::vector arguments, std::vector keys, const std::vector& aggregates) { ArrayVector keys_arrays; for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); @@ -332,16 +332,16 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key ArrayVector out_columns; - for (size_t i_agg = 0; i_agg < aggregands.size(); ++i_agg) { + for (size_t i_agg = 0; i_agg < arguments.size(); ++i_agg) { ScalarVector aggregated_scalars; - ARROW_ASSIGN_OR_RAISE(auto grouped_aggregand, - ApplyGroupings(*groupings, *aggregands[i_agg].make_array())); + ARROW_ASSIGN_OR_RAISE(auto grouped_argument, + ApplyGroupings(*groupings, *arguments[i_agg].make_array())); for (int64_t i_group = 0; i_group < n_groups; ++i_group) { ARROW_ASSIGN_OR_RAISE(Datum d, CallFunction(aggregates[i_agg].function, - {grouped_aggregand->value_slice(i_group)})); + {grouped_argument->value_slice(i_group)})); aggregated_scalars.push_back(d.scalar()); } @@ -352,7 +352,7 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key keys_struct = checked_pointer_cast(groupings_and_values->GetFieldByName("values")); - for (size_t i_key = 0; i_key < aggregands.size(); ++i_key) { + for (size_t i_key = 0; i_key < arguments.size(); ++i_key) { out_columns.push_back(keys_struct->field(i_key)); } @@ -361,11 +361,11 @@ Result NaiveGroupBy(std::vector aggregands, std::vector key } void ValidateGroupBy(const std::vector& aggregates, - std::vector aggregands, std::vector keys) { + std::vector arguments, std::vector keys) { ASSERT_OK_AND_ASSIGN(Datum expected, - group_helpers::NaiveGroupBy(aggregands, keys, aggregates)); + group_helpers::NaiveGroupBy(arguments, keys, aggregates)); - ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(aggregands, keys, aggregates)); + ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(arguments, keys, aggregates)); // Ordering of groups is not important, so sort by key columns to ensure the comparison // doesn't fail spuriously @@ -396,11 +396,11 @@ void ValidateGroupBy(const std::vector& aggregates, } // namespace group_helpers TEST(GroupBy, SumOnlyBooleanKey) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(boolean(), "[1, 0, 1, 0, null, 0, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"sum", nullptr}, })); @@ -419,11 +419,11 @@ TEST(GroupBy, SumOnlyBooleanKey) { } TEST(GroupBy, SumOnly8bitKey) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int8(), "[1, 2, 3, 1, 2, 2, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"sum", nullptr}, })); @@ -445,11 +445,11 @@ TEST(GroupBy, SumOnly8bitKey) { } TEST(GroupBy, SumOnly32bitKey) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int32(), "[1, 2, 3, 1, 2, 2, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"sum", nullptr}, })); @@ -469,11 +469,12 @@ TEST(GroupBy, SumOnly32bitKey) { } TEST(GroupBy, SumOnly) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); + auto argument = + ArrayFromJSON(float64(), "[1.0, 0.0, null, 4.0, 3.25, 0.125, -0.25, 0.75]"); + auto key = ArrayFromJSON(int64(), "[1, 2, 3, null, 1, 2, 2, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"sum", nullptr}, })); @@ -486,18 +487,18 @@ TEST(GroupBy, SumOnly) { [4.25, 1], [-0.125, 2], [null, 3], - [0.75, null] + [4.75, null] ])"), aggregated_and_grouped, /*verbose=*/true); } TEST(GroupBy, SumOnlyFloatingPointKey) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(float64(), "[1, 2, 3, 1, 2, 2, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"sum", nullptr}, })); @@ -517,11 +518,11 @@ TEST(GroupBy, SumOnlyFloatingPointKey) { } TEST(GroupBy, MinMaxOnly) { - auto aggregand = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, + internal::GroupBy({argument}, {key}, { {"min_max", nullptr}, })); @@ -544,20 +545,19 @@ TEST(GroupBy, MinMaxOnly) { } TEST(GroupBy, CountAndSum) { - auto aggregand = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); + auto argument = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); CountOptions count_options; - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - // NB: passing an aggregand twice or also using it as a key is legal - internal::GroupBy({aggregand, aggregand, key}, {key}, - { - {"count", &count_options}, - {"sum", nullptr}, - {"sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + // NB: passing an argument twice or also using it as a key is legal + internal::GroupBy({argument, argument, key}, {key}, + { + {"count", &count_options}, + {"sum", nullptr}, + {"sum", nullptr}, + })); AssertDatumsEqual( ArrayFromJSON(struct_({ @@ -578,11 +578,11 @@ TEST(GroupBy, CountAndSum) { } TEST(GroupBy, StringKey) { - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); + auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {key}, {{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", int64()), @@ -599,12 +599,12 @@ TEST(GroupBy, StringKey) { } TEST(GroupBy, DictKey) { - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); + auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); auto key = ArrayFromJSON(dictionary(int32(), utf8()), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {key}, {{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", int64()), @@ -621,14 +621,14 @@ TEST(GroupBy, DictKey) { } TEST(GroupBy, MultipleKeys) { - auto aggregand = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); + auto argument = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); auto str_key = ArrayFromJSON(utf8(), R"(["beta", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - internal::GroupBy({aggregand}, {int_key, str_key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {int_key, str_key}, {{"sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -647,17 +647,17 @@ TEST(GroupBy, MultipleKeys) { } TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { - auto aggregand = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); + auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - group_helpers::ValidateGroupBy({{"sum", nullptr}}, {aggregand}, {key}); + group_helpers::ValidateGroupBy({{"sum", nullptr}}, {argument}, {key}); } TEST(GroupBy, RandomArraySum) { auto rand = random::RandomArrayGenerator(0xdeadbeef); - for (size_t i = 3; i < 14; i += 2) { - for (auto null_probability : {0.0, 0.001, 0.1, 0.5, 0.999, 1.0}) { + for (size_t i = 10; i < 14; i += 2) { + for (auto null_probability : {0.0, 0.1, 0.5, 1.0}) { int64_t length = 1UL << i; auto summand = rand.Float32(length, -100, 100, null_probability); auto key = rand.Int64(length, 0, 12); diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R index 7fb01f33607..13d493b6faf 100644 --- a/r/R/arrowExports.R +++ b/r/R/arrowExports.R @@ -284,8 +284,8 @@ compute__CallFunction <- function(func_name, args, options){ .Call(`_arrow_compute__CallFunction`, func_name, args, options) } -compute__GroupBy <- function(aggregands, keys, options){ - .Call(`_arrow_compute__GroupBy`, aggregands, keys, options) +compute__GroupBy <- function(arguments, keys, options){ + .Call(`_arrow_compute__GroupBy`, arguments, keys, options) } list_compute_functions <- function(){ diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp index 65bb7de6cc8..daf1303ebc2 100644 --- a/r/src/arrowExports.cpp +++ b/r/src/arrowExports.cpp @@ -616,13 +616,13 @@ BEGIN_CPP11 END_CPP11 } // compute.cpp -SEXP compute__GroupBy(cpp11::list aggregands, cpp11::list keys, cpp11::list options); -extern "C" SEXP _arrow_compute__GroupBy(SEXP aggregands_sexp, SEXP keys_sexp, SEXP options_sexp){ +SEXP compute__GroupBy(cpp11::list arguments, cpp11::list keys, cpp11::list options); +extern "C" SEXP _arrow_compute__GroupBy(SEXP arguments_sexp, SEXP keys_sexp, SEXP options_sexp){ BEGIN_CPP11 - arrow::r::Input::type aggregands(aggregands_sexp); + arrow::r::Input::type arguments(arguments_sexp); arrow::r::Input::type keys(keys_sexp); arrow::r::Input::type options(options_sexp); - return cpp11::as_sexp(compute__GroupBy(aggregands, keys, options)); + return cpp11::as_sexp(compute__GroupBy(arguments, keys, options)); END_CPP11 } // compute.cpp diff --git a/r/src/compute.cpp b/r/src/compute.cpp index b1b303ff784..07380354b12 100644 --- a/r/src/compute.cpp +++ b/r/src/compute.cpp @@ -200,7 +200,7 @@ SEXP compute__CallFunction(std::string func_name, cpp11::list args, cpp11::list } // [[arrow::export]] -SEXP compute__GroupBy(cpp11::list aggregands, cpp11::list keys, cpp11::list options) { +SEXP compute__GroupBy(cpp11::list arguments, cpp11::list keys, cpp11::list options) { // options is a list of pairs: string function name, list of options std::vector> keep_alives; @@ -215,9 +215,9 @@ SEXP compute__GroupBy(cpp11::list aggregands, cpp11::list keys, cpp11::list opti keep_alives.push_back(std::move(opts)); } - auto datum_aggregands = arrow::r::from_r_list(aggregands); + auto datum_arguments = arrow::r::from_r_list(arguments); auto datum_keys = arrow::r::from_r_list(keys); - auto out = ValueOrStop(arrow::compute::internal::GroupBy(datum_aggregands, datum_keys, + auto out = ValueOrStop(arrow::compute::internal::GroupBy(datum_arguments, datum_keys, aggregates, gc_context())); return from_datum(std::move(out)); } From a57268c54160ea5ba268f884fa5035cc37e5e861 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 18:10:17 -0400 Subject: [PATCH 30/49] lint: nullptr --- cpp/src/arrow/compute/api_aggregate.h | 7 +++---- cpp/src/arrow/compute/kernels/hash_aggregate.cc | 8 +++----- cpp/src/arrow/compute/kernels/hash_aggregate_test.cc | 1 - 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 7b9321c24e7..a16aad4f150 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -339,10 +339,9 @@ struct ARROW_EXPORT Aggregate { /// Internal use only: helper function for testing HashAggregateKernels. /// This will be replaced by streaming execution operators. ARROW_EXPORT -Result GroupBy(const std::vector& arguments, - const std::vector& keys, +Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, - ExecContext* ctx = nullptr); + ExecContext* ctx = NULLPTR); /// Interna use only: Assemble lists of indices of identical elements. /// @@ -365,7 +364,7 @@ Result GroupBy(const std::vector& arguments, /// {"ids": 3, "groupings": [5]} /// ] ARROW_EXPORT -Result> MakeGroupings(Datum ids, ExecContext* ctx = nullptr); +Result> MakeGroupings(Datum ids, ExecContext* ctx = NULLPTR); } // namespace internal } // namespace compute diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index badd828207d..fd3cc1503fd 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -859,8 +859,7 @@ Result> GroupIdentifier::Make( return GroupIdentifierImpl::Make(ctx, descrs); } -Result GroupBy(const std::vector& arguments, - const std::vector& keys, +Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, ExecContext* ctx) { if (ctx == nullptr) { ExecContext default_ctx; @@ -869,9 +868,8 @@ Result GroupBy(const std::vector& arguments, // Construct and initialize HashAggregateKernels ARROW_ASSIGN_OR_RAISE(auto argument_descrs, - ExecBatch::Make(arguments).Map([](ExecBatch batch) { - return batch.GetDescriptors(); - })); + ExecBatch::Make(arguments).Map( + [](ExecBatch batch) { return batch.GetDescriptors(); })); ARROW_ASSIGN_OR_RAISE(auto kernels, MakeKernels(aggregates, argument_descrs)); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index d9876df2f4c..843dcc75f4a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -673,4 +673,3 @@ TEST(GroupBy, RandomArraySum) { } // namespace compute } // namespace arrow - From ea4d387b9745cd2a3ed8d68a3f99d0f96938ef7a Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 20:38:57 -0400 Subject: [PATCH 31/49] fix benchmark --- .../arrow/compute/kernels/aggregate_benchmark.cc | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index 4f395d9a84a..80ab037586c 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -305,22 +305,10 @@ BENCHMARK_TEMPLATE(ReferenceSum, SumBitmapVectorizeUnroll) // static void BenchmarkGroupBy(benchmark::State& state, - std::vector aggregates, + std::vector aggregates, std::vector arguments, std::vector keys) { - std::vector arguments; - for (const Datum& argument : arguments) { - arguments.push_back(argument); - } - - for (const Datum& key : keys) { - arguments.push_back(key); - } - - GroupByOptions options; - options.aggregates = aggregates; - for (auto _ : state) { - ABORT_NOT_OK(CallFunction("group_by", arguments, &options).status()); + ABORT_NOT_OK(GroupBy(arguments, keys, aggregates).status()); } } From fff598577163df1cb7634aab09e1b1e188ab2fb6 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 21:08:32 -0400 Subject: [PATCH 32/49] crispy compilers --- .../arrow/compute/kernels/hash_aggregate.cc | 47 +++++++++++-------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index fd3cc1503fd..799a93f02d4 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -17,6 +17,7 @@ #include "arrow/compute/api_aggregate.h" +#include #include #include #include @@ -271,13 +272,16 @@ struct VarLengthKeyEncoder : KeyEncoder { auto raw_offsets = reinterpret_cast(offset_buf->mutable_data()); auto raw_keys = key_buf->mutable_data(); - int32_t current_offset = 0; + Offset current_offset = 0; for (int32_t i = 0; i < length; ++i) { - auto key_length = util::SafeLoadAs(encoded_bytes[i]); raw_offsets[i] = current_offset; + + auto key_length = util::SafeLoadAs(encoded_bytes[i]); encoded_bytes[i] += sizeof(Offset); + memcpy(raw_keys + current_offset, encoded_bytes[i], key_length); encoded_bytes[i] += key_length; + current_offset += key_length; } raw_offsets[length] = current_offset; @@ -333,7 +337,8 @@ struct GroupIdentifierImpl : GroupIdentifier { return Status::NotImplemented("Keys of type ", *key); } - return impl; + + return std::move(impl); } Result Consume(const ExecBatch& batch) override { @@ -603,6 +608,21 @@ struct GroupedSumImpl : public GroupedAggregator { // ---------------------------------------------------------------------- // MinMax implementation +template +struct Extrema : std::numeric_limits {}; + +template <> +struct Extrema { + static constexpr float min() { return -std::numeric_limits::infinity(); } + static constexpr float max() { return std::numeric_limits::infinity(); } +}; + +template <> +struct Extrema { + static constexpr double min() { return -std::numeric_limits::infinity(); } + static constexpr double max() { return std::numeric_limits::infinity(); } +}; + struct GroupedMinMaxImpl : public GroupedAggregator { using ConsumeImpl = std::function&, const uint32_t*, void*, void*, @@ -640,7 +660,8 @@ struct GroupedMinMaxImpl : public GroupedAggregator { [&] { BitUtil::SetBit(has_nulls, *group++); }); }; - GetResizeImpls(); + resize_min_impl = MakeResizeImpl(Extrema::max()); + resize_max_impl = MakeResizeImpl(Extrema::min()); return Status::OK(); } @@ -656,19 +677,6 @@ struct GroupedMinMaxImpl : public GroupedAggregator { return Status::NotImplemented("Grouped MinMax data of type ", type); } - template ::CType> - enable_if_floating_point GetResizeImpls() { - auto inf = std::numeric_limits::infinity(); - resize_min_impl = MakeResizeImpl(inf); - resize_max_impl = MakeResizeImpl(-inf); - } - - template ::CType> - enable_if_integer GetResizeImpls() { - resize_max_impl = MakeResizeImpl(std::numeric_limits::min()); - resize_min_impl = MakeResizeImpl(std::numeric_limits::max()); - } - ConsumeImpl consume_impl; ResizeImpl resize_min_impl, resize_max_impl; }; @@ -750,7 +758,7 @@ KernelInit MakeInit() { auto impl = ::arrow::internal::make_unique(); ctx->SetStatus(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); if (ctx->HasError()) return nullptr; - return impl; + return std::move(impl); }; } @@ -828,7 +836,8 @@ Result>> InitKernels( &kernel_ctx, KernelInitArgs{&kernels[i], {in_descrs[i].type}, options}); if (kernel_ctx.HasError()) return kernel_ctx.status(); } - return states; + + return std::move(states); } Result ResolveKernels( From 362aa4f6d806d7dbdd9cac3c87ca8678c531391c Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Wed, 17 Mar 2021 21:14:41 -0400 Subject: [PATCH 33/49] fix unaligned load --- cpp/src/arrow/compute/kernels/hash_aggregate.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 799a93f02d4..91613505d97 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -262,7 +262,7 @@ struct VarLengthKeyEncoder : KeyEncoder { Offset length_sum = 0; for (int32_t i = 0; i < length; ++i) { - length_sum += reinterpret_cast(encoded_bytes)[0]; + length_sum += util::SafeLoadAs(encoded_bytes[i]); } ARROW_ASSIGN_OR_RAISE(auto offset_buf, From a39e941b78738e42b13b6914e4ed1755471a31ec Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 18 Mar 2021 13:25:24 -0400 Subject: [PATCH 34/49] bitshift width was unclear --- cpp/src/arrow/compute/kernels/hash_aggregate_test.cc | 3 +-- 1 file changed, 1 insertion(+), 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 843dcc75f4a..163e03c7321 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -656,9 +656,8 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { TEST(GroupBy, RandomArraySum) { auto rand = random::RandomArrayGenerator(0xdeadbeef); - for (size_t i = 10; i < 14; i += 2) { + for (int64_t length : {1 << 10, 1 << 12, 1 << 15}) { for (auto null_probability : {0.0, 0.1, 0.5, 1.0}) { - int64_t length = 1UL << i; auto summand = rand.Float32(length, -100, 100, null_probability); auto key = rand.Int64(length, 0, 12); From b0e410fe50b6c7f494772eec2e215803b9a6031d Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Thu, 18 Mar 2021 13:27:00 -0400 Subject: [PATCH 35/49] msvc: explicit cast --- cpp/src/arrow/compute/kernels/hash_aggregate.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 91613505d97..ec6c8da1e6b 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -229,7 +229,8 @@ struct VarLengthKeyEncoder : KeyEncoder { VisitArrayDataInline( data, [&](util::string_view bytes) { - lengths[i++] += kExtraByteForNull + sizeof(Offset) + bytes.size(); + lengths[i++] += + kExtraByteForNull + sizeof(Offset) + static_cast(bytes.size()); }, [&] { lengths[i++] += kExtraByteForNull + sizeof(Offset); }); } From 926606763e258b38d3b65a479d4faa5ca980a551 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Fri, 19 Mar 2021 13:14:50 -0400 Subject: [PATCH 36/49] GroupIdentifier->Grouper, use HashAggregateFunctions to store kernels --- cpp/src/arrow/compute/api_aggregate.h | 12 +- cpp/src/arrow/compute/function.cc | 14 ++ cpp/src/arrow/compute/function.h | 15 ++ .../arrow/compute/kernels/hash_aggregate.cc | 141 ++++++++++++------ .../compute/kernels/hash_aggregate_test.cc | 37 ++--- cpp/src/arrow/compute/registry.cc | 13 +- cpp/src/arrow/compute/registry_internal.h | 1 + cpp/src/arrow/dataset/partition.cc | 8 +- 8 files changed, 160 insertions(+), 81 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index a16aad4f150..c7dd8019edc 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -310,13 +310,13 @@ namespace internal { /// Internal use only: streaming group identifier. /// Consumes batches of keys and yields batches of the group ids. -class ARROW_EXPORT GroupIdentifier { +class ARROW_EXPORT Grouper { public: - virtual ~GroupIdentifier() = default; + virtual ~Grouper() = default; - /// Construct a GroupIdentifier which receives the specified key types - static Result> Make( - ExecContext* ctx, const std::vector& descrs); + /// Construct a Grouper which receives the specified key types + static Result> Make(ExecContext* ctx, + const std::vector& descrs); /// Consume a batch of keys, producing an array of the corresponding /// group ids as an integer column. The yielded batch also includes the current group @@ -343,7 +343,7 @@ Result GroupBy(const std::vector& arguments, const std::vector& aggregates, ExecContext* ctx = NULLPTR); -/// Interna use only: Assemble lists of indices of identical elements. +/// Internal use only: Assemble lists of indices of identical elements. /// /// \param[in] ids An integral array which will be used as grouping criteria. /// Nulls are invalid. diff --git a/cpp/src/arrow/compute/function.cc b/cpp/src/arrow/compute/function.cc index 70d7d998e9c..815537b4bb9 100644 --- a/cpp/src/arrow/compute/function.cc +++ b/cpp/src/arrow/compute/function.cc @@ -126,6 +126,11 @@ const Kernel* DispatchExactImpl(const Function* func, checked_cast(func)->kernels(), values); } + if (func->kind() == Function::HASH_AGGREGATE) { + return DispatchExactImpl(checked_cast(func)->kernels(), + values); + } + return nullptr; } @@ -263,6 +268,15 @@ Status ScalarAggregateFunction::AddKernel(ScalarAggregateKernel kernel) { return Status::OK(); } +Status HashAggregateFunction::AddKernel(HashAggregateKernel kernel) { + RETURN_NOT_OK(CheckArity(kernel.signature->in_types())); + if (arity_.is_varargs && !kernel.signature->is_varargs()) { + return Status::Invalid("Function accepts varargs but kernel signature does not"); + } + kernels_.emplace_back(std::move(kernel)); + return Status::OK(); +} + Result MetaFunction::Execute(const std::vector& args, const FunctionOptions* options, ExecContext* ctx) const { diff --git a/cpp/src/arrow/compute/function.h b/cpp/src/arrow/compute/function.h index 52faa65199f..9a3e1c1852f 100644 --- a/cpp/src/arrow/compute/function.h +++ b/cpp/src/arrow/compute/function.h @@ -311,6 +311,21 @@ class ARROW_EXPORT ScalarAggregateFunction Status AddKernel(ScalarAggregateKernel kernel); }; +class ARROW_EXPORT HashAggregateFunction + : public detail::FunctionImpl { + public: + using KernelType = HashAggregateKernel; + + HashAggregateFunction(std::string name, const Arity& arity, const FunctionDoc* doc, + const FunctionOptions* default_options = NULLPTR) + : detail::FunctionImpl( + std::move(name), Function::HASH_AGGREGATE, arity, doc, default_options) {} + + /// \brief Add a kernel (function implementation). Returns error if the + /// kernel's signature does not match the function's arity. + Status AddKernel(HashAggregateKernel kernel); +}; + /// \brief A function that dispatches to other functions. Must implement /// MetaFunction::ExecuteImpl. /// diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index ec6c8da1e6b..34e268bdb17 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -297,10 +297,10 @@ struct VarLengthKeyEncoder : KeyEncoder { std::shared_ptr type_; }; -struct GroupIdentifierImpl : GroupIdentifier { - static Result> Make( - ExecContext* ctx, const std::vector& keys) { - auto impl = ::arrow::internal::make_unique(); +struct GrouperImpl : Grouper { + static Result> Make(ExecContext* ctx, + const std::vector& keys) { + auto impl = ::arrow::internal::make_unique(); impl->encoders_.resize(keys.size()); impl->ctx_ = ctx; @@ -752,37 +752,27 @@ struct GroupedMinMaxImpl : public GroupedAggregator { MinMaxOptions options_; }; -template -KernelInit MakeInit() { - return [](KernelContext* ctx, - const KernelInitArgs& args) -> std::unique_ptr { - auto impl = ::arrow::internal::make_unique(); +template +HashAggregateKernel MakeKernel(InputType argument_type) { + HashAggregateKernel kernel; + + kernel.init = [](KernelContext* ctx, + const KernelInitArgs& args) -> std::unique_ptr { + auto impl = ::arrow::internal::make_unique(); + // FIXME(bkietz) Init should not take a type. That should be an unboxed template arg + // for the Impl. Otherwise we're not exposing dispatch as well as we should. ctx->SetStatus(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); if (ctx->HasError()) return nullptr; return std::move(impl); }; -} - -// this isn't really in the spirit of things, but I'll get around to defining -// HashAggregateFunctions later -Result MakeKernel(const std::string& function_name) { - HashAggregateKernel kernel; - - if (function_name == "count") { - kernel.init = MakeInit(); - } else if (function_name == "sum") { - kernel.init = MakeInit(); - } else if (function_name == "min_max") { - kernel.init = MakeInit(); - } else { - return Status::NotImplemented("Grouped aggregate ", function_name); - } kernel.signature = KernelSignature::Make( - {{}, {}, {}}, OutputType([](KernelContext* ctx, - const std::vector&) -> Result { - return checked_cast(ctx->state())->out_type(); - })); + {std::move(argument_type), InputType::Array(Type::UINT32), + InputType::Scalar(Type::UINT32)}, + OutputType( + [](KernelContext* ctx, const std::vector&) -> Result { + return checked_cast(ctx->state())->out_type(); + })); kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { ctx->SetStatus(checked_cast(ctx->state())->Consume(batch)); @@ -801,23 +791,30 @@ Result MakeKernel(const std::string& function_name) { return kernel; } -Result> MakeKernels( - const std::vector& aggregates, const std::vector& in_descrs) { +Result> GetKernels( + ExecContext* ctx, const std::vector& aggregates, + const std::vector& in_descrs) { if (aggregates.size() != in_descrs.size()) { return Status::Invalid(aggregates.size(), " aggregate functions were specified but ", in_descrs.size(), " arguments were provided."); } - std::vector kernels(in_descrs.size()); + std::vector kernels(in_descrs.size()); for (size_t i = 0; i < aggregates.size(); ++i) { - ARROW_ASSIGN_OR_RAISE(kernels[i], MakeKernel(aggregates[i].function)); + ARROW_ASSIGN_OR_RAISE(auto function, + ctx->func_registry()->GetFunction(aggregates[i].function)); + ARROW_ASSIGN_OR_RAISE( + const Kernel* kernel, + function->DispatchExact( + {in_descrs[i], ValueDescr::Array(uint32()), ValueDescr::Scalar(uint32())})); + kernels[i] = static_cast(kernel); } return kernels; } Result>> InitKernels( - const std::vector& kernels, ExecContext* ctx, + const std::vector& kernels, ExecContext* ctx, const std::vector& aggregates, const std::vector& in_descrs) { std::vector> states(kernels.size()); @@ -833,8 +830,13 @@ Result>> InitKernels( } KernelContext kernel_ctx{ctx}; - states[i] = kernels[i].init( - &kernel_ctx, KernelInitArgs{&kernels[i], {in_descrs[i].type}, options}); + states[i] = kernels[i]->init(&kernel_ctx, KernelInitArgs{kernels[i], + { + in_descrs[i].type, + uint32(), + uint32(), + }, + options}); if (kernel_ctx.HasError()) return kernel_ctx.status(); } @@ -842,7 +844,7 @@ Result>> InitKernels( } Result ResolveKernels( - const std::vector& kernels, + const std::vector& kernels, const std::vector>& states, ExecContext* ctx, const std::vector& descrs) { FieldVector fields(descrs.size()); @@ -851,7 +853,7 @@ Result ResolveKernels( KernelContext kernel_ctx{ctx}; kernel_ctx.SetState(states[i].get()); - ARROW_ASSIGN_OR_RAISE(auto descr, kernels[i].signature->out_type().Resolve( + ARROW_ASSIGN_OR_RAISE(auto descr, kernels[i]->signature->out_type().Resolve( &kernel_ctx, { descrs[i].type, uint32(), @@ -864,9 +866,9 @@ Result ResolveKernels( } // namespace -Result> GroupIdentifier::Make( - ExecContext* ctx, const std::vector& descrs) { - return GroupIdentifierImpl::Make(ctx, descrs); +Result> Grouper::Make(ExecContext* ctx, + const std::vector& descrs) { + return GrouperImpl::Make(ctx, descrs); } Result GroupBy(const std::vector& arguments, const std::vector& keys, @@ -881,7 +883,7 @@ Result GroupBy(const std::vector& arguments, const std::vector GroupBy(const std::vector& arguments, const std::vectorexec_chunksize())); - // Construct GroupIdentifier + // Construct Grouper ARROW_ASSIGN_OR_RAISE(auto key_descrs, ExecBatch::Make(keys).Map([](ExecBatch batch) { return batch.GetDescriptors(); })); - ARROW_ASSIGN_OR_RAISE(auto group_identifier, GroupIdentifier::Make(ctx, key_descrs)); + ARROW_ASSIGN_OR_RAISE(auto grouper, Grouper::Make(ctx, key_descrs)); for (ValueDescr& key_descr : key_descrs) { out_fields.push_back(field("", std::move(key_descr.type))); @@ -915,7 +917,7 @@ Result GroupBy(const std::vector& arguments, const std::vectorConsume(key_batch)); + ARROW_ASSIGN_OR_RAISE(ExecBatch id_batch, grouper->Consume(key_batch)); // consume group ids with HashAggregateKernels for (size_t i = 0; i < kernels.size(); ++i) { @@ -923,7 +925,7 @@ Result GroupBy(const std::vector& arguments, const std::vectorconsume(&batch_ctx, batch); if (batch_ctx.HasError()) return batch_ctx.status(); } } @@ -936,12 +938,12 @@ Result GroupBy(const std::vector& arguments, const std::vectorfinalize(&batch_ctx, &out); if (batch_ctx.HasError()) return batch_ctx.status(); *it++ = out.array(); } - ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, group_identifier->GetUniques()); + ARROW_ASSIGN_OR_RAISE(ExecBatch out_keys, grouper->GetUniques()); for (const auto& key : out_keys.values) { *it++ = key.array(); } @@ -999,6 +1001,49 @@ Result> MakeGroupings(Datum ids, ExecContext* ctx) std::vector{"ids", "groupings"}); } +namespace { +const FunctionDoc count_doc{"Count the number of null / non-null values", + ("By default, non-null values are counted.\n" + "This can be changed through CountOptions."), + {"array", "group_id_array", "group_count"}, + "CountOptions"}; + +const FunctionDoc sum_doc{"Sum values of a numeric array", + ("Null values are ignored."), + {"array", "group_id_array", "group_count"}}; + +const FunctionDoc min_max_doc{"Compute the minimum and maximum values of a numeric array", + ("Null values are ignored by default.\n" + "This can be changed through MinMaxOptions."), + {"array", "group_id_array", "group_count"}, + "MinMaxOptions"}; +} // namespace + +void RegisterHashAggregateBasic(FunctionRegistry* registry) { + { + static auto default_count_options = CountOptions::Defaults(); + auto func = std::make_shared( + "hash_count", Arity::Ternary(), &count_doc, &default_count_options); + DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + DCHECK_OK(registry->AddFunction(std::move(func))); + } + + { + auto func = + std::make_shared("hash_sum", Arity::Ternary(), &sum_doc); + DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + DCHECK_OK(registry->AddFunction(std::move(func))); + } + + { + static auto default_minmax_options = MinMaxOptions::Defaults(); + auto func = std::make_shared( + "hash_min_max", Arity::Ternary(), &min_max_doc, &default_minmax_options); + DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); + DCHECK_OK(registry->AddFunction(std::move(func))); + } +} + } // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 163e03c7321..742ebf86b3c 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -335,13 +335,16 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys for (size_t i_agg = 0; i_agg < arguments.size(); ++i_agg) { ScalarVector aggregated_scalars; + // trim "hash_" prefix + auto scalar_agg_function = aggregates[i_agg].function.substr(5); + ARROW_ASSIGN_OR_RAISE(auto grouped_argument, ApplyGroupings(*groupings, *arguments[i_agg].make_array())); for (int64_t i_group = 0; i_group < n_groups; ++i_group) { - ARROW_ASSIGN_OR_RAISE(Datum d, - CallFunction(aggregates[i_agg].function, - {grouped_argument->value_slice(i_group)})); + ARROW_ASSIGN_OR_RAISE( + Datum d, + CallFunction(scalar_agg_function, {grouped_argument->value_slice(i_group)})); aggregated_scalars.push_back(d.scalar()); } @@ -402,7 +405,7 @@ TEST(GroupBy, SumOnlyBooleanKey) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"sum", nullptr}, + {"hash_sum", nullptr}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -425,7 +428,7 @@ TEST(GroupBy, SumOnly8bitKey) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"sum", nullptr}, + {"hash_sum", nullptr}, })); ASSERT_OK(aggregated_and_grouped.array_as()->ValidateFull()); @@ -451,7 +454,7 @@ TEST(GroupBy, SumOnly32bitKey) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"sum", nullptr}, + {"hash_sum", nullptr}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -476,7 +479,7 @@ TEST(GroupBy, SumOnly) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"sum", nullptr}, + {"hash_sum", nullptr}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -500,7 +503,7 @@ TEST(GroupBy, SumOnlyFloatingPointKey) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"sum", nullptr}, + {"hash_sum", nullptr}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -524,7 +527,7 @@ TEST(GroupBy, MinMaxOnly) { ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, internal::GroupBy({argument}, {key}, { - {"min_max", nullptr}, + {"hash_min_max", nullptr}, })); AssertDatumsEqual(ArrayFromJSON(struct_({ @@ -554,9 +557,9 @@ TEST(GroupBy, CountAndSum) { // NB: passing an argument twice or also using it as a key is legal internal::GroupBy({argument, argument, key}, {key}, { - {"count", &count_options}, - {"sum", nullptr}, - {"sum", nullptr}, + {"hash_count", &count_options}, + {"hash_sum", nullptr}, + {"hash_sum", nullptr}, })); AssertDatumsEqual( @@ -582,7 +585,7 @@ TEST(GroupBy, StringKey) { auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {key}, {{"hash_sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", int64()), @@ -604,7 +607,7 @@ TEST(GroupBy, DictKey) { R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {key}, {{"hash_sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", int64()), @@ -628,7 +631,7 @@ TEST(GroupBy, MultipleKeys) { ASSERT_OK_AND_ASSIGN( Datum aggregated_and_grouped, - internal::GroupBy({argument}, {int_key, str_key}, {{"sum", nullptr}})); + internal::GroupBy({argument}, {int_key, str_key}, {{"hash_sum", nullptr}})); AssertDatumsEqual(ArrayFromJSON(struct_({ field("", float64()), @@ -650,7 +653,7 @@ TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - group_helpers::ValidateGroupBy({{"sum", nullptr}}, {argument}, {key}); + group_helpers::ValidateGroupBy({{"hash_sum", nullptr}}, {argument}, {key}); } TEST(GroupBy, RandomArraySum) { @@ -663,7 +666,7 @@ TEST(GroupBy, RandomArraySum) { group_helpers::ValidateGroupBy( { - {"sum", nullptr}, + {"hash_sum", nullptr}, }, {summand}, {key}); } diff --git a/cpp/src/arrow/compute/registry.cc b/cpp/src/arrow/compute/registry.cc index 9385c5c2a16..3a8a3a0eb85 100644 --- a/cpp/src/arrow/compute/registry.cc +++ b/cpp/src/arrow/compute/registry.cc @@ -126,18 +126,19 @@ static std::unique_ptr CreateBuiltInRegistry() { RegisterScalarValidity(registry.get()); RegisterScalarFillNull(registry.get()); + // Vector functions + RegisterVectorHash(registry.get()); + RegisterVectorSelection(registry.get()); + RegisterVectorNested(registry.get()); + RegisterVectorSort(registry.get()); + // Aggregate functions RegisterScalarAggregateBasic(registry.get()); RegisterScalarAggregateMode(registry.get()); RegisterScalarAggregateQuantile(registry.get()); RegisterScalarAggregateTDigest(registry.get()); RegisterScalarAggregateVariance(registry.get()); - - // Vector functions - RegisterVectorHash(registry.get()); - RegisterVectorSelection(registry.get()); - RegisterVectorNested(registry.get()); - RegisterVectorSort(registry.get()); + RegisterHashAggregateBasic(registry.get()); return registry; } diff --git a/cpp/src/arrow/compute/registry_internal.h b/cpp/src/arrow/compute/registry_internal.h index 3b0f4475328..e4008cf3f27 100644 --- a/cpp/src/arrow/compute/registry_internal.h +++ b/cpp/src/arrow/compute/registry_internal.h @@ -47,6 +47,7 @@ void RegisterScalarAggregateMode(FunctionRegistry* registry); void RegisterScalarAggregateQuantile(FunctionRegistry* registry); void RegisterScalarAggregateTDigest(FunctionRegistry* registry); void RegisterScalarAggregateVariance(FunctionRegistry* registry); +void RegisterHashAggregateBasic(FunctionRegistry* registry); } // namespace internal } // namespace compute diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 5a4fc34b48d..cbdbbdfbbcb 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -592,10 +592,10 @@ Result> MakeGroupings(const StructArray& by) { } compute::ExecContext ctx; - ARROW_ASSIGN_OR_RAISE(auto group_identifier, compute::internal::GroupIdentifier::Make( - &ctx, key_batch.GetDescriptors())); + ARROW_ASSIGN_OR_RAISE( + auto grouper, compute::internal::Grouper::Make(&ctx, key_batch.GetDescriptors())); - ARROW_ASSIGN_OR_RAISE(auto id_batch, group_identifier->Consume(key_batch)); + ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); ARROW_ASSIGN_OR_RAISE(auto unique_ids_and_groupings, compute::internal::MakeGroupings(id_batch[0])); @@ -603,7 +603,7 @@ Result> MakeGroupings(const StructArray& by) { auto unique_ids = MakeArray(std::move(unique_ids_and_groupings->data()->child_data[0])); // if unique_ids is not sorted then groupings are out of order WRT groupings - ARROW_ASSIGN_OR_RAISE(auto uniques, group_identifier->GetUniques()); + ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); ArrayVector unique_rows_fields(uniques.num_values()); for (int i = 0; i < by.num_fields(); ++i) { unique_rows_fields[i] = uniques[i].make_array(); From 1c2973b5beb7a0ad21b2f38a3d2252a1d217a593 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Sat, 20 Mar 2021 21:35:07 -0400 Subject: [PATCH 37/49] add unit tests for Grouper --- cpp/src/arrow/array/array_binary.h | 14 +- cpp/src/arrow/array/array_primitive.h | 8 +- cpp/src/arrow/compute/api_aggregate.h | 36 ++-- cpp/src/arrow/compute/exec.cc | 34 ++++ cpp/src/arrow/compute/exec.h | 28 +-- .../arrow/compute/kernels/hash_aggregate.cc | 84 +++++---- .../compute/kernels/hash_aggregate_test.cc | 178 ++++++++++++++++++ cpp/src/arrow/dataset/partition.cc | 15 +- 8 files changed, 294 insertions(+), 103 deletions(-) diff --git a/cpp/src/arrow/array/array_binary.h b/cpp/src/arrow/array/array_binary.h index d3ae93318ba..db3c640b9a4 100644 --- a/cpp/src/arrow/array/array_binary.h +++ b/cpp/src/arrow/array/array_binary.h @@ -117,13 +117,13 @@ class BaseBinaryArray : public FlatArray { } } - IteratorType begin() { return IteratorType(*this); } + IteratorType begin() const { return IteratorType(*this); } - IteratorType end() { return IteratorType(*this, length()); } + IteratorType end() const { return IteratorType(*this, length()); } protected: // For subclasses - BaseBinaryArray() : raw_value_offsets_(NULLPTR), raw_data_(NULLPTR) {} + BaseBinaryArray() = default; // Protected method for constructors void SetData(const std::shared_ptr& data) { @@ -132,8 +132,8 @@ class BaseBinaryArray : public FlatArray { raw_data_ = data->GetValuesSafe(2, /*offset=*/0); } - const offset_type* raw_value_offsets_; - const uint8_t* raw_data_; + const offset_type* raw_value_offsets_ = NULLPTR; + const uint8_t* raw_data_ = NULLPTR; }; /// Concrete Array class for variable-size binary data @@ -231,9 +231,9 @@ class ARROW_EXPORT FixedSizeBinaryArray : public PrimitiveArray { const uint8_t* raw_values() const { return raw_values_ + data_->offset * byte_width_; } - IteratorType begin() { return IteratorType(*this); } + IteratorType begin() const { return IteratorType(*this); } - IteratorType end() { return IteratorType(*this, length()); } + IteratorType end() const { return IteratorType(*this, length()); } protected: void SetData(const std::shared_ptr& data) { diff --git a/cpp/src/arrow/array/array_primitive.h b/cpp/src/arrow/array/array_primitive.h index f9ac60f6cb9..b601eb770c3 100644 --- a/cpp/src/arrow/array/array_primitive.h +++ b/cpp/src/arrow/array/array_primitive.h @@ -64,9 +64,9 @@ class NumericArray : public PrimitiveArray { // For API compatibility with BinaryArray etc. value_type GetView(int64_t i) const { return Value(i); } - IteratorType begin() { return IteratorType(*this); } + IteratorType begin() const { return IteratorType(*this); } - IteratorType end() { return IteratorType(*this, length()); } + IteratorType end() const { return IteratorType(*this, length()); } protected: using PrimitiveArray::PrimitiveArray; @@ -99,9 +99,9 @@ class ARROW_EXPORT BooleanArray : public PrimitiveArray { /// values. Result is not cached. int64_t true_count() const; - IteratorType begin() { return IteratorType(*this); } + IteratorType begin() const { return IteratorType(*this); } - IteratorType end() { return IteratorType(*this, length()); } + IteratorType end() const { return IteratorType(*this, length()); } protected: using PrimitiveArray::PrimitiveArray; diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index c7dd8019edc..89b06357377 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -315,8 +315,8 @@ class ARROW_EXPORT Grouper { virtual ~Grouper() = default; /// Construct a Grouper which receives the specified key types - static Result> Make(ExecContext* ctx, - const std::vector& descrs); + static Result> Make(const std::vector& descrs, + ExecContext* ctx = NULLPTR); /// Consume a batch of keys, producing an array of the corresponding /// group ids as an integer column. The yielded batch also includes the current group @@ -345,26 +345,32 @@ Result GroupBy(const std::vector& arguments, const std::vector>`, -/// which is a mapping from unique ids to lists of -/// indices into `ids` where that value appears +/// \param[in] ids An unsigned, all-valid integral array which will be +/// used as grouping criteria. +/// \param[in] max_id An upper bound for the elements of ids +/// \return A (max_id + 1)-long ListArray where the slot at i contains a +/// list of indices where i appears in ids. /// /// MakeGroupings([ -/// 7, -/// 7, +/// 2, +/// 2, /// 5, /// 5, -/// 7, +/// 2, /// 3 -/// ]) == [ -/// {"ids": 7, "groupings": [0, 1, 4]}, -/// {"ids": 5, "groupings": [2, 3]}, -/// {"ids": 3, "groupings": [5]} +/// ], 7) == [ +/// [], +/// [], +/// [0, 1, 4], +/// [5], +/// [], +/// [2, 3], +/// [], +/// [] /// ] ARROW_EXPORT -Result> MakeGroupings(Datum ids, ExecContext* ctx = NULLPTR); +Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, + ExecContext* ctx = NULLPTR); } // namespace internal } // namespace compute diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index f76a311cf90..e6be9cd92e7 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -36,6 +36,7 @@ #include "arrow/compute/registry.h" #include "arrow/compute/util_internal.h" #include "arrow/datum.h" +#include "arrow/record_batch.h" #include "arrow/scalar.h" #include "arrow/status.h" #include "arrow/type.h" @@ -57,6 +58,39 @@ using internal::CpuInfo; namespace compute { +ExecBatch::ExecBatch(const RecordBatch& batch) + : values(batch.num_columns()), length(batch.num_rows()) { + auto columns = batch.column_data(); + std::move(columns.begin(), columns.end(), values.begin()); +} + +Result ExecBatch::Make(std::vector values) { + if (values.empty()) { + return Status::Invalid("Cannot infer ExecBatch length without at least one value"); + } + + int64_t length = -1; + for (const auto& value : values) { + if (value.is_scalar()) { + if (length == -1) { + length = 1; + } + continue; + } + + if (length == -1) { + length = value.length(); + continue; + } + + if (length != value.length()) { + return Status::Invalid( + "Arrays used to construct an ExecBatch must have equal length"); + } + } + + return ExecBatch(std::move(values), length); +} namespace { Result> AllocateDataBuffer(KernelContext* ctx, int64_t length, diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index 391dab3fa6a..ff1fbfb4ee5 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -169,33 +169,9 @@ struct ExecBatch { ExecBatch(std::vector values, int64_t length) : values(std::move(values)), length(length) {} - static Result Make(std::vector values) { - if (values.empty()) { - return Status::Invalid("Cannot infer ExecBatch length without at least one value"); - } - - int64_t length = -1; - for (const auto& value : values) { - if (value.is_scalar()) { - if (length == -1) { - length = 1; - } - continue; - } - - if (length == -1) { - length = value.length(); - continue; - } - - if (length != value.length()) { - return Status::Invalid( - "Arrays used to construct an ExecBatch must have equal length"); - } - } + explicit ExecBatch(const RecordBatch& batch); - return ExecBatch(std::move(values), length); - } + static Result Make(std::vector values); /// The values representing positional arguments to be passed to a kernel's /// exec function for processing. diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 34e268bdb17..623015b9806 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -298,8 +298,8 @@ struct VarLengthKeyEncoder : KeyEncoder { }; struct GrouperImpl : Grouper { - static Result> Make(ExecContext* ctx, - const std::vector& keys) { + static Result> Make(const std::vector& keys, + ExecContext* ctx) { auto impl = ::arrow::internal::make_unique(); impl->encoders_.resize(keys.size()); @@ -866,9 +866,13 @@ Result ResolveKernels( } // namespace -Result> Grouper::Make(ExecContext* ctx, - const std::vector& descrs) { - return GrouperImpl::Make(ctx, descrs); +Result> Grouper::Make(const std::vector& descrs, + ExecContext* ctx) { + if (ctx == nullptr) { + static ExecContext default_ctx; + return Make(descrs, &default_ctx); + } + return GrouperImpl::Make(descrs, ctx); } Result GroupBy(const std::vector& arguments, const std::vector& keys, @@ -901,7 +905,7 @@ Result GroupBy(const std::vector& arguments, const std::vector GroupBy(const std::vector& arguments, const std::vector> CountsToOffsets(std::shared_ptr counts) { - TypedBufferBuilder offset_builder; - RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); - - int32_t current_offset = 0; - offset_builder.UnsafeAppend(current_offset); - - for (int64_t i = 0; i < counts->length(); ++i) { - DCHECK_NE(counts->Value(i), 0); - current_offset += static_cast(counts->Value(i)); - offset_builder.UnsafeAppend(current_offset); - } - - return offset_builder.Finish(); -} -} // namespace - -Result> MakeGroupings(Datum ids, ExecContext* ctx) { +Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, + ExecContext* ctx) { if (ctx == nullptr) { ExecContext default_ctx; - return MakeGroupings(ids, &default_ctx); + return MakeGroupings(ids, max_id, &default_ctx); } if (ids.null_count() != 0) { return Status::Invalid("MakeGroupings with null ids"); } - ARROW_ASSIGN_OR_RAISE(auto sort_indices, - compute::SortIndices(ids, compute::SortOptions::Defaults(), ctx)); - - ARROW_ASSIGN_OR_RAISE(auto counts_and_values, compute::ValueCounts(ids)); - - auto unique_ids = counts_and_values->GetFieldByName("values"); + ARROW_ASSIGN_OR_RAISE( + auto offsets, AllocateBuffer(sizeof(int32_t) * (max_id + 2), ctx->memory_pool())); + auto raw_offsets = reinterpret_cast(offsets->mutable_data()); - auto counts = - checked_pointer_cast(counts_and_values->GetFieldByName("counts")); - ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); - - auto groupings = - std::make_shared(list(sort_indices->type()), unique_ids->length(), - std::move(offsets), std::move(sort_indices)); + std::memset(raw_offsets, 0, offsets->size()); + for (int i = 0; i < ids.length(); ++i) { + DCHECK_LE(ids.Value(i), max_id); + raw_offsets[ids.Value(i)] += 1; + } + int32_t length = 0; + for (uint32_t id = 0; id < max_id + 1; ++id) { + auto offset = raw_offsets[id]; + raw_offsets[id] = length; + length += offset; + } + raw_offsets[max_id + 1] = length; + DCHECK_EQ(ids.length(), length); + + ARROW_ASSIGN_OR_RAISE(auto offsets_copy, + offsets->CopySlice(0, offsets->size(), ctx->memory_pool())); + raw_offsets = reinterpret_cast(offsets_copy->mutable_data()); + + ARROW_ASSIGN_OR_RAISE(auto sort_indices, AllocateBuffer(sizeof(int32_t) * ids.length(), + ctx->memory_pool())); + auto raw_sort_indices = reinterpret_cast(sort_indices->mutable_data()); + for (int i = 0; i < ids.length(); ++i) { + raw_sort_indices[raw_offsets[ids.Value(i)]++] = i; + } - return StructArray::Make(ArrayVector{std::move(unique_ids), std::move(groupings)}, - std::vector{"ids", "groupings"}); + return std::make_shared( + list(int32()), max_id + 1, std::move(offsets), + std::make_shared(ids.length(), std::move(sort_indices))); } namespace { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 742ebf86b3c..a37c64e7cf7 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -31,6 +31,7 @@ #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/kernels/aggregate_internal.h" +#include "arrow/compute/kernels/codegen_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" #include "arrow/type.h" @@ -398,6 +399,183 @@ void ValidateGroupBy(const std::vector& aggregates, } // namespace } // namespace group_helpers +struct TestGrouper { + explicit TestGrouper(std::vector descrs) : descrs_(std::move(descrs)) { + grouper_ = internal::Grouper::Make(descrs_).ValueOrDie(); + + FieldVector fields; + for (const auto& descr : descrs_) { + fields.push_back(field("", descr.type)); + } + key_schema_ = schema(std::move(fields)); + } + + void ExpectConsume(const std::string& key_json, const std::string& expected) { + auto key_batch = ExecBatch(*RecordBatchFromJSON(key_schema_, key_json)); + + Datum ids; + ConsumeAndValidate(key_batch, &ids); + + AssertDatumsEqual(ArrayFromJSON(uint32(), expected), ids, + /*verbose=*/true); + } + + void ConsumeAndValidate(const ExecBatch& key_batch, Datum* ids = nullptr) { + ASSERT_OK_AND_ASSIGN(auto id_batch, grouper_->Consume(key_batch)); + + ValidateConsume(key_batch, id_batch); + + if (ids) { + *ids = id_batch[0]; + } + } + + void ValidateConsume(const ExecBatch& key_batch, const ExecBatch& id_batch) { + int64_t new_num_groups = id_batch[1].scalar_as().value; + + if (uniques_.length == -1) { + ASSERT_OK_AND_ASSIGN(uniques_, grouper_->GetUniques()); + } else if (new_num_groups > uniques_.length) { + ASSERT_OK_AND_ASSIGN(ExecBatch new_uniques, grouper_->GetUniques()); + + // check that uniques_ are prefixes of new_uniques + for (int i = 0; i < uniques_.num_values(); ++i) { + auto prefix = new_uniques[i].array()->Slice(0, uniques_.length); + AssertDatumsEqual(uniques_[i], prefix, /*verbose=*/true); + } + + uniques_ = std::move(new_uniques); + } + + // check that the ids encode an equivalent key sequence + for (int i = 0; i < key_batch.num_values(); ++i) { + SCOPED_TRACE(std::to_string(i) + "th key array"); + ASSERT_OK_AND_ASSIGN(auto expected, Take(uniques_[i], id_batch[0])); + AssertDatumsEqual(expected, key_batch[i], /*verbose=*/true); + } + } + + std::vector descrs_; + std::shared_ptr key_schema_; + std::unique_ptr grouper_; + ExecBatch uniques_ = ExecBatch({}, -1); +}; + +TEST(Grouper, BooleanKey) { + TestGrouper g({boolean()}); + + g.ExpectConsume("[[true], [true]]", "[0, 0]"); + + g.ExpectConsume("[[true], [true]]", "[0, 0]"); + + g.ExpectConsume("[[false], [null]]", "[1, 2]"); + + g.ExpectConsume("[[true], [false], [true], [false], [null], [false], [null]]", + "[0, 1, 0, 1, 2, 1, 2]"); +} + +TEST(Grouper, NumericKey) { + for (auto ty : internal::NumericTypes()) { + SCOPED_TRACE("key type: " + ty->ToString()); + + TestGrouper g({ty}); + + g.ExpectConsume("[[3], [3]]", "[0, 0]"); + + g.ExpectConsume("[[3], [3]]", "[0, 0]"); + + g.ExpectConsume("[[27], [81]]", "[1, 2]"); + + g.ExpectConsume("[[3], [27], [3], [27], [null], [81], [27], [81]]", + "[0, 1, 0, 1, 3, 2, 1, 2]"); + } +} + +TEST(Grouper, StringKey) { + for (auto ty : {utf8(), + // dictionary(int32(), utf8()), + large_utf8()}) { + SCOPED_TRACE("key type: " + ty->ToString()); + + TestGrouper g({ty}); + + g.ExpectConsume(R"([["eh"], ["eh"]])", "[0, 0]"); + + g.ExpectConsume(R"([["eh"], ["eh"]])", "[0, 0]"); + + g.ExpectConsume(R"([["bee"], [null]])", "[1, 2]"); + } +} + +TEST(Grouper, StringInt64Key) { + TestGrouper g({utf8(), int64()}); + + g.ExpectConsume(R"([["eh", 0], ["eh", 0]])", "[0, 0]"); + + g.ExpectConsume(R"([["eh", 0], ["eh", null]])", "[0, 1]"); + + g.ExpectConsume(R"([["eh", 1], ["bee", 1]])", "[2, 3]"); + + g.ExpectConsume(R"([["eh", null], ["bee", 1]])", "[1, 3]"); +} + +TEST(Grouper, DoubleStringInt64Key) { + TestGrouper g({float64(), utf8(), int64()}); + + g.ExpectConsume(R"([[1.5, "eh", 0], [1.5, "eh", 0]])", "[0, 0]"); + + g.ExpectConsume(R"([[1.5, "eh", 0], [1.5, "eh", 0]])", "[0, 0]"); + + g.ExpectConsume(R"([[1.0, "eh", 0], [1.0, "be", null]])", "[1, 2]"); + + // note: -0 and +0 hash differently + g.ExpectConsume(R"([[-0.0, "be", 7], [0.0, "be", 7]])", "[3, 4]"); +} + +TEST(Grouper, RandomInt64Keys) { + TestGrouper g({int64()}); + for (int i = 0; i < 4; ++i) { + SCOPED_TRACE(std::to_string(i) + "th key batch"); + + ExecBatch key_batch{ + *random::GenerateBatch(g.key_schema_->fields(), 1 << 12, 0xDEADBEEF)}; + g.ConsumeAndValidate(key_batch); + } +} + +TEST(Grouper, RandomStringInt64Keys) { + TestGrouper g({utf8(), int64()}); + for (int i = 0; i < 4; ++i) { + SCOPED_TRACE(std::to_string(i) + "th key batch"); + + ExecBatch key_batch{ + *random::GenerateBatch(g.key_schema_->fields(), 1 << 12, 0xDEADBEEF)}; + g.ConsumeAndValidate(key_batch); + } +} + +TEST(GroupByHelpers, MakeGroupings) { + auto ExpectGroupings = [](std::string ids_json, uint32_t max_id, + std::string expected_json) { + auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), ids_json)); + auto expected = ArrayFromJSON(list(int32()), expected_json); + + ASSERT_OK_AND_ASSIGN(auto actual, internal::MakeGroupings(*ids, max_id)); + AssertArraysEqual(*expected, *actual, /*verbose=*/true); + }; + + ExpectGroupings("[]", 0, "[[]]"); + + ExpectGroupings("[0, 0, 0]", 0, "[[0, 1, 2]]"); + + ExpectGroupings("[0, 0, 0, 1, 1, 2]", 3, "[[0, 1, 2], [3, 4], [5], []]"); + + ExpectGroupings("[2, 1, 2, 1, 1, 2]", 4, "[[], [1, 3, 4], [0, 2, 5], [], []]"); + + ExpectGroupings("[2, 2, 5, 5, 2, 3]", 7, + "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); +} + TEST(GroupBy, SumOnlyBooleanKey) { auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); auto key = ArrayFromJSON(boolean(), "[1, 0, 1, 0, null, 0, null]"); diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index cbdbbdfbbcb..0dc37604b50 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -591,17 +591,14 @@ Result> MakeGroupings(const StructArray& by) { key_batch.values.emplace_back(key); } - compute::ExecContext ctx; - ARROW_ASSIGN_OR_RAISE( - auto grouper, compute::internal::Grouper::Make(&ctx, key_batch.GetDescriptors())); + ARROW_ASSIGN_OR_RAISE(auto grouper, + compute::internal::Grouper::Make(key_batch.GetDescriptors())); ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); - ARROW_ASSIGN_OR_RAISE(auto unique_ids_and_groupings, - compute::internal::MakeGroupings(id_batch[0])); - - auto unique_ids = MakeArray(std::move(unique_ids_and_groupings->data()->child_data[0])); - // if unique_ids is not sorted then groupings are out of order WRT groupings + auto ids = id_batch[0].array_as(); + auto max_id = id_batch[1].scalar_as().value - 1; + ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::MakeGroupings(*ids, max_id)); ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); ArrayVector unique_rows_fields(uniques.num_values()); @@ -611,8 +608,6 @@ Result> MakeGroupings(const StructArray& by) { ARROW_ASSIGN_OR_RAISE(auto unique_rows, StructArray::Make(std::move(unique_rows_fields), by.type()->fields())); - auto groupings = MakeArray(std::move(unique_ids_and_groupings->data()->child_data[1])); - return StructArray::Make(ArrayVector{std::move(unique_rows), std::move(groupings)}, std::vector{"values", "groupings"}); } From a0114c97ce76f84d7d7d0654b7b88cce788e2ecb Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 10:29:07 -0400 Subject: [PATCH 38/49] remove dataset::{MakeGrouping,ApplyGroupings} --- cpp/src/arrow/compute/api_aggregate.h | 26 ++ .../arrow/compute/kernels/hash_aggregate.cc | 9 + .../compute/kernels/hash_aggregate_test.cc | 333 +++++------------- cpp/src/arrow/dataset/partition.cc | 160 ++++----- cpp/src/arrow/dataset/partition.h | 59 ---- cpp/src/arrow/dataset/partition_test.cc | 152 +------- cpp/src/arrow/record_batch.h | 2 +- 7 files changed, 209 insertions(+), 532 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 89b06357377..49a4eceee61 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -372,6 +372,32 @@ ARROW_EXPORT Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, ExecContext* ctx = NULLPTR); +/// \brief Produce a ListArray whose slots are selections of `array` which correspond to +/// the provided groupings. +/// +/// For example, +/// ApplyGroupings([ +/// [], +/// [], +/// [0, 1, 4], +/// [5], +/// [], +/// [2, 3], +/// [], +/// [] +/// ], [2, 2, 5, 5, 2, 3]) == [ +/// [], +/// [], +/// [2, 2, 2], +/// [3], +/// [], +/// [5, 5], +/// [], +/// [] +/// ] +Result> ApplyGroupings(const ListArray& groupings, + const Array& array); + } // namespace internal } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 623015b9806..8cc8c11a270 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -958,6 +958,15 @@ Result GroupBy(const std::vector& arguments, const std::vector> ApplyGroupings(const ListArray& groupings, + const Array& array) { + ARROW_ASSIGN_OR_RAISE(Datum sorted, + compute::Take(array, groupings.data()->child_data[0])); + + return std::make_shared(list(array.type()), groupings.length(), + groupings.value_offsets(), sorted.make_array()); +} + Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, ExecContext* ctx) { if (ctx == nullptr) { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index a37c64e7cf7..b0e7a52da8b 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -60,202 +60,6 @@ namespace compute { namespace group_helpers { namespace { -// Transform an array of counts to offsets which will divide a ListArray -// into an equal number of slices with corresponding lengths. -Result> CountsToOffsets(std::shared_ptr counts) { - TypedBufferBuilder offset_builder; - RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1)); - - int32_t current_offset = 0; - offset_builder.UnsafeAppend(current_offset); - - for (int64_t i = 0; i < counts->length(); ++i) { - DCHECK_NE(counts->Value(i), 0); - current_offset += static_cast(counts->Value(i)); - offset_builder.UnsafeAppend(current_offset); - } - - std::shared_ptr offsets; - RETURN_NOT_OK(offset_builder.Finish(&offsets)); - return offsets; -} - -class StructDictionary { - public: - struct Encoded { - std::shared_ptr indices; - std::shared_ptr dictionary; - }; - - static Result Encode(const ArrayVector& columns) { - Encoded out{nullptr, std::make_shared()}; - - for (const auto& column : columns) { - if (column->null_count() != 0) { - return Status::NotImplemented("Grouping on a field with nulls"); - } - - RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices)); - } - - return out; - } - - Result> Decode(std::shared_ptr fused_indices, - FieldVector fields) { - std::vector builders(dictionaries_.size()); - for (Int32Builder& b : builders) { - RETURN_NOT_OK(b.Resize(fused_indices->length())); - } - - std::vector codes(dictionaries_.size()); - for (int64_t i = 0; i < fused_indices->length(); ++i) { - Expand(fused_indices->Value(i), codes.data()); - - auto builder_it = builders.begin(); - for (int32_t index : codes) { - builder_it++->UnsafeAppend(index); - } - } - - ArrayVector columns(dictionaries_.size()); - for (size_t i = 0; i < dictionaries_.size(); ++i) { - std::shared_ptr indices; - RETURN_NOT_OK(builders[i].FinishInternal(&indices)); - - ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices)); - - if (fields[i]->type()->id() == Type::DICTIONARY) { - RETURN_NOT_OK(RestoreDictionaryEncoding( - checked_pointer_cast(fields[i]->type()), &column)); - } - - columns[i] = column.make_array(); - } - - return StructArray::Make(std::move(columns), std::move(fields)); - } - - private: - Status AddOne(Datum column, std::shared_ptr* fused_indices) { - if (column.type()->id() != Type::DICTIONARY) { - ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(std::move(column))); - } - - auto dict_column = column.array_as(); - dictionaries_.push_back(dict_column->dictionary()); - ARROW_ASSIGN_OR_RAISE(auto indices, compute::Cast(*dict_column->indices(), int32())); - - if (*fused_indices == nullptr) { - *fused_indices = checked_pointer_cast(std::move(indices)); - return IncreaseSize(); - } - - // It's useful to think about the case where each of dictionaries_ has size 10. - // In this case the decimal digit in the ones place is the code in dictionaries_[0], - // the tens place corresponds to the code in dictionaries_[1], etc. - // The incumbent indices must be shifted to the hundreds place so as not to collide. - ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices, - compute::Multiply(indices, MakeScalar(size_))); - - ARROW_ASSIGN_OR_RAISE(new_fused_indices, - compute::Add(new_fused_indices, *fused_indices)); - - *fused_indices = checked_pointer_cast(new_fused_indices.make_array()); - return IncreaseSize(); - } - - // expand a fused code into component dict codes, order is in order of addition - void Expand(int32_t fused_code, int32_t* codes) { - for (size_t i = 0; i < dictionaries_.size(); ++i) { - auto dictionary_size = static_cast(dictionaries_[i]->length()); - codes[i] = fused_code % dictionary_size; - fused_code /= dictionary_size; - } - } - - Status RestoreDictionaryEncoding(std::shared_ptr expected_type, - Datum* column) { - DCHECK_NE(column->type()->id(), Type::DICTIONARY); - ARROW_ASSIGN_OR_RAISE(*column, compute::DictionaryEncode(std::move(*column))); - - if (expected_type->index_type()->id() == Type::INT32) { - // dictionary_encode has already yielded the expected index_type - return Status::OK(); - } - - // cast the indices to the expected index type - auto dictionary = std::move(column->mutable_array()->dictionary); - column->mutable_array()->type = int32(); - - ARROW_ASSIGN_OR_RAISE(*column, - compute::Cast(std::move(*column), expected_type->index_type())); - - column->mutable_array()->dictionary = std::move(dictionary); - column->mutable_array()->type = expected_type; - return Status::OK(); - } - - Status IncreaseSize() { - auto factor = static_cast(dictionaries_.back()->length()); - - if (arrow::internal::MultiplyWithOverflow(size_, factor, &size_)) { - return Status::CapacityError("Max groups exceeded"); - } - return Status::OK(); - } - - int32_t size_ = 1; - ArrayVector dictionaries_; -}; - -Result> MakeGroupings(const StructArray& keys) { - if (keys.num_fields() == 0) { - return Status::Invalid("Grouping with no keys"); - } - - if (keys.null_count() != 0) { - return Status::Invalid("Grouping with null keys"); - } - - ARROW_ASSIGN_OR_RAISE(auto fused, StructDictionary::Encode(keys.fields())); - - ARROW_ASSIGN_OR_RAISE(auto sort_indices, compute::SortIndices(*fused.indices)); - ARROW_ASSIGN_OR_RAISE(Datum sorted, compute::Take(fused.indices, *sort_indices)); - fused.indices = checked_pointer_cast(sorted.make_array()); - - ARROW_ASSIGN_OR_RAISE(auto fused_counts_and_values, - compute::ValueCounts(fused.indices)); - fused.indices.reset(); - - auto unique_fused_indices = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("values")); - ARROW_ASSIGN_OR_RAISE( - auto unique_rows, - fused.dictionary->Decode(std::move(unique_fused_indices), keys.type()->fields())); - - auto counts = - checked_pointer_cast(fused_counts_and_values->GetFieldByName("counts")); - ARROW_ASSIGN_OR_RAISE(auto offsets, CountsToOffsets(std::move(counts))); - - auto grouped_sort_indices = - std::make_shared(list(sort_indices->type()), unique_rows->length(), - std::move(offsets), std::move(sort_indices)); - - return StructArray::Make( - ArrayVector{std::move(unique_rows), std::move(grouped_sort_indices)}, - std::vector{"values", "groupings"}); -} - -Result> ApplyGroupings(const ListArray& groupings, - const Array& array) { - ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(array, groupings.data()->child_data[0])); - - return std::make_shared(list(array.type()), groupings.length(), - groupings.value_offsets(), sorted.make_array()); -} - struct ScalarVectorToArray { template ::BuilderType, @@ -318,34 +122,35 @@ struct ScalarVectorToArray { Result NaiveGroupBy(std::vector arguments, std::vector keys, const std::vector& aggregates) { - ArrayVector keys_arrays; - for (const Datum& key : keys) keys_arrays.push_back(key.make_array()); - std::vector key_names(keys_arrays.size(), ""); - ARROW_ASSIGN_OR_RAISE(auto keys_struct, - StructArray::Make(std::move(keys_arrays), std::move(key_names))); + ARROW_ASSIGN_OR_RAISE(auto key_batch, ExecBatch::Make(std::move(keys))); - ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*keys_struct)); + ARROW_ASSIGN_OR_RAISE(auto grouper, + internal::Grouper::Make(key_batch.GetDescriptors())); - auto groupings = - checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); + ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); - int64_t n_groups = groupings->length(); + int64_t num_groups = id_batch[1].scalar_as().value; + ARROW_ASSIGN_OR_RAISE( + auto groupings, + internal::MakeGroupings(*id_batch[0].array_as(), num_groups - 1)); ArrayVector out_columns; - for (size_t i_agg = 0; i_agg < arguments.size(); ++i_agg) { - ScalarVector aggregated_scalars; - + for (size_t i = 0; i < arguments.size(); ++i) { // trim "hash_" prefix - auto scalar_agg_function = aggregates[i_agg].function.substr(5); + auto scalar_agg_function = aggregates[i].function.substr(5); - ARROW_ASSIGN_OR_RAISE(auto grouped_argument, - ApplyGroupings(*groupings, *arguments[i_agg].make_array())); + ARROW_ASSIGN_OR_RAISE( + auto grouped_argument, + internal::ApplyGroupings(*groupings, *arguments[i].make_array())); - for (int64_t i_group = 0; i_group < n_groups; ++i_group) { + ScalarVector aggregated_scalars; + + for (int64_t i_group = 0; i_group < num_groups; ++i_group) { + auto slice = grouped_argument->value_slice(i_group); + if (slice->length() == 0) continue; ARROW_ASSIGN_OR_RAISE( - Datum d, - CallFunction(scalar_agg_function, {grouped_argument->value_slice(i_group)})); + Datum d, CallFunction(scalar_agg_function, {slice}, aggregates[i].options)); aggregated_scalars.push_back(d.scalar()); } @@ -354,10 +159,9 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys out_columns.push_back(aggregated_column.make_array()); } - keys_struct = - checked_pointer_cast(groupings_and_values->GetFieldByName("values")); - for (size_t i_key = 0; i_key < arguments.size(); ++i_key) { - out_columns.push_back(keys_struct->field(i_key)); + ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); + for (const Datum& key : uniques.values) { + out_columns.push_back(key.make_array()); } std::vector out_names(out_columns.size(), ""); @@ -371,28 +175,6 @@ void ValidateGroupBy(const std::vector& aggregates, ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(arguments, keys, aggregates)); - // Ordering of groups is not important, so sort by key columns to ensure the comparison - // doesn't fail spuriously - - for (Datum* out : {&expected, &actual}) { - auto out_columns = out->array_as()->fields(); - - SortOptions sort_options; - FieldVector key_fields; - ArrayVector key_columns; - for (size_t i = 0; i < keys.size(); ++i) { - auto name = std::to_string(i); - sort_options.sort_keys.emplace_back(name); - key_fields.push_back(field(name, out_columns[0]->type())); - key_columns.push_back(out_columns[0]); - } - auto key_batch = RecordBatch::Make(schema(std::move(key_fields)), out->length(), - std::move(key_columns)); - - ASSERT_OK_AND_ASSIGN(Datum sort_indices, SortIndices(key_batch, sort_options)); - ASSERT_OK_AND_ASSIGN(*out, Take(*out, sort_indices, TakeOptions::NoBoundsCheck())); - } - AssertDatumsEqual(expected, actual, /*verbose=*/true); } @@ -411,13 +193,18 @@ struct TestGrouper { } void ExpectConsume(const std::string& key_json, const std::string& expected) { - auto key_batch = ExecBatch(*RecordBatchFromJSON(key_schema_, key_json)); + ExpectConsume(ExecBatch(*RecordBatchFromJSON(key_schema_, key_json)), + ArrayFromJSON(uint32(), expected)); + } + void ExpectConsume(const std::vector& key_batch, Datum expected) { + ExpectConsume(*ExecBatch::Make(key_batch), expected); + } + + void ExpectConsume(const ExecBatch& key_batch, Datum expected) { Datum ids; ConsumeAndValidate(key_batch, &ids); - - AssertDatumsEqual(ArrayFromJSON(uint32(), expected), ids, - /*verbose=*/true); + AssertDatumsEqual(expected, ids, /*verbose=*/true); } void ConsumeAndValidate(const ExecBatch& key_batch, Datum* ids = nullptr) { @@ -492,9 +279,7 @@ TEST(Grouper, NumericKey) { } TEST(Grouper, StringKey) { - for (auto ty : {utf8(), - // dictionary(int32(), utf8()), - large_utf8()}) { + for (auto ty : {utf8(), large_utf8()}) { SCOPED_TRACE("key type: " + ty->ToString()); TestGrouper g({ty}); @@ -507,6 +292,34 @@ TEST(Grouper, StringKey) { } } +TEST(Grouper, DictKey) { + TestGrouper g({dictionary(int32(), utf8())}); + + // unification of dictionaries on encode is not yet supported + const auto dict = ArrayFromJSON(utf8(), R"(["ex", "why", "zee", null])"); + + auto WithIndices = [&](const std::string& indices) { + return Datum(*DictionaryArray::FromArrays(ArrayFromJSON(int32(), indices), dict)); + }; + + // NB: null index is not considered equivalent to index=3 (which encodes null in dict) + g.ExpectConsume({WithIndices(" [3, 1, null, 0, 2]")}, + ArrayFromJSON(uint32(), "[0, 1, 2, 3, 4]")); + + g = TestGrouper({dictionary(int32(), utf8())}); + + g.ExpectConsume({WithIndices(" [0, 1, 2, 3, null]")}, + ArrayFromJSON(uint32(), "[0, 1, 2, 3, 4]")); + + g.ExpectConsume({WithIndices(" [3, 1, null, 0, 2]")}, + ArrayFromJSON(uint32(), "[3, 1, 4, 0, 2]")); + + ASSERT_RAISES(NotImplemented, + g.grouper_->Consume(*ExecBatch::Make({*DictionaryArray::FromArrays( + ArrayFromJSON(int32(), "[0, 1]"), + ArrayFromJSON(utf8(), R"(["different", "dictionary"])"))}))); +} + TEST(Grouper, StringInt64Key) { TestGrouper g({utf8(), int64()}); @@ -517,6 +330,32 @@ TEST(Grouper, StringInt64Key) { g.ExpectConsume(R"([["eh", 1], ["bee", 1]])", "[2, 3]"); g.ExpectConsume(R"([["eh", null], ["bee", 1]])", "[1, 3]"); + + g = TestGrouper({utf8(), int64()}); + + g.ExpectConsume(R"([ + ["ex", 0], + ["ex", 0], + ["why", 0], + ["ex", 1], + ["why", 0], + ["ex", 1], + ["ex", 0], + ["why", 1] + ])", + "[0, 0, 1, 2, 1, 2, 0, 3]"); + + g.ExpectConsume(R"([ + ["ex", 0], + [null, 0], + [null, 0], + ["ex", 1], + [null, null], + ["ex", 1], + ["ex", 0], + ["why", null] + ])", + "[0, 4, 4, 2, 5, 2, 0, 6]"); } TEST(Grouper, DoubleStringInt64Key) { diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 0dc37604b50..7fc4d7744a0 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -71,56 +71,90 @@ std::shared_ptr Partitioning::Default() { return std::make_shared(); } -inline Expression ConjunctionFromGroupingRow(Scalar* row) { - ScalarVector* values = &checked_cast(row)->value; - std::vector equality_expressions(values->size()); - for (size_t i = 0; i < values->size(); ++i) { - const std::string& name = row->type->field(static_cast(i))->name(); - if (values->at(i)->is_valid) { - equality_expressions[i] = equal(field_ref(name), literal(std::move(values->at(i)))); - } else { - equality_expressions[i] = is_null(field_ref(name)); - } +static Result ApplyGroupings( + const ListArray& groupings, const std::shared_ptr& batch) { + ARROW_ASSIGN_OR_RAISE(Datum sorted, + compute::Take(batch, groupings.data()->child_data[0])); + + const auto& sorted_batch = *sorted.record_batch(); + + RecordBatchVector out(static_cast(groupings.length())); + for (size_t i = 0; i < out.size(); ++i) { + out[i] = sorted_batch.Slice(groupings.value_offset(i), groupings.value_length(i)); } - return and_(std::move(equality_expressions)); + + return out; } Result KeyValuePartitioning::Partition( const std::shared_ptr& batch) const { - FieldVector by_fields; - ArrayVector by_columns; + std::vector key_indices; + int num_keys = 0; - std::shared_ptr rest = batch; + // assemble vector of indices of fields in batch on which we'll partition for (const auto& partition_field : schema_->fields()) { ARROW_ASSIGN_OR_RAISE( - auto match, FieldRef(partition_field->name()).FindOneOrNone(*rest->schema())) - if (match.empty()) continue; + auto match, FieldRef(partition_field->name()).FindOneOrNone(*batch->schema())) - by_fields.push_back(partition_field); - by_columns.push_back(rest->column(match[0])); - ARROW_ASSIGN_OR_RAISE(rest, rest->RemoveColumn(match[0])); + if (match.empty()) continue; + key_indices.push_back(match[0]); + ++num_keys; } - if (by_fields.empty()) { + if (key_indices.empty()) { // no fields to group by; return the whole batch return PartitionedBatches{{batch}, {literal(true)}}; } - ARROW_ASSIGN_OR_RAISE(auto by, - StructArray::Make(std::move(by_columns), std::move(by_fields))); - ARROW_ASSIGN_OR_RAISE(auto groupings_and_values, MakeGroupings(*by)); - auto groupings = - checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); - auto unique_rows = groupings_and_values->GetFieldByName("values"); + // assemble an ExecBatch of the key columns + compute::ExecBatch key_batch({}, batch->num_rows()); + for (int i : key_indices) { + key_batch.values.emplace_back(batch->column_data(i)); + } + + ARROW_ASSIGN_OR_RAISE(auto grouper, + compute::internal::Grouper::Make(key_batch.GetDescriptors())); + + ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); + + int64_t num_groups = id_batch[1].scalar_as().value; + auto ids = id_batch[0].array_as(); + ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::MakeGroupings( + *ids, static_cast(num_groups - 1))); + + ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); + ArrayVector unique_arrays(num_keys); + for (int i = 0; i < num_keys; ++i) { + unique_arrays[i] = uniques.values[i].make_array(); + } PartitionedBatches out; - ARROW_ASSIGN_OR_RAISE(out.batches, ApplyGroupings(*groupings, rest)); - out.expressions.resize(out.batches.size()); - for (size_t i = 0; i < out.batches.size(); ++i) { - ARROW_ASSIGN_OR_RAISE(auto row, unique_rows->GetScalar(i)); - out.expressions[i] = ConjunctionFromGroupingRow(row.get()); + // assemble partition expressions from the unique keys + out.expressions.resize(static_cast(num_groups)); + for (int64_t group = 0; group < num_groups; ++group) { + std::vector exprs(num_keys); + + for (int i = 0; i < num_keys; ++i) { + ARROW_ASSIGN_OR_RAISE(auto val, unique_arrays[i]->GetScalar(group)); + const auto& name = batch->schema()->field(key_indices[i])->name(); + + exprs[i] = val->is_valid ? equal(field_ref(name), literal(std::move(val))) + : is_null(field_ref(name)); + } + out.expressions[group] = and_(std::move(exprs)); + } + + // remove key columns from batch to which we'll be applying the groupings + auto rest = batch; + std::sort(key_indices.begin(), key_indices.end(), std::greater()); + for (int i : key_indices) { + // indices are in descending order; indices larger than i (which would be invalidated + // here) have already been handled + ARROW_ASSIGN_OR_RAISE(rest, rest->RemoveColumn(i)); } + ARROW_ASSIGN_OR_RAISE(out.batches, ApplyGroupings(*groupings, rest)); + return out; } @@ -273,10 +307,11 @@ Result DirectoryPartitioning::FormatValues( return fs::internal::JoinAbstractPath(std::move(segments)); } +namespace { class KeyValuePartitioningFactory : public PartitioningFactory { protected: explicit KeyValuePartitioningFactory(PartitioningFactoryOptions options) - : options_(options) {} + : options_(std::move(options)) {} int GetOrInsertField(const std::string& name) { auto it_inserted = @@ -437,6 +472,8 @@ class DirectoryPartitioningFactory : public KeyValuePartitioningFactory { std::vector field_names_; }; +} // namespace + std::shared_ptr DirectoryPartitioning::MakeFactory( std::vector field_names, PartitioningFactoryOptions options) { return std::shared_ptr( @@ -577,64 +614,5 @@ Result> PartitioningOrFactory::GetOrInferSchema( return factory()->Inspect(paths); } -Result> MakeGroupings(const StructArray& by) { - if (by.num_fields() == 0) { - return Status::Invalid("Grouping with no criteria"); - } - - if (by.null_count() != 0) { - return Status::Invalid("Grouping with null criteria"); - } - - compute::ExecBatch key_batch({}, by.length()); - for (const auto& key : by.fields()) { - key_batch.values.emplace_back(key); - } - - ARROW_ASSIGN_OR_RAISE(auto grouper, - compute::internal::Grouper::Make(key_batch.GetDescriptors())); - - ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); - - auto ids = id_batch[0].array_as(); - auto max_id = id_batch[1].scalar_as().value - 1; - ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::MakeGroupings(*ids, max_id)); - - ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); - ArrayVector unique_rows_fields(uniques.num_values()); - for (int i = 0; i < by.num_fields(); ++i) { - unique_rows_fields[i] = uniques[i].make_array(); - } - ARROW_ASSIGN_OR_RAISE(auto unique_rows, StructArray::Make(std::move(unique_rows_fields), - by.type()->fields())); - - return StructArray::Make(ArrayVector{std::move(unique_rows), std::move(groupings)}, - std::vector{"values", "groupings"}); -} - -Result> ApplyGroupings(const ListArray& groupings, - const Array& array) { - ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(array, groupings.data()->child_data[0])); - - return std::make_shared(list(array.type()), groupings.length(), - groupings.value_offsets(), sorted.make_array()); -} - -Result ApplyGroupings(const ListArray& groupings, - const std::shared_ptr& batch) { - ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(batch, groupings.data()->child_data[0])); - - const auto& sorted_batch = *sorted.record_batch(); - - RecordBatchVector out(static_cast(groupings.length())); - for (size_t i = 0; i < out.size(); ++i) { - out[i] = sorted_batch.Slice(groupings.value_offset(i), groupings.value_length(i)); - } - - return out; -} - } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/partition.h b/cpp/src/arrow/dataset/partition.h index bd739f4cf09..74e6c607106 100644 --- a/cpp/src/arrow/dataset/partition.h +++ b/cpp/src/arrow/dataset/partition.h @@ -299,64 +299,5 @@ class ARROW_DS_EXPORT PartitioningOrFactory { std::shared_ptr partitioning_; }; -/// \brief Assemble lists of indices of identical rows. -/// -/// \param[in] by A StructArray whose columns will be used as grouping criteria. -/// Top level nulls are invalid, as are empty criteria (no grouping -/// columns). -/// \return A array of type `struct>`, -/// which is a mapping from unique rows (field "values") to lists of -/// indices into `by` where that row appears (field "groupings"). -/// -/// For example, -/// MakeGroupings([ -/// {"a": "ex", "b": 0}, -/// {"a": "ex", "b": 0}, -/// {"a": "why", "b": 0}, -/// {"a": "why", "b": 0}, -/// {"a": "ex", "b": 0}, -/// {"a": "why", "b": 1} -/// ]) == [ -/// {"values": {"a": "ex", "b": 0}, "groupings": [0, 1, 4]}, -/// {"values": {"a": "why", "b": 0}, "groupings": [2, 3]}, -/// {"values": {"a": "why", "b": 1}, "groupings": [5]} -/// ] -ARROW_DS_EXPORT -Result> MakeGroupings(const StructArray& by); - -/// \brief Produce a ListArray whose slots are selections of `array` which correspond to -/// the provided groupings. -/// -/// For example, -/// ApplyGroupings([[0, 1, 4], [2, 3], [5]], [ -/// {"a": "ex", "b": 0, "passenger": 0}, -/// {"a": "ex", "b": 0, "passenger": 1}, -/// {"a": "why", "b": 0, "passenger": 2}, -/// {"a": "why", "b": 0, "passenger": 3}, -/// {"a": "ex", "b": 0, "passenger": 4}, -/// {"a": "why", "b": 1, "passenger": 5} -/// ]) == [ -/// [ -/// {"a": "ex", "b": 0, "passenger": 0}, -/// {"a": "ex", "b": 0, "passenger": 1}, -/// {"a": "ex", "b": 0, "passenger": 4}, -/// ], -/// [ -/// {"a": "why", "b": 0, "passenger": 2}, -/// {"a": "why", "b": 0, "passenger": 3}, -/// ], -/// [ -/// {"a": "why", "b": 1, "passenger": 5} -/// ] -/// ] -ARROW_DS_EXPORT -Result> ApplyGroupings(const ListArray& groupings, - const Array& array); - -/// \brief Produce selections of a RecordBatch which correspond to the provided groupings. -ARROW_DS_EXPORT -Result ApplyGroupings(const ListArray& groupings, - const std::shared_ptr& batch); - } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/dataset/partition_test.cc b/cpp/src/arrow/dataset/partition_test.cc index 456b2852311..06c3cc67674 100644 --- a/cpp/src/arrow/dataset/partition_test.cc +++ b/cpp/src/arrow/dataset/partition_test.cc @@ -138,9 +138,13 @@ class TestPartitioning : public ::testing::Test { }; TEST_F(TestPartitioning, Partition) { + auto dataset_schema = + schema({field("a", int32()), field("b", utf8()), field("c", uint32())}); + auto partition_schema = schema({field("a", int32()), field("b", utf8())}); - auto schema_ = schema({field("a", int32()), field("b", utf8()), field("c", uint32())}); - auto remaining_schema = schema({field("c", uint32())}); + + auto physical_schema = schema({field("c", uint32())}); + auto partitioning = std::make_shared(partition_schema); std::string json = R"([{"a": 3, "b": "x", "c": 0}, {"a": 3, "b": "x", "c": 1}, @@ -149,15 +153,22 @@ TEST_F(TestPartitioning, Partition) { {"a": null, "b": "z", "c": 4}, {"a": null, "b": null, "c": 5} ])"; - std::vector expected_batches = {R"([{"c": 0}, {"c": 1}])", R"([{"c": 2}])", - R"([{"c": 3}, {"c": 5}])", - R"([{"c": 4}])"}; + + std::vector expected_batches = { + R"([{"c": 0}, {"c": 1}])", + R"([{"c": 2}])", + R"([{"c": 3}, {"c": 5}])", + R"([{"c": 4}])", + }; + std::vector expected_expressions = { and_(equal(field_ref("a"), literal(3)), equal(field_ref("b"), literal("x"))), and_(equal(field_ref("a"), literal(1)), is_null(field_ref("b"))), and_(is_null(field_ref("a")), is_null(field_ref("b"))), - and_(is_null(field_ref("a")), equal(field_ref("b"), literal("z")))}; - AssertPartition(partitioning, schema_, json, remaining_schema, expected_batches, + and_(is_null(field_ref("a")), equal(field_ref("b"), literal("z"))), + }; + + AssertPartition(partitioning, dataset_schema, json, physical_schema, expected_batches, expected_expressions); } @@ -713,132 +724,5 @@ TEST(TestStripPrefixAndFilename, Basic) { "year=2019/month=12/day=01")); } -void AssertGrouping(const FieldVector& by_fields, const std::string& batch_json, - const std::string& expected_json) { - FieldVector fields_with_ids = by_fields; - fields_with_ids.push_back(field("ids", list(int32()))); - auto expected = ArrayFromJSON(struct_(fields_with_ids), expected_json); - - FieldVector fields_with_id = by_fields; - fields_with_id.push_back(field("id", int32())); - auto batch = RecordBatchFromJSON(schema(fields_with_id), batch_json); - - ASSERT_OK_AND_ASSIGN(auto by, batch->RemoveColumn(batch->num_columns() - 1) - .Map([](std::shared_ptr by) { - return by->ToStructArray(); - })); - - ASSERT_OK_AND_ASSIGN(auto groupings_and_values, MakeGroupings(*by)); - ASSERT_OK(groupings_and_values->ValidateFull()); - - auto groupings = - checked_pointer_cast(groupings_and_values->GetFieldByName("groupings")); - - ASSERT_OK_AND_ASSIGN(std::shared_ptr grouped_ids, - ApplyGroupings(*groupings, *batch->GetColumnByName("id"))); - ASSERT_OK(grouped_ids->ValidateFull()); - - ArrayVector columns = - checked_cast(*groupings_and_values->GetFieldByName("values")) - .fields(); - columns.push_back(grouped_ids); - - ASSERT_OK_AND_ASSIGN(auto actual, StructArray::Make(columns, fields_with_ids)); - ASSERT_OK(actual->ValidateFull()); - - AssertArraysEqual(*expected, *actual, /*verbose=*/true); -} - -TEST(GroupTest, Basics) { - AssertGrouping({field("a", utf8()), field("b", int32())}, R"([ - {"a": "ex", "b": 0, "id": 0}, - {"a": "ex", "b": 0, "id": 1}, - {"a": "why", "b": 0, "id": 2}, - {"a": "ex", "b": 1, "id": 3}, - {"a": "why", "b": 0, "id": 4}, - {"a": "ex", "b": 1, "id": 5}, - {"a": "ex", "b": 0, "id": 6}, - {"a": "why", "b": 1, "id": 7} - ])", - R"([ - {"a": "ex", "b": 0, "ids": [0, 1, 6]}, - {"a": "why", "b": 0, "ids": [2, 4]}, - {"a": "ex", "b": 1, "ids": [3, 5]}, - {"a": "why", "b": 1, "ids": [7]} - ])"); -} - -TEST(GroupTest, WithNulls) { - AssertGrouping({field("a", utf8()), field("b", int32())}, - R"([ - {"a": "ex", "b": 0, "id": 0}, - {"a": null, "b": 0, "id": 1}, - {"a": null, "b": 0, "id": 2}, - {"a": "ex", "b": 1, "id": 3}, - {"a": null, "b": null, "id": 4}, - {"a": "ex", "b": 1, "id": 5}, - {"a": "ex", "b": 0, "id": 6}, - {"a": "why", "b": null, "id": 7} - ])", - R"([ - {"a": "ex", "b": 0, "ids": [0, 6]}, - {"a": null, "b": 0, "ids": [1, 2]}, - {"a": "ex", "b": 1, "ids": [3, 5]}, - {"a": null, "b": null, "ids": [4]}, - {"a": "why", "b": null, "ids": [7]} - ])"); - - AssertGrouping({field("a", dictionary(int32(), utf8())), field("b", int32())}, - R"([ - {"a": "ex", "b": 0, "id": 0}, - {"a": null, "b": 0, "id": 1}, - {"a": null, "b": 0, "id": 2}, - {"a": "ex", "b": 1, "id": 3}, - {"a": null, "b": null, "id": 4}, - {"a": "ex", "b": 1, "id": 5}, - {"a": "ex", "b": 0, "id": 6}, - {"a": "why", "b": null, "id": 7} - ])", - R"([ - {"a": "ex", "b": 0, "ids": [0, 6]}, - {"a": null, "b": 0, "ids": [1, 2]}, - {"a": "ex", "b": 1, "ids": [3, 5]}, - {"a": null, "b": null, "ids": [4]}, - {"a": "why", "b": null, "ids": [7]} - ])"); - - auto has_nulls = checked_pointer_cast( - ArrayFromJSON(struct_({field("a", utf8()), field("b", int32())}), R"([ - {"a": "ex", "b": 0}, - null, - {"a": "why", "b": 0}, - {"a": "ex", "b": 1}, - {"a": "why", "b": 0}, - {"a": "ex", "b": 1}, - {"a": "ex", "b": 0}, - null - ])")); - ASSERT_RAISES(Invalid, MakeGroupings(*has_nulls)); -} - -TEST(GroupTest, GroupOnDictionary) { - AssertGrouping({field("a", dictionary(int32(), utf8())), field("b", int32())}, R"([ - {"a": "ex", "b": 0, "id": 0}, - {"a": "ex", "b": 0, "id": 1}, - {"a": "why", "b": 0, "id": 2}, - {"a": "ex", "b": 1, "id": 3}, - {"a": "why", "b": 0, "id": 4}, - {"a": "ex", "b": 1, "id": 5}, - {"a": "ex", "b": 0, "id": 6}, - {"a": "why", "b": 1, "id": 7} - ])", - R"([ - {"a": "ex", "b": 0, "ids": [0, 1, 6]}, - {"a": "why", "b": 0, "ids": [2, 4]}, - {"a": "ex", "b": 1, "ids": [3, 5]}, - {"a": "why", "b": 1, "ids": [7]} - ])"); -} - } // namespace dataset } // namespace arrow diff --git a/cpp/src/arrow/record_batch.h b/cpp/src/arrow/record_batch.h index 95229eb78d5..4650e806360 100644 --- a/cpp/src/arrow/record_batch.h +++ b/cpp/src/arrow/record_batch.h @@ -87,7 +87,7 @@ class ARROW_EXPORT RecordBatch { // \return the table's schema /// \return true if batches are equal - std::shared_ptr schema() const { return schema_; } + const std::shared_ptr& schema() const { return schema_; } /// \brief Retrieve all columns at once std::vector> columns() const; From 63b833d60010b68975b8a81dfdbb1bdcdc32dc5b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 10:53:24 -0400 Subject: [PATCH 39/49] provide Grouper::num_groups() --- cpp/src/arrow/compute/api_aggregate.h | 123 +++++++++--------- cpp/src/arrow/compute/exec.cc | 5 + cpp/src/arrow/compute/exec.h | 2 + .../arrow/compute/kernels/hash_aggregate.cc | 41 +++--- .../compute/kernels/hash_aggregate_test.cc | 27 ++-- cpp/src/arrow/dataset/partition.cc | 8 +- 6 files changed, 100 insertions(+), 106 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 49a4eceee61..31e727632c8 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -316,15 +316,71 @@ class ARROW_EXPORT Grouper { /// Construct a Grouper which receives the specified key types static Result> Make(const std::vector& descrs, - ExecContext* ctx = NULLPTR); + ExecContext* ctx = default_exec_context()); - /// Consume a batch of keys, producing an array of the corresponding - /// group ids as an integer column. The yielded batch also includes the current group - /// count, which is necessary for efficient resizing of kernel storage. - virtual Result Consume(const ExecBatch& batch) = 0; + /// Consume a batch of keys, producing the corresponding group ids as an integer array. + virtual Result Consume(const ExecBatch& batch) = 0; /// Get current unique keys. May be called multiple times. virtual Result GetUniques() = 0; + + /// Get the current number of groups. + virtual uint32_t num_groups() const = 0; + + /// \brief Assemble lists of indices of identical elements. + /// + /// \param[in] ids An unsigned, all-valid integral array which will be + /// used as grouping criteria. + /// \param[in] max_id An upper bound for the elements of ids + /// \return A (max_id + 1)-long ListArray where the slot at i contains a + /// list of indices where i appears in ids. + /// + /// MakeGroupings([ + /// 2, + /// 2, + /// 5, + /// 5, + /// 2, + /// 3 + /// ], 7) == [ + /// [], + /// [], + /// [0, 1, 4], + /// [5], + /// [], + /// [2, 3], + /// [], + /// [] + /// ] + static Result> MakeGroupings( + const UInt32Array& ids, uint32_t max_id, ExecContext* ctx = default_exec_context()); + + /// \brief Produce a ListArray whose slots are selections of `array` which correspond to + /// the provided groupings. + /// + /// For example, + /// ApplyGroupings([ + /// [], + /// [], + /// [0, 1, 4], + /// [5], + /// [], + /// [2, 3], + /// [], + /// [] + /// ], [2, 2, 5, 5, 2, 3]) == [ + /// [], + /// [], + /// [2, 2, 2], + /// [3], + /// [], + /// [5, 5], + /// [], + /// [] + /// ] + static Result> ApplyGroupings( + const ListArray& groupings, const Array& array, + ExecContext* ctx = default_exec_context()); }; /// \brief Configure a grouped aggregation @@ -341,62 +397,7 @@ struct ARROW_EXPORT Aggregate { ARROW_EXPORT Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, - ExecContext* ctx = NULLPTR); - -/// Internal use only: Assemble lists of indices of identical elements. -/// -/// \param[in] ids An unsigned, all-valid integral array which will be -/// used as grouping criteria. -/// \param[in] max_id An upper bound for the elements of ids -/// \return A (max_id + 1)-long ListArray where the slot at i contains a -/// list of indices where i appears in ids. -/// -/// MakeGroupings([ -/// 2, -/// 2, -/// 5, -/// 5, -/// 2, -/// 3 -/// ], 7) == [ -/// [], -/// [], -/// [0, 1, 4], -/// [5], -/// [], -/// [2, 3], -/// [], -/// [] -/// ] -ARROW_EXPORT -Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, - ExecContext* ctx = NULLPTR); - -/// \brief Produce a ListArray whose slots are selections of `array` which correspond to -/// the provided groupings. -/// -/// For example, -/// ApplyGroupings([ -/// [], -/// [], -/// [0, 1, 4], -/// [5], -/// [], -/// [2, 3], -/// [], -/// [] -/// ], [2, 2, 5, 5, 2, 3]) == [ -/// [], -/// [], -/// [2, 2, 2], -/// [3], -/// [], -/// [5, 5], -/// [], -/// [] -/// ] -Result> ApplyGroupings(const ListArray& groupings, - const Array& array); + ExecContext* ctx = default_exec_context()); } // namespace internal } // namespace compute diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index e6be9cd92e7..c3187a3995a 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -58,6 +58,11 @@ using internal::CpuInfo; namespace compute { +ExecContext* default_exec_context() { + static ExecContext default_ctx; + return &default_ctx; +} + ExecBatch::ExecBatch(const RecordBatch& batch) : values(batch.num_columns()), length(batch.num_rows()) { auto columns = batch.column_data(); diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index ff1fbfb4ee5..8bd28937d16 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -119,6 +119,8 @@ class ARROW_EXPORT ExecContext { bool use_threads_ = true; }; +ARROW_EXPORT ExecContext* default_exec_context(); + // TODO: Consider standardizing on uint16 selection vectors and only use them // when we can ensure that each value is 64K length or smaller diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 8cc8c11a270..c329b890356 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -342,7 +342,7 @@ struct GrouperImpl : Grouper { return std::move(impl); } - Result Consume(const ExecBatch& batch) override { + Result Consume(const ExecBatch& batch) override { std::vector offsets_batch(batch.length + 1); for (int i = 0; i < batch.num_values(); ++i) { encoders_[i]->AddLength(*batch[i].array(), offsets_batch.data()); @@ -391,11 +391,11 @@ struct GrouperImpl : Grouper { } ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); - return ExecBatch( - {UInt32Array(batch.length, std::move(group_ids)), Datum(num_groups_)}, - batch.length); + return Datum(UInt32Array(batch.length, std::move(group_ids))); } + uint32_t num_groups() const override { return num_groups_; } + Result GetUniques() override { ExecBatch out({}, num_groups_); @@ -868,20 +868,11 @@ Result ResolveKernels( Result> Grouper::Make(const std::vector& descrs, ExecContext* ctx) { - if (ctx == nullptr) { - static ExecContext default_ctx; - return Make(descrs, &default_ctx); - } return GrouperImpl::Make(descrs, ctx); } Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, ExecContext* ctx) { - if (ctx == nullptr) { - ExecContext default_ctx; - return GroupBy(arguments, keys, aggregates, &default_ctx); - } - // Construct and initialize HashAggregateKernels ARROW_ASSIGN_OR_RAISE(auto argument_descrs, ExecBatch::Make(arguments).Map( @@ -921,14 +912,14 @@ Result GroupBy(const std::vector& arguments, const std::vectorConsume(key_batch)); + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); // consume group ids with HashAggregateKernels for (size_t i = 0; i < kernels.size(); ++i) { KernelContext batch_ctx{ctx}; batch_ctx.SetState(states[i].get()); - ARROW_ASSIGN_OR_RAISE( - auto batch, ExecBatch::Make({argument_batch[i], id_batch[0], id_batch[1]})); + ARROW_ASSIGN_OR_RAISE(auto batch, ExecBatch::Make({argument_batch[i], id_batch, + Datum(grouper->num_groups())})); kernels[i]->consume(&batch_ctx, batch); if (batch_ctx.HasError()) return batch_ctx.status(); } @@ -958,22 +949,20 @@ Result GroupBy(const std::vector& arguments, const std::vector> ApplyGroupings(const ListArray& groupings, - const Array& array) { +Result> Grouper::ApplyGroupings(const ListArray& groupings, + const Array& array, + ExecContext* ctx) { ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(array, groupings.data()->child_data[0])); + compute::Take(array, groupings.data()->child_data[0], + TakeOptions::NoBoundsCheck(), ctx)); return std::make_shared(list(array.type()), groupings.length(), groupings.value_offsets(), sorted.make_array()); } -Result> MakeGroupings(const UInt32Array& ids, uint32_t max_id, - ExecContext* ctx) { - if (ctx == nullptr) { - ExecContext default_ctx; - return MakeGroupings(ids, max_id, &default_ctx); - } - +Result> Grouper::MakeGroupings(const UInt32Array& ids, + uint32_t max_id, + ExecContext* ctx) { if (ids.null_count() != 0) { return Status::Invalid("MakeGroupings with null ids"); } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index b0e7a52da8b..148848c7231 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -127,12 +127,11 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys ARROW_ASSIGN_OR_RAISE(auto grouper, internal::Grouper::Make(key_batch.GetDescriptors())); - ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); - int64_t num_groups = id_batch[1].scalar_as().value; ARROW_ASSIGN_OR_RAISE( - auto groupings, - internal::MakeGroupings(*id_batch[0].array_as(), num_groups - 1)); + auto groupings, internal::Grouper::MakeGroupings(*id_batch.array_as(), + grouper->num_groups() - 1)); ArrayVector out_columns; @@ -142,11 +141,11 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys ARROW_ASSIGN_OR_RAISE( auto grouped_argument, - internal::ApplyGroupings(*groupings, *arguments[i].make_array())); + internal::Grouper::ApplyGroupings(*groupings, *arguments[i].make_array())); ScalarVector aggregated_scalars; - for (int64_t i_group = 0; i_group < num_groups; ++i_group) { + for (int64_t i_group = 0; i_group < grouper->num_groups(); ++i_group) { auto slice = grouped_argument->value_slice(i_group); if (slice->length() == 0) continue; ARROW_ASSIGN_OR_RAISE( @@ -208,21 +207,19 @@ struct TestGrouper { } void ConsumeAndValidate(const ExecBatch& key_batch, Datum* ids = nullptr) { - ASSERT_OK_AND_ASSIGN(auto id_batch, grouper_->Consume(key_batch)); + ASSERT_OK_AND_ASSIGN(Datum id_batch, grouper_->Consume(key_batch)); ValidateConsume(key_batch, id_batch); if (ids) { - *ids = id_batch[0]; + *ids = std::move(id_batch); } } - void ValidateConsume(const ExecBatch& key_batch, const ExecBatch& id_batch) { - int64_t new_num_groups = id_batch[1].scalar_as().value; - + void ValidateConsume(const ExecBatch& key_batch, const Datum& id_batch) { if (uniques_.length == -1) { ASSERT_OK_AND_ASSIGN(uniques_, grouper_->GetUniques()); - } else if (new_num_groups > uniques_.length) { + } else if (static_cast(grouper_->num_groups()) > uniques_.length) { ASSERT_OK_AND_ASSIGN(ExecBatch new_uniques, grouper_->GetUniques()); // check that uniques_ are prefixes of new_uniques @@ -237,7 +234,7 @@ struct TestGrouper { // check that the ids encode an equivalent key sequence for (int i = 0; i < key_batch.num_values(); ++i) { SCOPED_TRACE(std::to_string(i) + "th key array"); - ASSERT_OK_AND_ASSIGN(auto expected, Take(uniques_[i], id_batch[0])); + ASSERT_OK_AND_ASSIGN(auto expected, Take(uniques_[i], id_batch)); AssertDatumsEqual(expected, key_batch[i], /*verbose=*/true); } } @@ -393,13 +390,13 @@ TEST(Grouper, RandomStringInt64Keys) { } } -TEST(GroupByHelpers, MakeGroupings) { +TEST(Grouper, MakeGroupings) { auto ExpectGroupings = [](std::string ids_json, uint32_t max_id, std::string expected_json) { auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), ids_json)); auto expected = ArrayFromJSON(list(int32()), expected_json); - ASSERT_OK_AND_ASSIGN(auto actual, internal::MakeGroupings(*ids, max_id)); + ASSERT_OK_AND_ASSIGN(auto actual, internal::Grouper::MakeGroupings(*ids, max_id)); AssertArraysEqual(*expected, *actual, /*verbose=*/true); }; diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 7fc4d7744a0..628dfc9e337 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -115,11 +115,11 @@ Result KeyValuePartitioning::Partition( ARROW_ASSIGN_OR_RAISE(auto grouper, compute::internal::Grouper::Make(key_batch.GetDescriptors())); - ARROW_ASSIGN_OR_RAISE(auto id_batch, grouper->Consume(key_batch)); + ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); - int64_t num_groups = id_batch[1].scalar_as().value; - auto ids = id_batch[0].array_as(); - ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::MakeGroupings( + int64_t num_groups = 0; + auto ids = id_batch.array_as(); + ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::Grouper::MakeGroupings( *ids, static_cast(num_groups - 1))); ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); From d0c01ac09694de0d6e662a6723e53d68d640c9eb Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 11:06:31 -0400 Subject: [PATCH 40/49] validate ApplyGroupings --- .../compute/kernels/hash_aggregate_test.cc | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 148848c7231..db998359006 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -390,6 +390,17 @@ TEST(Grouper, RandomStringInt64Keys) { } } +TEST(Grouper, RandomStringInt64DoubleInt32Keys) { + TestGrouper g({utf8(), int64(), float64(), int32()}); + for (int i = 0; i < 4; ++i) { + SCOPED_TRACE(std::to_string(i) + "th key batch"); + + ExecBatch key_batch{ + *random::GenerateBatch(g.key_schema_->fields(), 1 << 12, 0xDEADBEEF)}; + g.ConsumeAndValidate(key_batch); + } +} + TEST(Grouper, MakeGroupings) { auto ExpectGroupings = [](std::string ids_json, uint32_t max_id, std::string expected_json) { @@ -398,6 +409,17 @@ TEST(Grouper, MakeGroupings) { ASSERT_OK_AND_ASSIGN(auto actual, internal::Grouper::MakeGroupings(*ids, max_id)); AssertArraysEqual(*expected, *actual, /*verbose=*/true); + + // validate ApplyGroupings + ASSERT_OK_AND_ASSIGN(auto grouped_ids, + internal::Grouper::ApplyGroupings(*actual, *ids)); + + for (uint32_t id = 0; id <= max_id; ++id) { + auto ids_slice = checked_pointer_cast(grouped_ids->value_slice(id)); + for (auto slot : *ids_slice) { + EXPECT_EQ(slot, id); + } + } }; ExpectGroupings("[]", 0, "[[]]"); From d0006e951f092d5edc841bb626cbb50c4840a7c8 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 12:49:45 -0400 Subject: [PATCH 41/49] rewrite tests for readability, remove redundant cases --- .../compute/kernels/hash_aggregate_test.cc | 381 ++++++++---------- 1 file changed, 169 insertions(+), 212 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index db998359006..55bcf6edc8d 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -43,6 +43,7 @@ #include "arrow/testing/gtest_common.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/random.h" +#include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" namespace arrow { @@ -52,12 +53,6 @@ using internal::checked_cast; using internal::checked_pointer_cast; namespace compute { - -// Copy-pasta from partition.cc -// -// In the finished product this will only be a test helper for group_by -// and partition.cc will rely on a no-aggregate call to group_by. -namespace group_helpers { namespace { struct ScalarVectorToArray { @@ -65,7 +60,7 @@ struct ScalarVectorToArray { typename BuilderType = typename TypeTraits::BuilderType, typename ScalarType = typename TypeTraits::ScalarType> Status UseBuilder(const AppendScalar& append) { - BuilderType builder(type(), default_memory_pool()); + BuilderType builder(type_, default_memory_pool()); for (const auto& s : scalars_) { if (s->is_valid) { RETURN_NOT_OK(append(checked_cast(*s), &builder)); @@ -101,21 +96,38 @@ struct ScalarVectorToArray { return UseBuilder(AppendBuffer{}); } + Status Visit(const StructType& type) { + data_ = ArrayData::Make(type_, static_cast(scalars_.size()), + {/*null_bitmap=*/nullptr}); + ScalarVector field_scalars(scalars_.size()); + + for (int field_index = 0; field_index < type.num_fields(); ++field_index) { + for (size_t i = 0; i < scalars_.size(); ++i) { + field_scalars[i] = + checked_cast(scalars_[i].get())->value[field_index]; + } + + ARROW_ASSIGN_OR_RAISE(Datum field, ScalarVectorToArray{}.Convert(field_scalars)); + data_->child_data.push_back(field.array()); + } + return Status::OK(); + } + Status Visit(const DataType& type) { return Status::NotImplemented("ScalarVectorToArray for type ", type); } - Result Convert(ScalarVector scalars) && { + Result Convert(const ScalarVector& scalars) && { if (scalars.size() == 0) { return Status::NotImplemented("ScalarVectorToArray with no scalars"); } scalars_ = std::move(scalars); - RETURN_NOT_OK(VisitTypeInline(*type(), this)); + type_ = scalars_[0]->type; + RETURN_NOT_OK(VisitTypeInline(*type_, this)); return Datum(std::move(data_)); } - const std::shared_ptr& type() { return scalars_[0]->type; } - + std::shared_ptr type_; ScalarVector scalars_; std::shared_ptr data_; }; @@ -154,7 +166,7 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys } ARROW_ASSIGN_OR_RAISE(Datum aggregated_column, - ScalarVectorToArray{}.Convert(std::move(aggregated_scalars))); + ScalarVectorToArray{}.Convert(aggregated_scalars)); out_columns.push_back(aggregated_column.make_array()); } @@ -169,8 +181,7 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys void ValidateGroupBy(const std::vector& aggregates, std::vector arguments, std::vector keys) { - ASSERT_OK_AND_ASSIGN(Datum expected, - group_helpers::NaiveGroupBy(arguments, keys, aggregates)); + ASSERT_OK_AND_ASSIGN(Datum expected, NaiveGroupBy(arguments, keys, aggregates)); ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(arguments, keys, aggregates)); @@ -178,7 +189,6 @@ void ValidateGroupBy(const std::vector& aggregates, } } // namespace -} // namespace group_helpers struct TestGrouper { explicit TestGrouper(std::vector descrs) : descrs_(std::move(descrs)) { @@ -434,86 +444,24 @@ TEST(Grouper, MakeGroupings) { "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); } -TEST(GroupBy, SumOnlyBooleanKey) { - auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(boolean(), "[1, 0, 1, 0, null, 0, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, - { - {"hash_sum", nullptr}, - })); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", boolean()), - }), - R"([ - [1, true], - [3, false], - [0.875, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, SumOnly8bitKey) { - auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int8(), "[1, 2, 3, 1, 2, 2, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, - { - {"hash_sum", nullptr}, - })); - - ASSERT_OK(aggregated_and_grouped.array_as()->ValidateFull()); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int8()), - }), - R"([ - [4.25, 1], - [-0.125, 2], - [null, 3], - [0.75, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, SumOnly32bitKey) { - auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int32(), "[1, 2, 3, 1, 2, 2, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, - { - {"hash_sum", nullptr}, - })); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int32()), - }), - R"([ - [4.25, 1], - [-0.125, 2], - [null, 3], - [0.75, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - TEST(GroupBy, SumOnly) { - auto argument = - ArrayFromJSON(float64(), "[1.0, 0.0, null, 4.0, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 3, null, 1, 2, 2, null]"); + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("key", int64())}), R"([ + [1.0, 1], + [null, 1], + [0.0, 2], + [null, 3], + [4.0, null], + [3.25, 1], + [0.125, 2], + [-0.25, 2], + [0.75, null], + [null, 3] + ])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, + internal::GroupBy({batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}, { {"hash_sum", nullptr}, })); @@ -532,36 +480,24 @@ TEST(GroupBy, SumOnly) { /*verbose=*/true); } -TEST(GroupBy, SumOnlyFloatingPointKey) { - auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(float64(), "[1, 2, 3, 1, 2, 2, null]"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, - { - {"hash_sum", nullptr}, - })); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", float64()), - }), - R"([ - [4.25, 1], - [-0.125, 2], - [null, 3], - [0.75, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - TEST(GroupBy, MinMaxOnly) { - auto argument = ArrayFromJSON(float64(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 3, 1, 2, 2, null]"); + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("key", int64())}), R"([ + [1.0, 1], + [null, 1], + [0.0, 2], + [null, 3], + [4.0, null], + [3.25, 1], + [0.125, 2], + [-0.25, 2], + [0.75, null], + [null, 3] + ])"); ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, + internal::GroupBy({batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}, { {"hash_min_max", nullptr}, })); @@ -577,26 +513,45 @@ TEST(GroupBy, MinMaxOnly) { [{"min": 1.0, "max": 3.25}, 1], [{"min": -0.25, "max": 0.125}, 2], [{"min": null, "max": null}, 3], - [{"min": 0.75, "max": 0.75}, null] + [{"min": 0.75, "max": 4.0}, null] ])"), aggregated_and_grouped, /*verbose=*/true); } TEST(GroupBy, CountAndSum) { - auto argument = ArrayFromJSON(float32(), "[1.0, 0.0, null, 3.25, 0.125, -0.25, 0.75]"); - auto key = ArrayFromJSON(int64(), "[1, 2, 1, 3, 2, 3, null]"); + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("key", int64())}), R"([ + [1.0, 1], + [null, 1], + [0.0, 2], + [null, 3], + [4.0, null], + [3.25, 1], + [0.125, 2], + [-0.25, 2], + [0.75, null], + [null, 3] + ])"); CountOptions count_options; - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - // NB: passing an argument twice or also using it as a key is legal - internal::GroupBy({argument, argument, key}, {key}, - { - {"hash_count", &count_options}, - {"hash_sum", nullptr}, - {"hash_sum", nullptr}, - })); + ASSERT_OK_AND_ASSIGN( + Datum aggregated_and_grouped, + internal::GroupBy( + { + // NB: passing an argument twice or also using it as a key is legal + batch->GetColumnByName("argument"), + batch->GetColumnByName("argument"), + batch->GetColumnByName("key"), + }, + { + batch->GetColumnByName("key"), + }, + { + {"hash_count", &count_options}, + {"hash_sum", nullptr}, + {"hash_sum", nullptr}, + })); AssertDatumsEqual( ArrayFromJSON(struct_({ @@ -607,104 +562,106 @@ TEST(GroupBy, CountAndSum) { field("", int64()), }), R"([ - [1, 1.0, 2, 1], - [2, 0.125, 4, 2], - [2, 3.0, 6, 3], - [1, 0.75, null, null] + [2, 4.25, 3, 1], + [3, -0.125, 6, 2], + [0, null, 6, 3], + [2, 4.75, null, null] ])"), aggregated_and_grouped, /*verbose=*/true); } -TEST(GroupBy, StringKey) { - auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); - auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, {{"hash_sum", nullptr}})); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", int64()), - field("", utf8()), - }), - R"([ - [10, "alfa"], - [14, "beta"], - [6, "gamma"], - [12, null] +TEST(GroupBy, SumOnlyStringAndDictKeys) { + for (auto key_type : {utf8(), dictionary(int32(), utf8())}) { + SCOPED_TRACE("key type: " + key_type->ToString()); + + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("key", key_type)}), R"([ + [1.0, "alfa"], + [null, "alfa"], + [0.0, "beta"], + [null, "gama"], + [4.0, null ], + [3.25, "alfa"], + [0.125, "beta"], + [-0.25, "beta"], + [0.75, null ], + [null, "gama"] + ])"); + + ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, + internal::GroupBy({batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}, + { + {"hash_sum", nullptr}, + })); + + AssertDatumsEqual(ArrayFromJSON(struct_({ + field("", float64()), + field("", key_type), + }), + R"([ + [4.25, "alfa"], + [-0.125, "beta"], + [null, "gama"], + [4.75, null ] ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, DictKey) { - auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12, 9]"); - auto key = ArrayFromJSON(dictionary(int32(), utf8()), - R"(["alfa", "beta", "gamma", "gamma", null, "beta"])"); - - ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped, - internal::GroupBy({argument}, {key}, {{"hash_sum", nullptr}})); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", int64()), - field("", dictionary(int32(), utf8())), - }), - R"([ - [10, "alfa"], - [14, "beta"], - [6, "gamma"], - [12, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); -} - -TEST(GroupBy, MultipleKeys) { - auto argument = ArrayFromJSON(float32(), "[0.125, 0.5, -0.75, 8, 1.0, 2.0]"); - auto int_key = ArrayFromJSON(int32(), "[0, 1, 0, 1, 0, 1]"); - auto str_key = - ArrayFromJSON(utf8(), R"(["beta", "beta", "gamma", "gamma", null, "beta"])"); - - ASSERT_OK_AND_ASSIGN( - Datum aggregated_and_grouped, - internal::GroupBy({argument}, {int_key, str_key}, {{"hash_sum", nullptr}})); - - AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int32()), - field("", utf8()), - }), - R"([ - [0.125, 0, "beta"], - [2.5, 1, "beta"], - [-0.75, 0, "gamma"], - [8, 1, "gamma"], - [1.0, 0, null] - ])"), - aggregated_and_grouped, - /*verbose=*/true); + aggregated_and_grouped, + /*verbose=*/true); + } } TEST(GroupBy, ConcreteCaseWithValidateGroupBy) { - auto argument = ArrayFromJSON(int64(), "[10, 5, 4, 2, 12]"); - auto key = ArrayFromJSON(utf8(), R"(["alfa", "beta", "gamma", "gamma", "beta"])"); - - group_helpers::ValidateGroupBy({{"hash_sum", nullptr}}, {argument}, {key}); + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("key", utf8())}), R"([ + [1.0, "alfa"], + [null, "alfa"], + [0.0, "beta"], + [null, "gama"], + [4.0, null ], + [3.25, "alfa"], + [0.125, "beta"], + [-0.25, "beta"], + [0.75, null ], + [null, "gama"] + ])"); + + CountOptions count_non_null{CountOptions::COUNT_NON_NULL}, + count_null{CountOptions::COUNT_NULL}; + + MinMaxOptions emit_null{MinMaxOptions::EMIT_NULL}; + + using internal::Aggregate; + for (auto agg : { + Aggregate{"hash_sum", nullptr}, + Aggregate{"hash_count", &count_non_null}, + Aggregate{"hash_count", &count_null}, + Aggregate{"hash_min_max", nullptr}, + Aggregate{"hash_min_max", &emit_null}, + }) { + SCOPED_TRACE(agg.function); + ValidateGroupBy({agg}, {batch->GetColumnByName("argument")}, + {batch->GetColumnByName("key")}); + } } TEST(GroupBy, RandomArraySum) { - auto rand = random::RandomArrayGenerator(0xdeadbeef); - for (int64_t length : {1 << 10, 1 << 12, 1 << 15}) { - for (auto null_probability : {0.0, 0.1, 0.5, 1.0}) { - auto summand = rand.Float32(length, -100, 100, null_probability); - auto key = rand.Int64(length, 0, 12); + for (auto null_probability : {0.0, 0.01, 0.5, 1.0}) { + auto batch = random::GenerateBatch( + { + field("argument", float32(), + key_value_metadata( + {{"null_probability", std::to_string(null_probability)}})), + field("key", int64()), + }, + length, 0xDEADBEEF); - group_helpers::ValidateGroupBy( + ValidateGroupBy( { {"hash_sum", nullptr}, }, - {summand}, {key}); + {batch->GetColumnByName("argument")}, {batch->GetColumnByName("key")}); } } } From d43768352dbffce8995519ed0b6694636668633a Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 13:16:31 -0400 Subject: [PATCH 42/49] ~max_id~ -> num_groups --- cpp/src/arrow/compute/api_aggregate.h | 7 +++--- .../arrow/compute/kernels/hash_aggregate.cc | 14 +++++------ .../compute/kernels/hash_aggregate_test.cc | 25 +++++++++---------- cpp/src/arrow/dataset/partition.cc | 7 +++--- 4 files changed, 26 insertions(+), 27 deletions(-) diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 31e727632c8..de45894ceb8 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -331,8 +331,8 @@ class ARROW_EXPORT Grouper { /// /// \param[in] ids An unsigned, all-valid integral array which will be /// used as grouping criteria. - /// \param[in] max_id An upper bound for the elements of ids - /// \return A (max_id + 1)-long ListArray where the slot at i contains a + /// \param[in] num_groups An upper bound for the elements of ids + /// \return A num_groups-long ListArray where the slot at i contains a /// list of indices where i appears in ids. /// /// MakeGroupings([ @@ -353,7 +353,8 @@ class ARROW_EXPORT Grouper { /// [] /// ] static Result> MakeGroupings( - const UInt32Array& ids, uint32_t max_id, ExecContext* ctx = default_exec_context()); + const UInt32Array& ids, uint32_t num_groups, + ExecContext* ctx = default_exec_context()); /// \brief Produce a ListArray whose slots are selections of `array` which correspond to /// the provided groupings. diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index c329b890356..0c76d94c4a8 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -961,28 +961,28 @@ Result> Grouper::ApplyGroupings(const ListArray& grou } Result> Grouper::MakeGroupings(const UInt32Array& ids, - uint32_t max_id, + uint32_t num_groups, ExecContext* ctx) { if (ids.null_count() != 0) { return Status::Invalid("MakeGroupings with null ids"); } - ARROW_ASSIGN_OR_RAISE( - auto offsets, AllocateBuffer(sizeof(int32_t) * (max_id + 2), ctx->memory_pool())); + ARROW_ASSIGN_OR_RAISE(auto offsets, AllocateBuffer(sizeof(int32_t) * (num_groups + 1), + ctx->memory_pool())); auto raw_offsets = reinterpret_cast(offsets->mutable_data()); std::memset(raw_offsets, 0, offsets->size()); for (int i = 0; i < ids.length(); ++i) { - DCHECK_LE(ids.Value(i), max_id); + DCHECK_LT(ids.Value(i), num_groups); raw_offsets[ids.Value(i)] += 1; } int32_t length = 0; - for (uint32_t id = 0; id < max_id + 1; ++id) { + for (uint32_t id = 0; id < num_groups; ++id) { auto offset = raw_offsets[id]; raw_offsets[id] = length; length += offset; } - raw_offsets[max_id + 1] = length; + raw_offsets[num_groups] = length; DCHECK_EQ(ids.length(), length); ARROW_ASSIGN_OR_RAISE(auto offsets_copy, @@ -997,7 +997,7 @@ Result> Grouper::MakeGroupings(const UInt32Array& ids } return std::make_shared( - list(int32()), max_id + 1, std::move(offsets), + list(int32()), num_groups, std::move(offsets), std::make_shared(ids.length(), std::move(sort_indices))); } diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 55bcf6edc8d..1f82d7cdd0d 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -143,7 +143,7 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys ARROW_ASSIGN_OR_RAISE( auto groupings, internal::Grouper::MakeGroupings(*id_batch.array_as(), - grouper->num_groups() - 1)); + grouper->num_groups())); ArrayVector out_columns; @@ -412,36 +412,35 @@ TEST(Grouper, RandomStringInt64DoubleInt32Keys) { } TEST(Grouper, MakeGroupings) { - auto ExpectGroupings = [](std::string ids_json, uint32_t max_id, - std::string expected_json) { + auto ExpectGroupings = [](std::string ids_json, std::string expected_json) { auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), ids_json)); auto expected = ArrayFromJSON(list(int32()), expected_json); - ASSERT_OK_AND_ASSIGN(auto actual, internal::Grouper::MakeGroupings(*ids, max_id)); + auto num_groups = static_cast(expected->length()); + ASSERT_OK_AND_ASSIGN(auto actual, internal::Grouper::MakeGroupings(*ids, num_groups)); AssertArraysEqual(*expected, *actual, /*verbose=*/true); // validate ApplyGroupings ASSERT_OK_AND_ASSIGN(auto grouped_ids, internal::Grouper::ApplyGroupings(*actual, *ids)); - for (uint32_t id = 0; id <= max_id; ++id) { - auto ids_slice = checked_pointer_cast(grouped_ids->value_slice(id)); + for (uint32_t group = 0; group < num_groups; ++group) { + auto ids_slice = checked_pointer_cast(grouped_ids->value_slice(group)); for (auto slot : *ids_slice) { - EXPECT_EQ(slot, id); + EXPECT_EQ(slot, group); } } }; - ExpectGroupings("[]", 0, "[[]]"); + ExpectGroupings("[]", "[[]]"); - ExpectGroupings("[0, 0, 0]", 0, "[[0, 1, 2]]"); + ExpectGroupings("[0, 0, 0]", "[[0, 1, 2]]"); - ExpectGroupings("[0, 0, 0, 1, 1, 2]", 3, "[[0, 1, 2], [3, 4], [5], []]"); + ExpectGroupings("[0, 0, 0, 1, 1, 2]", "[[0, 1, 2], [3, 4], [5], []]"); - ExpectGroupings("[2, 1, 2, 1, 1, 2]", 4, "[[], [1, 3, 4], [0, 2, 5], [], []]"); + ExpectGroupings("[2, 1, 2, 1, 1, 2]", "[[], [1, 3, 4], [0, 2, 5], [], []]"); - ExpectGroupings("[2, 2, 5, 5, 2, 3]", 7, - "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); + ExpectGroupings("[2, 2, 5, 5, 2, 3]", "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); } TEST(GroupBy, SumOnly) { diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 628dfc9e337..43ccd777cf2 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -117,10 +117,9 @@ Result KeyValuePartitioning::Partition( ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); - int64_t num_groups = 0; auto ids = id_batch.array_as(); ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::Grouper::MakeGroupings( - *ids, static_cast(num_groups - 1))); + *ids, grouper->num_groups())); ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); ArrayVector unique_arrays(num_keys); @@ -131,8 +130,8 @@ Result KeyValuePartitioning::Partition( PartitionedBatches out; // assemble partition expressions from the unique keys - out.expressions.resize(static_cast(num_groups)); - for (int64_t group = 0; group < num_groups; ++group) { + out.expressions.resize(grouper->num_groups()); + for (uint32_t group = 0; group < grouper->num_groups(); ++group) { std::vector exprs(num_keys); for (int i = 0; i < num_keys; ++i) { From a07fc665eda3fc270111037e65e4529cf486767b Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 13:23:43 -0400 Subject: [PATCH 43/49] rename FunctionDoc vars to enable unity build --- .../arrow/compute/kernels/hash_aggregate.cc | 39 ++++++++++--------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 0c76d94c4a8..59b022f817a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -1002,35 +1002,36 @@ Result> Grouper::MakeGroupings(const UInt32Array& ids } namespace { -const FunctionDoc count_doc{"Count the number of null / non-null values", - ("By default, non-null values are counted.\n" - "This can be changed through CountOptions."), - {"array", "group_id_array", "group_count"}, - "CountOptions"}; - -const FunctionDoc sum_doc{"Sum values of a numeric array", - ("Null values are ignored."), - {"array", "group_id_array", "group_count"}}; - -const FunctionDoc min_max_doc{"Compute the minimum and maximum values of a numeric array", - ("Null values are ignored by default.\n" - "This can be changed through MinMaxOptions."), - {"array", "group_id_array", "group_count"}, - "MinMaxOptions"}; +const FunctionDoc hash_count_doc{"Count the number of null / non-null values", + ("By default, non-null values are counted.\n" + "This can be changed through CountOptions."), + {"array", "group_id_array", "group_count"}, + "CountOptions"}; + +const FunctionDoc hash_sum_doc{"Sum values of a numeric array", + ("Null values are ignored."), + {"array", "group_id_array", "group_count"}}; + +const FunctionDoc hash_min_max_doc{ + "Compute the minimum and maximum values of a numeric array", + ("Null values are ignored by default.\n" + "This can be changed through MinMaxOptions."), + {"array", "group_id_array", "group_count"}, + "MinMaxOptions"}; } // namespace void RegisterHashAggregateBasic(FunctionRegistry* registry) { { static auto default_count_options = CountOptions::Defaults(); auto func = std::make_shared( - "hash_count", Arity::Ternary(), &count_doc, &default_count_options); + "hash_count", Arity::Ternary(), &hash_count_doc, &default_count_options); DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); DCHECK_OK(registry->AddFunction(std::move(func))); } { - auto func = - std::make_shared("hash_sum", Arity::Ternary(), &sum_doc); + auto func = std::make_shared("hash_sum", Arity::Ternary(), + &hash_sum_doc); DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); DCHECK_OK(registry->AddFunction(std::move(func))); } @@ -1038,7 +1039,7 @@ void RegisterHashAggregateBasic(FunctionRegistry* registry) { { static auto default_minmax_options = MinMaxOptions::Defaults(); auto func = std::make_shared( - "hash_min_max", Arity::Ternary(), &min_max_doc, &default_minmax_options); + "hash_min_max", Arity::Ternary(), &hash_min_max_doc, &default_minmax_options); DCHECK_OK(func->AddKernel(MakeKernel(ValueDescr::ARRAY))); DCHECK_OK(registry->AddFunction(std::move(func))); } From 201fa3b599a42a51aa65958cff381890f897df01 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 13:54:04 -0400 Subject: [PATCH 44/49] Expose HashAggregateKernel in python, ensure hash_agg funcs can't Execute --- cpp/src/arrow/compute/function.cc | 4 +- .../compute/kernels/hash_aggregate_test.cc | 63 ++++++++++++++++++- python/pyarrow/_compute.pyx | 56 +++++++++++++++++ python/pyarrow/includes/libarrow.pxd | 11 ++++ 4 files changed, 131 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/function.cc b/cpp/src/arrow/compute/function.cc index 815537b4bb9..c8fc8b8dec0 100644 --- a/cpp/src/arrow/compute/function.cc +++ b/cpp/src/arrow/compute/function.cc @@ -189,8 +189,10 @@ Result Function::Execute(const std::vector& args, executor = detail::KernelExecutor::MakeScalar(); } else if (kind() == Function::VECTOR) { executor = detail::KernelExecutor::MakeVector(); - } else { + } else if (kind() == Function::SCALAR_AGGREGATE) { executor = detail::KernelExecutor::MakeScalarAggregate(); + } else { + return Status::NotImplemented("Direct execution of HASH_AGGREGATE functions"); } RETURN_NOT_OK(executor->Init(&kernel_ctx, {kernel, inputs, options})); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 1f82d7cdd0d..5451753f315 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -190,6 +190,41 @@ void ValidateGroupBy(const std::vector& aggregates, } // namespace +TEST(Grouper, SupportedKeys) { + ASSERT_OK(internal::Grouper::Make({boolean()})); + + ASSERT_OK(internal::Grouper::Make({int8(), uint16(), int32(), uint64()})); + + ASSERT_OK(internal::Grouper::Make({dictionary(int64(), utf8())})); + + ASSERT_OK(internal::Grouper::Make({float16(), float32(), float64()})); + + ASSERT_OK(internal::Grouper::Make({utf8(), binary(), large_utf8(), large_binary()})); + + ASSERT_OK(internal::Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)})); + + ASSERT_OK(internal::Grouper::Make({decimal128(32, 10), decimal256(76, 20)})); + + ASSERT_OK(internal::Grouper::Make({date32(), date64()})); + + for (auto unit : internal::AllTimeUnits()) { + ASSERT_OK(internal::Grouper::Make({timestamp(unit), duration(unit)})); + } + + ASSERT_OK(internal::Grouper::Make({day_time_interval(), month_interval()})); + + ASSERT_RAISES(NotImplemented, internal::Grouper::Make({struct_({field("", int64())})})); + + ASSERT_RAISES(NotImplemented, internal::Grouper::Make({struct_({})})); + + ASSERT_RAISES(NotImplemented, internal::Grouper::Make({list(int32())})); + + ASSERT_RAISES(NotImplemented, internal::Grouper::Make({fixed_size_list(int32(), 5)})); + + ASSERT_RAISES(NotImplemented, + internal::Grouper::Make({dense_union({field("", int32())})})); +} + struct TestGrouper { explicit TestGrouper(std::vector descrs) : descrs_(std::move(descrs)) { grouper_ = internal::Grouper::Make(descrs_).ValueOrDie(); @@ -286,7 +321,7 @@ TEST(Grouper, NumericKey) { } TEST(Grouper, StringKey) { - for (auto ty : {utf8(), large_utf8()}) { + for (auto ty : {utf8(), large_utf8(), fixed_size_binary(2)}) { SCOPED_TRACE("key type: " + ty->ToString()); TestGrouper g({ty}); @@ -295,7 +330,7 @@ TEST(Grouper, StringKey) { g.ExpectConsume(R"([["eh"], ["eh"]])", "[0, 0]"); - g.ExpectConsume(R"([["bee"], [null]])", "[1, 2]"); + g.ExpectConsume(R"([["be"], [null]])", "[1, 2]"); } } @@ -441,6 +476,30 @@ TEST(Grouper, MakeGroupings) { ExpectGroupings("[2, 1, 2, 1, 1, 2]", "[[], [1, 3, 4], [0, 2, 5], [], []]"); ExpectGroupings("[2, 2, 5, 5, 2, 3]", "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); + + auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), "[0, null, 1]")); + ASSERT_RAISES(Invalid, internal::Grouper::MakeGroupings(*ids, 5)); +} + +TEST(GroupBy, Errors) { + auto batch = RecordBatchFromJSON( + schema({field("argument", float64()), field("group_id", int32())}), R"([ + [1.0, 1], + [null, 1], + [0.0, 2], + [null, 3], + [4.0, 0], + [3.25, 1], + [0.125, 2], + [-0.25, 2], + [0.75, 0], + [null, 3] + ])"); + + ASSERT_RAISES( + NotImplemented, + CallFunction("hash_sum", {batch->GetColumnByName("argument"), + batch->GetColumnByName("group_id"), Datum(uint32_t(4))})); } TEST(GroupBy, SumOnly) { diff --git a/python/pyarrow/_compute.pyx b/python/pyarrow/_compute.pyx index 3cb152aa381..044cec9b7d4 100644 --- a/python/pyarrow/_compute.pyx +++ b/python/pyarrow/_compute.pyx @@ -58,6 +58,17 @@ cdef wrap_scalar_aggregate_function(const shared_ptr[CFunction]& sp_func): return func +cdef wrap_hash_aggregate_function(const shared_ptr[CFunction]& sp_func): + """ + Wrap a C++ aggregate Function in a ScalarAggregateFunction object. + """ + cdef HashAggregateFunction func = ( + HashAggregateFunction.__new__(HashAggregateFunction) + ) + func.init(sp_func) + return func + + cdef wrap_meta_function(const shared_ptr[CFunction]& sp_func): """ Wrap a C++ meta Function in a MetaFunction object. @@ -85,6 +96,8 @@ cdef wrap_function(const shared_ptr[CFunction]& sp_func): return wrap_vector_function(sp_func) elif c_kind == FunctionKind_SCALAR_AGGREGATE: return wrap_scalar_aggregate_function(sp_func) + elif c_kind == FunctionKind_HASH_AGGREGATE: + return wrap_hash_aggregate_function(sp_func) elif c_kind == FunctionKind_META: return wrap_meta_function(sp_func) else: @@ -117,6 +130,16 @@ cdef wrap_scalar_aggregate_kernel(const CScalarAggregateKernel* c_kernel): return kernel +cdef wrap_hash_aggregate_kernel(const CHashAggregateKernel* c_kernel): + if c_kernel == NULL: + raise ValueError('Kernel was NULL') + cdef HashAggregateKernel kernel = ( + HashAggregateKernel.__new__(HashAggregateKernel) + ) + kernel.init(c_kernel) + return kernel + + cdef class Kernel(_Weakrefable): """ A kernel object. @@ -165,6 +188,18 @@ cdef class ScalarAggregateKernel(Kernel): .format(frombytes(self.kernel.signature.get().ToString()))) +cdef class HashAggregateKernel(Kernel): + cdef: + const CHashAggregateKernel* kernel + + cdef void init(self, const CHashAggregateKernel* kernel) except *: + self.kernel = kernel + + def __repr__(self): + return ("HashAggregateKernel<{}>" + .format(frombytes(self.kernel.signature.get().ToString()))) + + FunctionDoc = namedtuple( "FunctionDoc", ("summary", "description", "arg_names", "options_class")) @@ -249,6 +284,8 @@ cdef class Function(_Weakrefable): return 'vector' elif c_kind == FunctionKind_SCALAR_AGGREGATE: return 'scalar_aggregate' + elif c_kind == FunctionKind_HASH_AGGREGATE: + return 'hash_aggregate' elif c_kind == FunctionKind_META: return 'meta' else: @@ -351,6 +388,25 @@ cdef class ScalarAggregateFunction(Function): return [wrap_scalar_aggregate_kernel(k) for k in kernels] +cdef class HashAggregateFunction(Function): + cdef: + const CHashAggregateFunction* func + + cdef void init(self, const shared_ptr[CFunction]& sp_func) except *: + Function.init(self, sp_func) + self.func = sp_func.get() + + @property + def kernels(self): + """ + The kernels implementing this function. + """ + cdef vector[const CHashAggregateKernel*] kernels = ( + self.func.kernels() + ) + return [wrap_hash_aggregate_kernel(k) for k in kernels] + + cdef class MetaFunction(Function): cdef: const CMetaFunction* func diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd index 9afe4d1e720..61deb658b0c 100644 --- a/python/pyarrow/includes/libarrow.pxd +++ b/python/pyarrow/includes/libarrow.pxd @@ -1729,6 +1729,10 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil: " arrow::compute::ScalarAggregateKernel"(CKernel): pass + cdef cppclass CHashAggregateKernel \ + " arrow::compute::HashAggregateKernel"(CKernel): + pass + cdef cppclass CArity" arrow::compute::Arity": int num_args c_bool is_varargs @@ -1738,6 +1742,8 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil: FunctionKind_VECTOR" arrow::compute::Function::VECTOR" FunctionKind_SCALAR_AGGREGATE \ " arrow::compute::Function::SCALAR_AGGREGATE" + FunctionKind_HASH_AGGREGATE \ + " arrow::compute::Function::HASH_AGGREGATE" FunctionKind_META \ " arrow::compute::Function::META" @@ -1771,6 +1777,11 @@ cdef extern from "arrow/compute/api.h" namespace "arrow::compute" nogil: (CFunction): vector[const CScalarAggregateKernel*] kernels() const + cdef cppclass CHashAggregateFunction\ + " arrow::compute::HashAggregateFunction"\ + (CFunction): + vector[const CHashAggregateKernel*] kernels() const + cdef cppclass CMetaFunction" arrow::compute::MetaFunction"(CFunction): pass From 5fa524eee56ad85b1d9652d858ca20f2813aad24 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 15:06:52 -0400 Subject: [PATCH 45/49] __dllexport ExecBatch --- cpp/src/arrow/compute/exec.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/compute/exec.h b/cpp/src/arrow/compute/exec.h index 8bd28937d16..7659442d8bf 100644 --- a/cpp/src/arrow/compute/exec.h +++ b/cpp/src/arrow/compute/exec.h @@ -166,7 +166,7 @@ class ARROW_EXPORT SelectionVector { /// TODO: Datum uses arrow/util/variant.h which may be a bit heavier-weight /// than is desirable for this class. Microbenchmarks would help determine for /// sure. See ARROW-8928. -struct ExecBatch { +struct ARROW_EXPORT ExecBatch { ExecBatch() = default; ExecBatch(std::vector values, int64_t length) : values(std::move(values)), length(length) {} From 46b40693694ed4f474d1726c01172445a15f00c9 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 15:09:37 -0400 Subject: [PATCH 46/49] BufferBuilder* bytes_builder() --- cpp/src/arrow/buffer_builder.h | 4 ++-- cpp/src/arrow/compute/kernels/hash_aggregate.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/buffer_builder.h b/cpp/src/arrow/buffer_builder.h index 53761f80c96..f525ec23c58 100644 --- a/cpp/src/arrow/buffer_builder.h +++ b/cpp/src/arrow/buffer_builder.h @@ -211,7 +211,7 @@ class TypedBufferBuilder< explicit TypedBufferBuilder(BufferBuilder builder) : bytes_builder_(std::move(builder)) {} - BufferBuilder& bytes_builder() { return bytes_builder_; } + BufferBuilder* bytes_builder() { return &bytes_builder_; } Status Append(T value) { return bytes_builder_.Append(reinterpret_cast(&value), sizeof(T)); @@ -294,7 +294,7 @@ class TypedBufferBuilder { explicit TypedBufferBuilder(BufferBuilder builder) : bytes_builder_(std::move(builder)) {} - BufferBuilder& bytes_builder() { return bytes_builder_; } + BufferBuilder* bytes_builder() { return &bytes_builder_; } Status Append(bool value) { ARROW_RETURN_NOT_OK(Reserve(1)); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index 59b022f817a..db6785ed79f 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -637,7 +637,7 @@ struct GroupedMinMaxImpl : public GroupedAggregator { return [anti_extreme](BufferBuilder* builder, int64_t added_groups) { TypedBufferBuilder typed_builder(std::move(*builder)); RETURN_NOT_OK(typed_builder.Append(added_groups, anti_extreme)); - *builder = std::move(typed_builder.bytes_builder()); + *builder = std::move(*typed_builder.bytes_builder()); return Status::OK(); }; } From 5b79c32803b43e159143749b277a3d45d4d28caa Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 16:41:23 -0400 Subject: [PATCH 47/49] review comments --- cpp/src/arrow/compare.h | 2 +- cpp/src/arrow/compute/api_aggregate.h | 4 +- cpp/src/arrow/compute/kernel.h | 1 + .../compute/kernels/aggregate_benchmark.cc | 4 +- .../arrow/compute/kernels/hash_aggregate.cc | 41 ++-- .../compute/kernels/hash_aggregate_test.cc | 178 +++++++----------- cpp/src/arrow/testing/generator.cc | 84 +++++++++ cpp/src/arrow/testing/generator.h | 3 + cpp/src/arrow/testing/gtest_util.cc | 13 +- cpp/src/arrow/testing/gtest_util.h | 10 +- python/pyarrow/_compute.pyx | 10 +- python/pyarrow/tests/test_compute.py | 5 + 12 files changed, 211 insertions(+), 144 deletions(-) diff --git a/cpp/src/arrow/compare.h b/cpp/src/arrow/compare.h index 387105de9e7..6769b23867b 100644 --- a/cpp/src/arrow/compare.h +++ b/cpp/src/arrow/compare.h @@ -71,7 +71,7 @@ class EqualOptions { return res; } - static EqualOptions Defaults() { return EqualOptions(); } + static EqualOptions Defaults() { return {}; } protected: double atol_ = kDefaultAbsoluteTolerance; diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index de45894ceb8..ca118ec5678 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -319,6 +319,8 @@ class ARROW_EXPORT Grouper { ExecContext* ctx = default_exec_context()); /// Consume a batch of keys, producing the corresponding group ids as an integer array. + /// Currently only uint32 indices will be produced, eventually the bit width will only + /// be as wide as necessary. virtual Result Consume(const ExecBatch& batch) = 0; /// Get current unique keys. May be called multiple times. @@ -342,7 +344,7 @@ class ARROW_EXPORT Grouper { /// 5, /// 2, /// 3 - /// ], 7) == [ + /// ], 8) == [ /// [], /// [], /// [0, 1, 4], diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index d3866853e0d..b99b41170d2 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -723,6 +723,7 @@ using HashAggregateFinalize = std::function; /// * consume: processes an ExecBatch (which includes the argument as well /// as an array of group identifiers) and updates the KernelState found in the /// KernelContext. +/// * merge: combines one KernelState with another. /// * finalize: produces the end result of the aggregation using the /// KernelState in the KernelContext. struct HashAggregateKernel : public Kernel { diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index 80ab037586c..1c34d12db1f 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -334,7 +334,7 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { /*min_length=*/3, /*max_length=*/32); - BenchmarkGroupBy(state, {{"sum", NULLPTR}}, {summand}, {key}); + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); }); GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { @@ -348,7 +348,7 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { /*min=*/0, /*max=*/15); - BenchmarkGroupBy(state, {{"sum", NULLPTR}}, {summand}, {key}); + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); }); // diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index db6785ed79f..d9750cb4760 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -47,6 +47,9 @@ struct KeyEncoder { // the first byte of an encoded key is used to indicate nullity static constexpr bool kExtraByteForNull = true; + static constexpr uint8_t kNullByte = 1; + static constexpr uint8_t kValidByte = 0; + virtual ~KeyEncoder() = default; virtual void AddLength(const ArrayData&, int32_t* lengths) = 0; @@ -58,19 +61,19 @@ struct KeyEncoder { // extract the null bitmap from the leading nullity bytes of encoded keys static Status DecodeNulls(MemoryPool* pool, int32_t length, uint8_t** encoded_bytes, - std::shared_ptr* null_buf, int32_t* null_count) { + std::shared_ptr* null_bitmap, int32_t* null_count) { // first count nulls to determine if a null bitmap is necessary *null_count = 0; for (int32_t i = 0; i < length; ++i) { - *null_count += encoded_bytes[i][0]; + *null_count += (encoded_bytes[i][0] == kNullByte); } if (*null_count > 0) { - ARROW_ASSIGN_OR_RAISE(*null_buf, AllocateBitmap(length, pool)); + ARROW_ASSIGN_OR_RAISE(*null_bitmap, AllocateBitmap(length, pool)); - uint8_t* nulls = (*null_buf)->mutable_data(); + uint8_t* validity = (*null_bitmap)->mutable_data(); for (int32_t i = 0; i < length; ++i) { - BitUtil::SetBitTo(nulls, i, !encoded_bytes[i][0]); + BitUtil::SetBitTo(validity, i, encoded_bytes[i][0] == kValidByte); encoded_bytes[i] += 1; } } else { @@ -96,12 +99,12 @@ struct BooleanKeyEncoder : KeyEncoder { data, [&](bool value) { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; + *encoded_ptr++ = kValidByte; *encoded_ptr++ = value; }, [&] { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; + *encoded_ptr++ = kNullByte; *encoded_ptr++ = 0; }); return Status::OK(); @@ -146,13 +149,13 @@ struct FixedWidthKeyEncoder : KeyEncoder { viewed, [&](util::string_view bytes) { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; + *encoded_ptr++ = kValidByte; memcpy(encoded_ptr, bytes.data(), byte_width_); encoded_ptr += byte_width_; }, [&] { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; + *encoded_ptr++ = kNullByte; memset(encoded_ptr, 0, byte_width_); encoded_ptr += byte_width_; }); @@ -191,8 +194,9 @@ struct DictionaryKeyEncoder : FixedWidthKeyEncoder { auto dict = MakeArray(data.dictionary); if (dictionary_) { if (!dictionary_->Equals(dict)) { - // TODO(bkietz) unify if necessary - return Status::NotImplemented("Dictionary keys with multiple dictionaries"); + // TODO(bkietz) unify if necessary. For now, just error if any batch's dictionary + // differs from the first we saw for this key + return Status::NotImplemented("Unifying differing dictionaries"); } } else { dictionary_ = std::move(dict); @@ -240,7 +244,7 @@ struct VarLengthKeyEncoder : KeyEncoder { data, [&](util::string_view bytes) { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 0; + *encoded_ptr++ = kValidByte; util::SafeStore(encoded_ptr, static_cast(bytes.size())); encoded_ptr += sizeof(Offset); memcpy(encoded_ptr, bytes.data(), bytes.size()); @@ -248,7 +252,7 @@ struct VarLengthKeyEncoder : KeyEncoder { }, [&] { auto& encoded_ptr = *encoded_bytes++; - *encoded_ptr++ = 1; + *encoded_ptr++ = kNullByte; util::SafeStore(encoded_ptr, static_cast(0)); encoded_ptr += sizeof(Offset); }); @@ -844,6 +848,7 @@ Result>> InitKernels( } Result ResolveKernels( + const std::vector& aggregates, const std::vector& kernels, const std::vector>& states, ExecContext* ctx, const std::vector& descrs) { @@ -859,7 +864,7 @@ Result ResolveKernels( uint32(), uint32(), })); - fields[i] = field("", std::move(descr.type)); + fields[i] = field(aggregates[i].function, std::move(descr.type)); } return fields; } @@ -883,8 +888,9 @@ Result GroupBy(const std::vector& arguments, const std::vector GroupBy(const std::vector& arguments, const std::vector::BuilderType, - typename ScalarType = typename TypeTraits::ScalarType> - Status UseBuilder(const AppendScalar& append) { - BuilderType builder(type_, default_memory_pool()); - for (const auto& s : scalars_) { - if (s->is_valid) { - RETURN_NOT_OK(append(checked_cast(*s), &builder)); - } else { - RETURN_NOT_OK(builder.AppendNull()); - } - } - return builder.FinishInternal(&data_); - } - - struct AppendValue { - template - Status operator()(const ScalarType& s, BuilderType* builder) const { - return builder->Append(s.value); - } - }; - - struct AppendBuffer { - template - Status operator()(const ScalarType& s, BuilderType* builder) const { - const Buffer& buffer = *s.value; - return builder->Append(util::string_view{buffer}); - } - }; - - template - enable_if_primitive_ctype Visit(const T&) { - return UseBuilder(AppendValue{}); - } - - template - enable_if_has_string_view Visit(const T&) { - return UseBuilder(AppendBuffer{}); - } - - Status Visit(const StructType& type) { - data_ = ArrayData::Make(type_, static_cast(scalars_.size()), - {/*null_bitmap=*/nullptr}); - ScalarVector field_scalars(scalars_.size()); - - for (int field_index = 0; field_index < type.num_fields(); ++field_index) { - for (size_t i = 0; i < scalars_.size(); ++i) { - field_scalars[i] = - checked_cast(scalars_[i].get())->value[field_index]; - } - - ARROW_ASSIGN_OR_RAISE(Datum field, ScalarVectorToArray{}.Convert(field_scalars)); - data_->child_data.push_back(field.array()); - } - return Status::OK(); - } - - Status Visit(const DataType& type) { - return Status::NotImplemented("ScalarVectorToArray for type ", type); - } - - Result Convert(const ScalarVector& scalars) && { - if (scalars.size() == 0) { - return Status::NotImplemented("ScalarVectorToArray with no scalars"); - } - scalars_ = std::move(scalars); - type_ = scalars_[0]->type; - RETURN_NOT_OK(VisitTypeInline(*type_, this)); - return Datum(std::move(data_)); - } - - std::shared_ptr type_; - ScalarVector scalars_; - std::shared_ptr data_; -}; - Result NaiveGroupBy(std::vector arguments, std::vector keys, const std::vector& aggregates) { ARROW_ASSIGN_OR_RAISE(auto key_batch, ExecBatch::Make(std::move(keys))); @@ -146,8 +71,11 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys grouper->num_groups())); ArrayVector out_columns; + std::vector out_names; for (size_t i = 0; i < arguments.size(); ++i) { + out_names.push_back(aggregates[i].function); + // trim "hash_" prefix auto scalar_agg_function = aggregates[i].function.substr(5); @@ -166,16 +94,17 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys } ARROW_ASSIGN_OR_RAISE(Datum aggregated_column, - ScalarVectorToArray{}.Convert(aggregated_scalars)); + ScalarVectorToArray(aggregated_scalars)); out_columns.push_back(aggregated_column.make_array()); } + int i = 0; ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); for (const Datum& key : uniques.values) { out_columns.push_back(key.make_array()); + out_names.push_back("key_" + std::to_string(i++)); } - std::vector out_names(out_columns.size(), ""); return StructArray::Make(std::move(out_columns), std::move(out_names)); } @@ -185,6 +114,9 @@ void ValidateGroupBy(const std::vector& aggregates, ASSERT_OK_AND_ASSIGN(Datum actual, GroupBy(arguments, keys, aggregates)); + ASSERT_OK(expected.make_array()->ValidateFull()); + ASSERT_OK(actual.make_array()->ValidateFull()); + AssertDatumsEqual(expected, actual, /*verbose=*/true); } @@ -269,18 +201,26 @@ struct TestGrouper { // check that uniques_ are prefixes of new_uniques for (int i = 0; i < uniques_.num_values(); ++i) { - auto prefix = new_uniques[i].array()->Slice(0, uniques_.length); - AssertDatumsEqual(uniques_[i], prefix, /*verbose=*/true); + auto new_unique = new_uniques[i].make_array(); + ASSERT_OK(new_unique->ValidateFull()); + + AssertDatumsEqual(uniques_[i], new_unique->Slice(0, uniques_.length), + /*verbose=*/true); } uniques_ = std::move(new_uniques); } // check that the ids encode an equivalent key sequence + auto ids = id_batch.make_array(); + ASSERT_OK(ids->ValidateFull()); + for (int i = 0; i < key_batch.num_values(); ++i) { SCOPED_TRACE(std::to_string(i) + "th key array"); - ASSERT_OK_AND_ASSIGN(auto expected, Take(uniques_[i], id_batch)); - AssertDatumsEqual(expected, key_batch[i], /*verbose=*/true); + auto original = key_batch[i].make_array(); + ASSERT_OK_AND_ASSIGN(auto encoded, Take(*uniques_[i].make_array(), *ids)); + AssertArraysEqual(*original, *encoded, /*verbose=*/true, + EqualOptions().nans_equal(true)); } } @@ -320,6 +260,20 @@ TEST(Grouper, NumericKey) { } } +TEST(Grouper, FloatingPointKey) { + TestGrouper g({float32()}); + + // -0.0 hashes differently from 0.0 + g.ExpectConsume("[[0.0], [-0.0]]", "[0, 1]"); + + g.ExpectConsume("[[Inf], [-Inf]]", "[2, 3]"); + + // assert(!(NaN == NaN)) does not cause spurious new groups + g.ExpectConsume("[[NaN], [NaN]]", "[4, 4]"); + + // TODO(bkietz) test denormal numbers, more NaNs +} + TEST(Grouper, StringKey) { for (auto ty : {utf8(), large_utf8(), fixed_size_binary(2)}) { SCOPED_TRACE("key type: " + ty->ToString()); @@ -337,7 +291,9 @@ TEST(Grouper, StringKey) { TEST(Grouper, DictKey) { TestGrouper g({dictionary(int32(), utf8())}); - // unification of dictionaries on encode is not yet supported + // For dictionary keys, all batches must share a single dictionary. + // Eventually, differing dictionaries will be unified and indices transposed + // during encoding to relieve this restriction. const auto dict = ArrayFromJSON(utf8(), R"(["ex", "why", "zee", null])"); auto WithIndices = [&](const std::string& indices) { @@ -356,10 +312,11 @@ TEST(Grouper, DictKey) { g.ExpectConsume({WithIndices(" [3, 1, null, 0, 2]")}, ArrayFromJSON(uint32(), "[3, 1, 4, 0, 2]")); - ASSERT_RAISES(NotImplemented, - g.grouper_->Consume(*ExecBatch::Make({*DictionaryArray::FromArrays( - ArrayFromJSON(int32(), "[0, 1]"), - ArrayFromJSON(utf8(), R"(["different", "dictionary"])"))}))); + EXPECT_RAISES_WITH_MESSAGE_THAT( + NotImplemented, HasSubstr("Unifying differing dictionaries"), + g.grouper_->Consume(*ExecBatch::Make({*DictionaryArray::FromArrays( + ArrayFromJSON(int32(), "[0, 1]"), + ArrayFromJSON(utf8(), R"(["different", "dictionary"])"))}))); } TEST(Grouper, StringInt64Key) { @@ -478,12 +435,13 @@ TEST(Grouper, MakeGroupings) { ExpectGroupings("[2, 2, 5, 5, 2, 3]", "[[], [], [0, 1, 4], [5], [], [2, 3], [], []]"); auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), "[0, null, 1]")); - ASSERT_RAISES(Invalid, internal::Grouper::MakeGroupings(*ids, 5)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("MakeGroupings with null ids"), + internal::Grouper::MakeGroupings(*ids, 5)); } TEST(GroupBy, Errors) { auto batch = RecordBatchFromJSON( - schema({field("argument", float64()), field("group_id", int32())}), R"([ + schema({field("argument", float64()), field("group_id", uint32())}), R"([ [1.0, 1], [null, 1], [0.0, 2], @@ -496,8 +454,8 @@ TEST(GroupBy, Errors) { [null, 3] ])"); - ASSERT_RAISES( - NotImplemented, + EXPECT_RAISES_WITH_MESSAGE_THAT( + NotImplemented, HasSubstr("Direct execution of HASH_AGGREGATE functions"), CallFunction("hash_sum", {batch->GetColumnByName("argument"), batch->GetColumnByName("group_id"), Datum(uint32_t(4))})); } @@ -525,8 +483,8 @@ TEST(GroupBy, SumOnly) { })); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", int64()), + field("hash_sum", float64()), + field("key_0", int64()), }), R"([ [4.25, 1], @@ -561,11 +519,11 @@ TEST(GroupBy, MinMaxOnly) { })); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", struct_({ - field("min", float64()), - field("max", float64()), - })), - field("", int64()), + field("hash_min_max", struct_({ + field("min", float64()), + field("max", float64()), + })), + field("key_0", int64()), }), R"([ [{"min": 1.0, "max": 3.25}, 1], @@ -613,11 +571,11 @@ TEST(GroupBy, CountAndSum) { AssertDatumsEqual( ArrayFromJSON(struct_({ - field("", int64()), + field("hash_count", int64()), // NB: summing a float32 array results in float64 sums - field("", float64()), - field("", int64()), - field("", int64()), + field("hash_sum", float64()), + field("hash_sum", int64()), + field("key_0", int64()), }), R"([ [2, 4.25, 3, 1], @@ -655,8 +613,8 @@ TEST(GroupBy, SumOnlyStringAndDictKeys) { })); AssertDatumsEqual(ArrayFromJSON(struct_({ - field("", float64()), - field("", key_type), + field("hash_sum", float64()), + field("key_0", key_type), }), R"([ [4.25, "alfa"], @@ -711,7 +669,7 @@ TEST(GroupBy, RandomArraySum) { field("argument", float32(), key_value_metadata( {{"null_probability", std::to_string(null_probability)}})), - field("key", int64()), + field("key", int64(), key_value_metadata({{"min", "0"}, {"max", "100"}})), }, length, 0xDEADBEEF); diff --git a/cpp/src/arrow/testing/generator.cc b/cpp/src/arrow/testing/generator.cc index 41c1f752160..71fad394d00 100644 --- a/cpp/src/arrow/testing/generator.cc +++ b/cpp/src/arrow/testing/generator.cc @@ -95,4 +95,88 @@ std::shared_ptr ConstantArrayGenerator::String(int64_t size, return ConstantArray(size, value); } +struct ScalarVectorToArrayImpl { + template ::BuilderType, + typename ScalarType = typename TypeTraits::ScalarType> + Status UseBuilder(const AppendScalar& append) { + BuilderType builder(type_, default_memory_pool()); + for (const auto& s : scalars_) { + if (s->is_valid) { + RETURN_NOT_OK(append(internal::checked_cast(*s), &builder)); + } else { + RETURN_NOT_OK(builder.AppendNull()); + } + } + return builder.FinishInternal(&data_); + } + + struct AppendValue { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + return builder->Append(s.value); + } + }; + + struct AppendBuffer { + template + Status operator()(const ScalarType& s, BuilderType* builder) const { + const Buffer& buffer = *s.value; + return builder->Append(util::string_view{buffer}); + } + }; + + template + enable_if_primitive_ctype Visit(const T&) { + return UseBuilder(AppendValue{}); + } + + template + enable_if_has_string_view Visit(const T&) { + return UseBuilder(AppendBuffer{}); + } + + Status Visit(const StructType& type) { + data_ = ArrayData::Make(type_, static_cast(scalars_.size()), + {/*null_bitmap=*/nullptr}); + data_->child_data.resize(type_->num_fields()); + + ScalarVector field_scalars(scalars_.size()); + + for (int field_index = 0; field_index < type.num_fields(); ++field_index) { + for (size_t i = 0; i < scalars_.size(); ++i) { + field_scalars[i] = + internal::checked_cast(scalars_[i].get())->value[field_index]; + } + + ARROW_ASSIGN_OR_RAISE(data_->child_data[field_index], + ScalarVectorToArrayImpl{}.Convert(field_scalars)); + } + return Status::OK(); + } + + Status Visit(const DataType& type) { + return Status::NotImplemented("ScalarVectorToArray for type ", type); + } + + Result> Convert(const ScalarVector& scalars) && { + if (scalars.size() == 0) { + return Status::NotImplemented("ScalarVectorToArray with no scalars"); + } + scalars_ = std::move(scalars); + type_ = scalars_[0]->type; + RETURN_NOT_OK(VisitTypeInline(*type_, this)); + return std::move(data_); + } + + std::shared_ptr type_; + ScalarVector scalars_; + std::shared_ptr data_; +}; + +Result> ScalarVectorToArray(const ScalarVector& scalars) { + ARROW_ASSIGN_OR_RAISE(auto data, ScalarVectorToArrayImpl{}.Convert(scalars)); + return MakeArray(std::move(data)); +} + } // namespace arrow diff --git a/cpp/src/arrow/testing/generator.h b/cpp/src/arrow/testing/generator.h index 9188dca5709..b9f531466bc 100644 --- a/cpp/src/arrow/testing/generator.h +++ b/cpp/src/arrow/testing/generator.h @@ -255,4 +255,7 @@ class ARROW_TESTING_EXPORT ConstantArrayGenerator { } }; +ARROW_EXPORT +Result> ScalarVectorToArray(const ScalarVector& scalars); + } // namespace arrow diff --git a/cpp/src/arrow/testing/gtest_util.cc b/cpp/src/arrow/testing/gtest_util.cc index 462a5237921..1e366539cbe 100644 --- a/cpp/src/arrow/testing/gtest_util.cc +++ b/cpp/src/arrow/testing/gtest_util.cc @@ -131,20 +131,21 @@ void AssertArraysEqualWith(const Array& expected, const Array& actual, bool verb } } -void AssertArraysEqual(const Array& expected, const Array& actual, bool verbose) { +void AssertArraysEqual(const Array& expected, const Array& actual, bool verbose, + const EqualOptions& options) { return AssertArraysEqualWith( expected, actual, verbose, - [](const Array& expected, const Array& actual, std::stringstream* diff) { - return expected.Equals(actual, EqualOptions().diff_sink(diff)); + [&](const Array& expected, const Array& actual, std::stringstream* diff) { + return expected.Equals(actual, options.diff_sink(diff)); }); } void AssertArraysApproxEqual(const Array& expected, const Array& actual, bool verbose, - const EqualOptions& option) { + const EqualOptions& options) { return AssertArraysEqualWith( expected, actual, verbose, - [&option](const Array& expected, const Array& actual, std::stringstream* diff) { - return expected.ApproxEquals(actual, option.diff_sink(diff)); + [&](const Array& expected, const Array& actual, std::stringstream* diff) { + return expected.ApproxEquals(actual, options.diff_sink(diff)); }); } diff --git a/cpp/src/arrow/testing/gtest_util.h b/cpp/src/arrow/testing/gtest_util.h index 634b09af834..ee71e7af913 100644 --- a/cpp/src/arrow/testing/gtest_util.h +++ b/cpp/src/arrow/testing/gtest_util.h @@ -212,10 +212,12 @@ std::vector AllTypeIds(); // If verbose is true, then the arrays will be pretty printed ARROW_TESTING_EXPORT void AssertArraysEqual(const Array& expected, const Array& actual, - bool verbose = false); -ARROW_TESTING_EXPORT void AssertArraysApproxEqual( - const Array& expected, const Array& actual, bool verbose = false, - const EqualOptions& option = EqualOptions::Defaults()); + bool verbose = false, + const EqualOptions& options = {}); +ARROW_TESTING_EXPORT void AssertArraysApproxEqual(const Array& expected, + const Array& actual, + bool verbose = false, + const EqualOptions& options = {}); // Returns true when values are both null ARROW_TESTING_EXPORT void AssertScalarsEqual( const Scalar& expected, const Scalar& actual, bool verbose = false, diff --git a/python/pyarrow/_compute.pyx b/python/pyarrow/_compute.pyx index 044cec9b7d4..f3a8eb860d4 100644 --- a/python/pyarrow/_compute.pyx +++ b/python/pyarrow/_compute.pyx @@ -60,7 +60,7 @@ cdef wrap_scalar_aggregate_function(const shared_ptr[CFunction]& sp_func): cdef wrap_hash_aggregate_function(const shared_ptr[CFunction]& sp_func): """ - Wrap a C++ aggregate Function in a ScalarAggregateFunction object. + Wrap a C++ aggregate Function in a HashAggregateFunction object. """ cdef HashAggregateFunction func = ( HashAggregateFunction.__new__(HashAggregateFunction) @@ -225,8 +225,12 @@ cdef class Function(_Weakrefable): in each input. Examples: dictionary encoding, sorting, extracting unique values... - * "aggregate" functions reduce the dimensionality of the inputs by - applying a reduction function. Examples: sum, minmax, mode... + * "scalar_aggregate" functions reduce the dimensionality of the inputs by + applying a reduction function. Examples: sum, min_max, mode... + + * "hash_aggregate" functions apply a reduction function to an input + subdivided by grouping criteria. They may not be directly called. + Examples: hash_sum, hash_min_max... * "meta" functions dispatch to other functions. """ diff --git a/python/pyarrow/tests/test_compute.py b/python/pyarrow/tests/test_compute.py index 673c1387c47..1b0d5f76d47 100644 --- a/python/pyarrow/tests/test_compute.py +++ b/python/pyarrow/tests/test_compute.py @@ -134,6 +134,11 @@ def test_get_function_aggregate(): pc.ScalarAggregateKernel, 8) +def test_get_function_aggregate(): + _check_get_function("hash_sum", pc.HashAggregateFunction, + pc.HashAggregateKernel, 1) + + def test_call_function_with_memory_pool(): arr = pa.array(["foo", "bar", "baz"]) indices = np.array([2, 2, 1]) From 2bff0088fb53938e44eaaada13336065482f39d4 Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Mon, 22 Mar 2021 17:09:36 -0400 Subject: [PATCH 48/49] add more benchmarks --- .../compute/kernels/aggregate_benchmark.cc | 116 +++++++++++++++++- cpp/src/arrow/testing/generator.h | 2 +- 2 files changed, 115 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc index 1c34d12db1f..42be0c36544 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_benchmark.cc @@ -322,7 +322,7 @@ static void BenchmarkGroupBy(benchmark::State& state, BenchmarkSetArgsWithSizes(bench, {1 * 1024 * 1024}); \ }) -GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { +GROUP_BY_BENCHMARK(SumDoublesGroupedByTinyStringSet, [&] { auto summand = rng.Float64(args.size, /*min=*/0.0, /*max=*/1.0e14, @@ -337,7 +337,37 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); }); -GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { +GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallStringSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.StringWithRepeats(args.size, + /*unique=*/256, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedByMediumStringSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.StringWithRepeats(args.size, + /*unique=*/4096, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedByTinyIntegerSet, [&] { auto summand = rng.Float64(args.size, /*min=*/0.0, /*max=*/1.0e14, @@ -351,6 +381,88 @@ GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); }); +GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntegerSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.Int64(args.size, + /*min=*/0, + /*max=*/255); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedByMediumIntegerSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto key = rng.Int64(args.size, + /*min=*/0, + /*max=*/4095); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedByTinyIntStringPairSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto int_key = rng.Int64(args.size, + /*min=*/0, + /*max=*/4); + auto str_key = rng.StringWithRepeats(args.size, + /*unique=*/4, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {int_key, str_key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedBySmallIntStringPairSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto int_key = rng.Int64(args.size, + /*min=*/0, + /*max=*/15); + auto str_key = rng.StringWithRepeats(args.size, + /*unique=*/16, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {int_key, str_key}); +}); + +GROUP_BY_BENCHMARK(SumDoublesGroupedByMediumIntStringPairSet, [&] { + auto summand = rng.Float64(args.size, + /*min=*/0.0, + /*max=*/1.0e14, + /*null_probability=*/args.null_proportion, + /*nan_probability=*/args.null_proportion / 10); + + auto int_key = rng.Int64(args.size, + /*min=*/0, + /*max=*/63); + auto str_key = rng.StringWithRepeats(args.size, + /*unique=*/64, + /*min_length=*/3, + /*max_length=*/32); + + BenchmarkGroupBy(state, {{"hash_sum", NULLPTR}}, {summand}, {int_key, str_key}); +}); + // // Sum // diff --git a/cpp/src/arrow/testing/generator.h b/cpp/src/arrow/testing/generator.h index b9f531466bc..c300022432a 100644 --- a/cpp/src/arrow/testing/generator.h +++ b/cpp/src/arrow/testing/generator.h @@ -255,7 +255,7 @@ class ARROW_TESTING_EXPORT ConstantArrayGenerator { } }; -ARROW_EXPORT +ARROW_TESTING_EXPORT Result> ScalarVectorToArray(const ScalarVector& scalars); } // namespace arrow From 147fe1fba0ea2d147e883d95ef8245e3058e9e4a Mon Sep 17 00:00:00 2001 From: Benjamin Kietzman Date: Tue, 23 Mar 2021 09:29:17 -0400 Subject: [PATCH 49/49] rename python tests, export HashAgg*, don't ref codegen_internal in tests --- .../compute/kernels/hash_aggregate_test.cc | 21 +++++++++++++++++-- python/pyarrow/compute.py | 2 ++ python/pyarrow/tests/test_compute.py | 4 ++-- 3 files changed, 23 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 985a6689825..7858d8bb147 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -139,7 +139,12 @@ TEST(Grouper, SupportedKeys) { ASSERT_OK(internal::Grouper::Make({date32(), date64()})); - for (auto unit : internal::AllTimeUnits()) { + for (auto unit : { + TimeUnit::SECOND, + TimeUnit::MILLI, + TimeUnit::MICRO, + TimeUnit::NANO, + }) { ASSERT_OK(internal::Grouper::Make({timestamp(unit), duration(unit)})); } @@ -244,7 +249,19 @@ TEST(Grouper, BooleanKey) { } TEST(Grouper, NumericKey) { - for (auto ty : internal::NumericTypes()) { + for (auto ty : { + uint8(), + int8(), + uint16(), + int16(), + uint32(), + int32(), + uint64(), + int64(), + float16(), + float32(), + float64(), + }) { SCOPED_TRACE("key type: " + ty->ToString()); TestGrouper g({ty}); diff --git a/python/pyarrow/compute.py b/python/pyarrow/compute.py index 3d7f5ecb4c3..2cdd843d81a 100644 --- a/python/pyarrow/compute.py +++ b/python/pyarrow/compute.py @@ -19,6 +19,8 @@ Function, FunctionOptions, FunctionRegistry, + HashAggregateFunction, + HashAggregateKernel, Kernel, ScalarAggregateFunction, ScalarAggregateKernel, diff --git a/python/pyarrow/tests/test_compute.py b/python/pyarrow/tests/test_compute.py index 1b0d5f76d47..112629fc702 100644 --- a/python/pyarrow/tests/test_compute.py +++ b/python/pyarrow/tests/test_compute.py @@ -129,12 +129,12 @@ def test_get_function_vector(): _check_get_function("unique", pc.VectorFunction, pc.VectorKernel, 8) -def test_get_function_aggregate(): +def test_get_function_scalar_aggregate(): _check_get_function("mean", pc.ScalarAggregateFunction, pc.ScalarAggregateKernel, 8) -def test_get_function_aggregate(): +def test_get_function_hash_aggregate(): _check_get_function("hash_sum", pc.HashAggregateFunction, pc.HashAggregateKernel, 1)